Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Logsdb and source only snapshots. #122199

Merged
merged 22 commits into from
Feb 13, 2025
Merged
Show file tree
Hide file tree
Changes from 8 commits
Commits
Show all changes
22 commits
Select commit Hold shift + click to select a range
624a365
Logsdb and source only snapshots.
martijnvg Feb 10, 2025
04c4bb1
suppress forbidden api usage
martijnvg Feb 10, 2025
a06803a
Merge remote-tracking branch 'es/main' into logsdb_search_only_snapshots
martijnvg Feb 10, 2025
385e821
Update docs/changelog/122199.yaml
martijnvg Feb 10, 2025
62334b3
removed unneeded annotation
martijnvg Feb 10, 2025
6730822
Now that SFM doesn't get serialized the parser needs trigger validati…
martijnvg Feb 10, 2025
cd9beed
iter tests
martijnvg Feb 11, 2025
e7c36da
Merge remote-tracking branch 'es/main' into logsdb_search_only_snapshots
martijnvg Feb 11, 2025
f3c6a9d
update mapping asserts to account for empty _source:{} in bwc cases
martijnvg Feb 11, 2025
d4777d5
disable assertion jvm arg work around
martijnvg Feb 11, 2025
43f9d5d
Revert changes to SFM's mode attribute initializer and instead disabl…
martijnvg Feb 11, 2025
f4bf08b
Merge remote-tracking branch 'es/main' into logsdb_search_only_snapshots
martijnvg Feb 11, 2025
0e3d2f7
Merge remote-tracking branch 'es/main' into logsdb_search_only_snapshots
martijnvg Feb 12, 2025
e2addaf
added comment
martijnvg Feb 12, 2025
29df0bc
hard code parameters
martijnvg Feb 12, 2025
371229a
applied test feedback
martijnvg Feb 12, 2025
d1019f8
[CI] Auto commit changes from spotless
elasticsearchmachine Feb 12, 2025
09e3988
iter
martijnvg Feb 12, 2025
44ceea7
remove redundant test code
martijnvg Feb 12, 2025
25ef316
Merge remote-tracking branch 'es/main' into logsdb_search_only_snapshots
martijnvg Feb 13, 2025
297eccd
Merge remote-tracking branch 'es/main' into logsdb_search_only_snapshots
martijnvg Feb 13, 2025
cad4a03
Merge remote-tracking branch 'es/main' into logsdb_search_only_snapshots
martijnvg Feb 13, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions docs/changelog/122199.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
pr: 122199
summary: Logsdb and source only snapshots
area: Logs
type: bug
issues: []
12 changes: 12 additions & 0 deletions server/src/main/java/org/elasticsearch/common/lucene/Lucene.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FilterCodecReader;
import org.apache.lucene.index.FilterDirectoryReader;
import org.apache.lucene.index.FilterLeafReader;
Expand Down Expand Up @@ -190,14 +192,24 @@ public static SegmentInfos pruneUnreferencedFiles(String segmentsFileName, Direc
throw new IllegalStateException("no commit found in the directory");
}
}
String parentField = null;
final IndexCommit cp = getIndexCommit(si, directory);
try (var reader = DirectoryReader.open(cp)) {
var topLevelFieldInfos = FieldInfos.getMergedFieldInfos(reader);
for (FieldInfo fieldInfo : topLevelFieldInfos) {
if (fieldInfo.isParentField()) {
parentField = fieldInfo.getName();
}
}
}
try (
IndexWriter writer = new IndexWriter(
directory,
indexWriterConfigWithNoMerging(Lucene.STANDARD_ANALYZER).setSoftDeletesField(Lucene.SOFT_DELETES_FIELD)
.setIndexCommit(cp)
.setCommitOnClose(false)
.setOpenMode(IndexWriterConfig.OpenMode.APPEND)
.setParentField(parentField)
)
) {
// do nothing and close this will kick off IndexFileDeleter which will remove all pending files
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -232,7 +232,8 @@ public IndexService(
mapperMetrics
);
this.indexFieldData = new IndexFieldDataService(indexSettings, indicesFieldDataCache, circuitBreakerService);
if (indexSettings.getIndexSortConfig().hasIndexSort()) {
boolean sourceOnly = Boolean.parseBoolean(indexSettings.getSettings().get("index.source_only"));
if (indexSettings.getIndexSortConfig().hasIndexSort() && sourceOnly == false) {
// we delay the actual creation of the sort order for this index because the mapping has not been merged yet.
// The sort order is validated right after the merge of the mapping later in the process.
this.indexSortSupplier = () -> indexSettings.getIndexSortConfig()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -176,7 +176,8 @@ public Builder(
true,
() -> null,
(n, c, o) -> Mode.valueOf(o.toString().toUpperCase(Locale.ROOT)),
m -> toType(m).enabled.explicit() ? null : toType(m).mode,
// Avoid initializing _source.mode if it doesn't need to be serialized:
m -> toType(m).enabled.explicit() ? null : toType(m).serializeMode ? toType(m).mode : null,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: let's add parentheses to denote the sequence of the conditional branches.

(b, n, v) -> b.field(n, v.toString().toLowerCase(Locale.ROOT)),
v -> v.toString().toLowerCase(Locale.ROOT)
).setMergeValidator((previous, current, conflicts) -> (previous == current) || current != Mode.STORED)
Expand Down Expand Up @@ -300,10 +301,11 @@ private static SourceFieldMapper resolveStaticInstance(final Mode sourceMode) {
if (indexMode == IndexMode.STANDARD && settingSourceMode == Mode.STORED) {
return DEFAULT;
}
SourceFieldMapper sourceFieldMapper = null;
if (onOrAfterDeprecateModeVersion(c.indexVersionCreated())) {
return resolveStaticInstance(settingSourceMode);
sourceFieldMapper = resolveStaticInstance(settingSourceMode);
} else {
return new SourceFieldMapper(
sourceFieldMapper = new SourceFieldMapper(
settingSourceMode,
Explicit.IMPLICIT_TRUE,
Strings.EMPTY_ARRAY,
Expand All @@ -312,6 +314,10 @@ private static SourceFieldMapper resolveStaticInstance(final Mode sourceMode) {
c.indexVersionCreated().onOrAfter(IndexVersions.SOURCE_MAPPER_MODE_ATTRIBUTE_NOOP)
);
}
// By default no attributes are specified and so the Builder doesn't get used.
// Need to validate the returned instance based on index mode:
indexMode.validateSourceFieldMapper(sourceFieldMapper);
return sourceFieldMapper;
},
c -> new Builder(
c.getIndexSettings().getMode(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -267,14 +267,14 @@ public void testSyntheticSourceInTimeSeries() throws IOException {
});
DocumentMapper mapper = createTimeSeriesModeDocumentMapper(mapping);
assertTrue(mapper.sourceMapper().isSynthetic());
assertEquals("{\"_source\":{}}", mapper.sourceMapper().toString());
assertEquals("{}", mapper.sourceMapper().toString());
}

public void testSyntheticSourceWithLogsIndexMode() throws IOException {
XContentBuilder mapping = fieldMapping(b -> { b.field("type", "keyword"); });
DocumentMapper mapper = createLogsModeDocumentMapper(mapping);
assertTrue(mapper.sourceMapper().isSynthetic());
assertEquals("{\"_source\":{}}", mapper.sourceMapper().toString());
assertEquals("{}", mapper.sourceMapper().toString());
}

public void testSupportsNonDefaultParameterValues() throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocValuesSkipIndexType;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
Expand Down Expand Up @@ -252,7 +253,7 @@ private SegmentCommitInfo syncSegment(
false,
IndexOptions.NONE,
DocValuesType.NONE,
fieldInfo.docValuesSkipIndexType(),
DocValuesSkipIndexType.NONE,
-1,
fieldInfo.attributes(),
0,
Expand Down
Loading