diff --git a/docs/changelog/114618.yaml b/docs/changelog/114618.yaml new file mode 100644 index 0000000000000..ada402fe35742 --- /dev/null +++ b/docs/changelog/114618.yaml @@ -0,0 +1,5 @@ +pr: 114618 +summary: Add a new index setting to skip recovery source when synthetic source is enabled +area: Logs +type: enhancement +issues: [] diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/CloneIndexIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/CloneIndexIT.java index b6930d06c11ec..deb191b4f1ce2 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/CloneIndexIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/CloneIndexIT.java @@ -15,6 +15,7 @@ import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.IndexVersions; import org.elasticsearch.index.query.TermsQueryBuilder; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.test.ESIntegTestCase; @@ -26,6 +27,7 @@ import static org.elasticsearch.action.admin.indices.create.ShrinkIndexIT.assertNoResizeSourceIndexSettings; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -143,6 +145,51 @@ public void testResizeChangeSyntheticSource() { assertThat(error.getMessage(), containsString("can't change setting [index.mapping.source.mode] during resize")); } + public void testResizeChangeRecoveryUseSyntheticSource() { + prepareCreate("source").setSettings( + indexSettings(between(1, 5), 0).put("index.mode", "logsdb") + .put( + "index.version.created", + IndexVersionUtils.randomVersionBetween( + random(), + IndexVersions.USE_SYNTHETIC_SOURCE_FOR_RECOVERY_BACKPORT, + IndexVersion.current() + ) + ) + ).setMapping("@timestamp", "type=date", "host.name", "type=keyword").get(); + updateIndexSettings(Settings.builder().put("index.blocks.write", true), "source"); + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, () -> { + indicesAdmin().prepareResizeIndex("source", "target") + .setResizeType(ResizeType.CLONE) + .setSettings( + Settings.builder() + .put( + "index.version.created", + IndexVersionUtils.randomVersionBetween( + random(), + IndexVersions.USE_SYNTHETIC_SOURCE_FOR_RECOVERY_BACKPORT, + IndexVersion.current() + ) + ) + .put("index.recovery.use_synthetic_source", true) + .put("index.mode", "logsdb") + .putNull("index.blocks.write") + .build() + ) + .get(); + }); + // The index.recovery.use_synthetic_source setting requires either index.mode or index.mapping.source.mode + // to be present in the settings. Since these are all unmodifiable settings with a non-deterministic evaluation + // order, any of them may trigger a failure first. + assertThat( + error.getMessage(), + anyOf( + containsString("can't change setting [index.mode] during resize"), + containsString("can't change setting [index.recovery.use_synthetic_source] during resize") + ) + ); + } + public void testResizeChangeIndexSorts() { prepareCreate("source").setSettings(indexSettings(between(1, 5), 0)) .setMapping("@timestamp", "type=date", "host.name", "type=keyword") diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/shard/IndexShardIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/shard/IndexShardIT.java index 6ffd5808cea73..870947db5bd85 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -715,7 +715,15 @@ public void testShardChangesWithDefaultDocType() throws Exception { } IndexShard shard = indexService.getShard(0); try ( - Translog.Snapshot luceneSnapshot = shard.newChangesSnapshot("test", 0, numOps - 1, true, randomBoolean(), randomBoolean()); + Translog.Snapshot luceneSnapshot = shard.newChangesSnapshot( + "test", + 0, + numOps - 1, + true, + randomBoolean(), + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ); Translog.Snapshot translogSnapshot = getTranslog(shard).newSnapshot() ) { List opsFromLucene = TestTranslog.drainSnapshot(luceneSnapshot, true); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index 61cf49ff6ca4e..6c5b34e3fdc45 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -147,7 +147,6 @@ import static org.elasticsearch.action.support.ActionTestUtils.assertNoFailureListener; import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; -import static org.elasticsearch.node.RecoverySettingsChunkSizePlugin.CHUNK_SIZE_SETTING; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.empty; @@ -247,7 +246,7 @@ private void assertOnGoingRecoveryState( public Settings.Builder createRecoverySettingsChunkPerSecond(long chunkSizeBytes) { return Settings.builder() // Set the chunk size in bytes - .put(CHUNK_SIZE_SETTING.getKey(), new ByteSizeValue(chunkSizeBytes, ByteSizeUnit.BYTES)) + .put(RecoverySettings.INDICES_RECOVERY_CHUNK_SIZE.getKey(), new ByteSizeValue(chunkSizeBytes, ByteSizeUnit.BYTES)) // Set one chunk of bytes per second. .put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), chunkSizeBytes, ByteSizeUnit.BYTES); } @@ -270,7 +269,7 @@ private void unthrottleRecovery() { Settings.builder() // 200mb is an arbitrary number intended to be large enough to avoid more throttling. .put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), "200mb") - .put(CHUNK_SIZE_SETTING.getKey(), RecoverySettings.DEFAULT_CHUNK_SIZE) + .put(RecoverySettings.INDICES_RECOVERY_CHUNK_SIZE.getKey(), RecoverySettings.DEFAULT_CHUNK_SIZE) ); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/recovery/TruncatedRecoveryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/recovery/TruncatedRecoveryIT.java index 38eef4f720623..ca2ff69ac9b17 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/recovery/TruncatedRecoveryIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/recovery/TruncatedRecoveryIT.java @@ -24,7 +24,7 @@ import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest; import org.elasticsearch.indices.recovery.RecoveryFilesInfoRequest; -import org.elasticsearch.node.RecoverySettingsChunkSizePlugin; +import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.transport.MockTransportService; @@ -41,7 +41,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; -import static org.elasticsearch.node.RecoverySettingsChunkSizePlugin.CHUNK_SIZE_SETTING; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -52,7 +51,7 @@ public class TruncatedRecoveryIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return Arrays.asList(MockTransportService.TestPlugin.class, RecoverySettingsChunkSizePlugin.class); + return Arrays.asList(MockTransportService.TestPlugin.class); } /** @@ -63,7 +62,11 @@ protected Collection> nodePlugins() { */ public void testCancelRecoveryAndResume() throws Exception { updateClusterSettings( - Settings.builder().put(CHUNK_SIZE_SETTING.getKey(), new ByteSizeValue(randomIntBetween(50, 300), ByteSizeUnit.BYTES)) + Settings.builder() + .put( + RecoverySettings.INDICES_RECOVERY_CHUNK_SIZE.getKey(), + new ByteSizeValue(randomIntBetween(50, 300), ByteSizeUnit.BYTES) + ) ); NodesStatsResponse nodeStats = clusterAdmin().prepareNodesStats().get(); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RestoreSnapshotIT.java index fe83073eeb780..5b54a0dbbb799 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RestoreSnapshotIT.java @@ -800,6 +800,24 @@ public void testRestoreChangeSyntheticSource() { assertThat(error.getMessage(), containsString("cannot modify setting [index.mapping.source.mode] on restore")); } + public void testRestoreChangeRecoveryUseSyntheticSource() { + Client client = client(); + createRepository("test-repo", "fs"); + String indexName = "test-idx"; + assertAcked(client.admin().indices().prepareCreate(indexName).setSettings(Settings.builder().put(indexSettings()))); + createSnapshot("test-repo", "test-snap", Collections.singletonList(indexName)); + cluster().wipeIndices(indexName); + var error = expectThrows(SnapshotRestoreException.class, () -> { + client.admin() + .cluster() + .prepareRestoreSnapshot(TEST_REQUEST_TIMEOUT, "test-repo", "test-snap") + .setIndexSettings(Settings.builder().put("index.recovery.use_synthetic_source", true)) + .setWaitForCompletion(true) + .get(); + }); + assertThat(error.getMessage(), containsString("cannot modify setting [index.recovery.use_synthetic_source] on restore")); + } + public void testRestoreChangeIndexSorts() { Client client = client(); createRepository("test-repo", "fs"); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java index 1f014a526b9a6..c1ba2bd710931 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java @@ -1572,6 +1572,7 @@ static void validateCloneIndex( private static final Set UNMODIFIABLE_SETTINGS_DURING_RESIZE = Set.of( IndexSettings.MODE.getKey(), SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.getKey(), + IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey(), IndexSortConfig.INDEX_SORT_FIELD_SETTING.getKey(), IndexSortConfig.INDEX_SORT_ORDER_SETTING.getKey(), IndexSortConfig.INDEX_SORT_MODE_SETTING.getKey(), diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 4b4529fdb8aa2..837e3314d9833 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -257,6 +257,7 @@ public void apply(Settings value, Settings current, Settings previous) { RecoverySettings.INDICES_RECOVERY_USE_SNAPSHOTS_SETTING, RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS, RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS_PER_NODE, + RecoverySettings.INDICES_RECOVERY_CHUNK_SIZE, RecoverySettings.NODE_BANDWIDTH_RECOVERY_FACTOR_READ_SETTING, RecoverySettings.NODE_BANDWIDTH_RECOVERY_FACTOR_WRITE_SETTING, RecoverySettings.NODE_BANDWIDTH_RECOVERY_OPERATOR_FACTOR_SETTING, diff --git a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java index 9b085c65ae8d8..da4b10b8296d6 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -189,6 +189,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IgnoredSourceFieldMapper.SKIP_IGNORED_SOURCE_WRITE_SETTING, IgnoredSourceFieldMapper.SKIP_IGNORED_SOURCE_READ_SETTING, SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING, + IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING, // validate that built-in similarities don't get redefined Setting.groupSetting("index.similarity.", (s) -> { diff --git a/server/src/main/java/org/elasticsearch/index/IndexSettings.java b/server/src/main/java/org/elasticsearch/index/IndexSettings.java index 254d7155d6865..751b194af3d50 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/server/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -38,6 +38,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -52,6 +53,7 @@ import static org.elasticsearch.index.mapper.MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING; import static org.elasticsearch.index.mapper.MapperService.INDEX_MAPPING_NESTED_FIELDS_LIMIT_SETTING; import static org.elasticsearch.index.mapper.MapperService.INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING; +import static org.elasticsearch.index.mapper.SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING; /** * This class encapsulates all index level settings and handles settings updates. @@ -656,6 +658,62 @@ public Iterator> settings() { Property.Final ); + public static final Setting RECOVERY_USE_SYNTHETIC_SOURCE_SETTING = Setting.boolSetting( + "index.recovery.use_synthetic_source", + false, + new Setting.Validator<>() { + @Override + public void validate(Boolean value) {} + + @Override + public void validate(Boolean enabled, Map, Object> settings) { + if (enabled == false) { + return; + } + + // Verify if synthetic source is enabled on the index; fail if it is not + var indexMode = (IndexMode) settings.get(MODE); + if (indexMode.defaultSourceMode() != SourceFieldMapper.Mode.SYNTHETIC) { + var sourceMode = (SourceFieldMapper.Mode) settings.get(INDEX_MAPPER_SOURCE_MODE_SETTING); + if (sourceMode != SourceFieldMapper.Mode.SYNTHETIC) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "The setting [%s] is only permitted when [%s] is set to [%s]. Current mode: [%s].", + RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey(), + INDEX_MAPPER_SOURCE_MODE_SETTING.getKey(), + SourceFieldMapper.Mode.SYNTHETIC.name(), + sourceMode.name() + ) + ); + } + } + + // Verify that all nodes can handle this setting + var version = (IndexVersion) settings.get(SETTING_INDEX_VERSION_CREATED); + if (version.before(IndexVersions.USE_SYNTHETIC_SOURCE_FOR_RECOVERY_BACKPORT)) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "The setting [%s] is unavailable on this cluster because some nodes are running older " + + "versions that do not support it. Please upgrade all nodes to the latest version " + + "and try again.", + RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey() + ) + ); + } + } + + @Override + public Iterator> settings() { + List> res = List.of(INDEX_MAPPER_SOURCE_MODE_SETTING, SETTING_INDEX_VERSION_CREATED, MODE); + return res.iterator(); + } + }, + Property.IndexScope, + Property.Final + ); + /** * Returns true if TSDB encoding is enabled. The default is true */ @@ -827,6 +885,7 @@ private void setRetentionLeaseMillis(final TimeValue retentionLease) { private volatile boolean skipIgnoredSourceRead; private final SourceFieldMapper.Mode indexMappingSourceMode; private final boolean recoverySourceEnabled; + private final boolean recoverySourceSyntheticEnabled; /** * The maximum number of refresh listeners allows on this shard. @@ -987,8 +1046,9 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti es87TSDBCodecEnabled = scopedSettings.get(TIME_SERIES_ES87TSDB_CODEC_ENABLED_SETTING); skipIgnoredSourceWrite = scopedSettings.get(IgnoredSourceFieldMapper.SKIP_IGNORED_SOURCE_WRITE_SETTING); skipIgnoredSourceRead = scopedSettings.get(IgnoredSourceFieldMapper.SKIP_IGNORED_SOURCE_READ_SETTING); - indexMappingSourceMode = scopedSettings.get(SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING); + indexMappingSourceMode = scopedSettings.get(INDEX_MAPPER_SOURCE_MODE_SETTING); recoverySourceEnabled = RecoverySettings.INDICES_RECOVERY_SOURCE_ENABLED_SETTING.get(nodeSettings); + recoverySourceSyntheticEnabled = scopedSettings.get(RECOVERY_USE_SYNTHETIC_SOURCE_SETTING); scopedSettings.addSettingsUpdateConsumer( MergePolicyConfig.INDEX_COMPOUND_FORMAT_SETTING, @@ -1680,6 +1740,13 @@ public boolean isRecoverySourceEnabled() { return recoverySourceEnabled; } + /** + * @return Whether recovery source should always be bypassed in favor of using synthetic source. + */ + public boolean isRecoverySourceSyntheticEnabled() { + return recoverySourceSyntheticEnabled; + } + /** * The bounds for {@code @timestamp} on this index or * {@code null} if there are no bounds. diff --git a/server/src/main/java/org/elasticsearch/index/IndexVersions.java b/server/src/main/java/org/elasticsearch/index/IndexVersions.java index 1f2bdf5a06234..cc565be67f2c6 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexVersions.java +++ b/server/src/main/java/org/elasticsearch/index/IndexVersions.java @@ -122,6 +122,7 @@ private static IndexVersion def(int id, Version luceneVersion) { public static final IndexVersion LOGSDB_DEFAULT_IGNORE_DYNAMIC_BEYOND_LIMIT_BACKPORT = def(8_519_00_0, Version.LUCENE_9_12_0); public static final IndexVersion TIME_BASED_K_ORDERED_DOC_ID_BACKPORT = def(8_520_00_0, Version.LUCENE_9_12_0); public static final IndexVersion DEPRECATE_SOURCE_MODE_MAPPER = def(8_521_00_0, Version.LUCENE_9_12_0); + public static final IndexVersion USE_SYNTHETIC_SOURCE_FOR_RECOVERY_BACKPORT = def(8_522_00_0, Version.LUCENE_9_12_0); /* * STOP! READ THIS FIRST! No, really, * ____ _____ ___ ____ _ ____ _____ _ ____ _____ _ _ ___ ____ _____ ___ ____ ____ _____ _ diff --git a/server/src/main/java/org/elasticsearch/index/engine/CombinedDocValues.java b/server/src/main/java/org/elasticsearch/index/engine/CombinedDocValues.java index 48fc76063f815..190a1ed8b457a 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/CombinedDocValues.java +++ b/server/src/main/java/org/elasticsearch/index/engine/CombinedDocValues.java @@ -24,6 +24,7 @@ final class CombinedDocValues { private final NumericDocValues primaryTermDV; private final NumericDocValues tombstoneDV; private final NumericDocValues recoverySource; + private final NumericDocValues recoverySourceSize; CombinedDocValues(LeafReader leafReader) throws IOException { this.versionDV = Objects.requireNonNull(leafReader.getNumericDocValues(VersionFieldMapper.NAME), "VersionDV is missing"); @@ -34,6 +35,7 @@ final class CombinedDocValues { ); this.tombstoneDV = leafReader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); this.recoverySource = leafReader.getNumericDocValues(SourceFieldMapper.RECOVERY_SOURCE_NAME); + this.recoverySourceSize = leafReader.getNumericDocValues(SourceFieldMapper.RECOVERY_SOURCE_SIZE_NAME); } long docVersion(int segmentDocId) throws IOException { @@ -79,4 +81,12 @@ boolean hasRecoverySource(int segmentDocId) throws IOException { assert recoverySource.docID() < segmentDocId; return recoverySource.advanceExact(segmentDocId); } + + long recoverySourceSize(int segmentDocId) throws IOException { + if (recoverySourceSize == null) { + return -1; + } + assert recoverySourceSize.docID() < segmentDocId; + return recoverySourceSize.advanceExact(segmentDocId) ? recoverySourceSize.longValue() : -1; + } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index 8311642ab7073..7e8f8d13c5857 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -940,7 +940,7 @@ public boolean allowSearchIdleOptimization() { * @param source the source of the request * @param fromSeqNo the start sequence number (inclusive) * @param toSeqNo the end sequence number (inclusive) - * @see #newChangesSnapshot(String, long, long, boolean, boolean, boolean) + * @see #newChangesSnapshot(String, long, long, boolean, boolean, boolean, long) */ public abstract int countChanges(String source, long fromSeqNo, long toSeqNo) throws IOException; @@ -954,7 +954,8 @@ public abstract Translog.Snapshot newChangesSnapshot( long toSeqNo, boolean requiredFullRange, boolean singleConsumer, - boolean accessStats + boolean accessStats, + long maxChunkSize ) throws IOException; /** diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index cd571a21c90d3..0237229f8c47b 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -2709,7 +2709,10 @@ private IndexWriterConfig getIndexWriterConfig() { // always configure soft-deletes field so an engine with soft-deletes disabled can open a Lucene index with soft-deletes. iwc.setSoftDeletesField(Lucene.SOFT_DELETES_FIELD); mergePolicy = new RecoverySourcePruneMergePolicy( - SourceFieldMapper.RECOVERY_SOURCE_NAME, + engineConfig.getIndexSettings().isRecoverySourceSyntheticEnabled() ? null : SourceFieldMapper.RECOVERY_SOURCE_NAME, + engineConfig.getIndexSettings().isRecoverySourceSyntheticEnabled() + ? SourceFieldMapper.RECOVERY_SOURCE_SIZE_NAME + : SourceFieldMapper.RECOVERY_SOURCE_NAME, engineConfig.getIndexSettings().getMode() == IndexMode.TIME_SERIES, softDeletesPolicy::getRetentionQuery, new SoftDeletesRetentionMergePolicy( @@ -3140,7 +3143,8 @@ public Translog.Snapshot newChangesSnapshot( long toSeqNo, boolean requiredFullRange, boolean singleConsumer, - boolean accessStats + boolean accessStats, + long maxChunkSize ) throws IOException { if (enableRecoverySource == false) { throw new IllegalStateException( @@ -3153,16 +3157,31 @@ public Translog.Snapshot newChangesSnapshot( refreshIfNeeded(source, toSeqNo); Searcher searcher = acquireSearcher(source, SearcherScope.INTERNAL); try { - LuceneChangesSnapshot snapshot = new LuceneChangesSnapshot( - searcher, - LuceneChangesSnapshot.DEFAULT_BATCH_SIZE, - fromSeqNo, - toSeqNo, - requiredFullRange, - singleConsumer, - accessStats, - config().getIndexSettings().getIndexVersionCreated() - ); + final Translog.Snapshot snapshot; + if (engineConfig.getIndexSettings().isRecoverySourceSyntheticEnabled()) { + snapshot = new LuceneSyntheticSourceChangesSnapshot( + engineConfig.getMapperService().mappingLookup(), + searcher, + SearchBasedChangesSnapshot.DEFAULT_BATCH_SIZE, + maxChunkSize, + fromSeqNo, + toSeqNo, + requiredFullRange, + accessStats, + config().getIndexSettings().getIndexVersionCreated() + ); + } else { + snapshot = new LuceneChangesSnapshot( + searcher, + SearchBasedChangesSnapshot.DEFAULT_BATCH_SIZE, + fromSeqNo, + toSeqNo, + requiredFullRange, + singleConsumer, + accessStats, + config().getIndexSettings().getIndexVersionCreated() + ); + } searcher = null; return snapshot; } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index 05cc6d148be5e..a333da2794a42 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -10,61 +10,33 @@ package org.elasticsearch.index.engine; import org.apache.lucene.codecs.StoredFieldsReader; -import org.apache.lucene.document.LongPoint; -import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.FieldDoc; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.SortField; import org.apache.lucene.search.TopDocs; -import org.apache.lucene.search.TopFieldCollectorManager; import org.apache.lucene.util.ArrayUtil; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.SequentialStoredFieldsLeafReader; -import org.elasticsearch.common.lucene.search.Queries; -import org.elasticsearch.core.IOUtils; +import org.elasticsearch.core.Assertions; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.fieldvisitor.FieldsVisitor; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.transport.Transports; -import java.io.Closeable; import java.io.IOException; import java.util.Comparator; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; /** * A {@link Translog.Snapshot} from changes in a Lucene index */ -final class LuceneChangesSnapshot implements Translog.Snapshot { - static final int DEFAULT_BATCH_SIZE = 1024; - - private final int searchBatchSize; - private final long fromSeqNo, toSeqNo; +public final class LuceneChangesSnapshot extends SearchBasedChangesSnapshot { private long lastSeenSeqNo; private int skippedOperations; - private final boolean requiredFullRange; private final boolean singleConsumer; - private final IndexSearcher indexSearcher; private int docIndex = 0; - private final boolean accessStats; - private final int totalHits; - private ScoreDoc[] scoreDocs; + private int maxDocIndex; private final ParallelArray parallelArray; - private final Closeable onClose; - - private final IndexVersion indexVersionCreated; private int storedFieldsReaderOrd = -1; private StoredFieldsReader storedFieldsReader = null; @@ -83,7 +55,7 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { * @param accessStats true if the stats of the snapshot can be accessed via {@link #totalOperations()} * @param indexVersionCreated the version on which this index was created */ - LuceneChangesSnapshot( + public LuceneChangesSnapshot( Engine.Searcher engineSearcher, int searchBatchSize, long fromSeqNo, @@ -93,50 +65,26 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { boolean accessStats, IndexVersion indexVersionCreated ) throws IOException { - if (fromSeqNo < 0 || toSeqNo < 0 || fromSeqNo > toSeqNo) { - throw new IllegalArgumentException("Invalid range; from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "]"); - } - if (searchBatchSize <= 0) { - throw new IllegalArgumentException("Search_batch_size must be positive [" + searchBatchSize + "]"); - } - final AtomicBoolean closed = new AtomicBoolean(); - this.onClose = () -> { - if (closed.compareAndSet(false, true)) { - IOUtils.close(engineSearcher); - } - }; - final long requestingSize = (toSeqNo - fromSeqNo) == Long.MAX_VALUE ? Long.MAX_VALUE : (toSeqNo - fromSeqNo + 1L); - this.creationThread = Thread.currentThread(); - this.searchBatchSize = requestingSize < searchBatchSize ? Math.toIntExact(requestingSize) : searchBatchSize; - this.fromSeqNo = fromSeqNo; - this.toSeqNo = toSeqNo; - this.lastSeenSeqNo = fromSeqNo - 1; - this.requiredFullRange = requiredFullRange; + super(engineSearcher, searchBatchSize, fromSeqNo, toSeqNo, requiredFullRange, accessStats, indexVersionCreated); + this.creationThread = Assertions.ENABLED ? Thread.currentThread() : null; this.singleConsumer = singleConsumer; - this.indexSearcher = newIndexSearcher(engineSearcher); - this.indexSearcher.setQueryCache(null); - this.accessStats = accessStats; this.parallelArray = new ParallelArray(this.searchBatchSize); - this.indexVersionCreated = indexVersionCreated; - final TopDocs topDocs = searchOperations(null, accessStats); - this.totalHits = Math.toIntExact(topDocs.totalHits.value); - this.scoreDocs = topDocs.scoreDocs; - fillParallelArray(scoreDocs, parallelArray); + this.lastSeenSeqNo = fromSeqNo - 1; + final TopDocs topDocs = nextTopDocs(); + this.maxDocIndex = topDocs.scoreDocs.length; + fillParallelArray(topDocs.scoreDocs, parallelArray); } @Override public void close() throws IOException { assert assertAccessingThread(); - onClose.close(); + super.close(); } @Override public int totalOperations() { assert assertAccessingThread(); - if (accessStats == false) { - throw new IllegalStateException("Access stats of a snapshot created with [access_stats] is false"); - } - return totalHits; + return super.totalOperations(); } @Override @@ -146,7 +94,7 @@ public int skippedOperations() { } @Override - public Translog.Operation next() throws IOException { + protected Translog.Operation nextOperation() throws IOException { assert assertAccessingThread(); Translog.Operation op = null; for (int idx = nextDocIndex(); idx != -1; idx = nextDocIndex()) { @@ -155,12 +103,6 @@ public Translog.Operation next() throws IOException { break; } } - if (requiredFullRange) { - rangeCheck(op); - } - if (op != null) { - lastSeenSeqNo = op.seqNo(); - } return op; } @@ -171,48 +113,15 @@ private boolean assertAccessingThread() { return true; } - private void rangeCheck(Translog.Operation op) { - if (op == null) { - if (lastSeenSeqNo < toSeqNo) { - throw new MissingHistoryOperationsException( - "Not all operations between from_seqno [" - + fromSeqNo - + "] " - + "and to_seqno [" - + toSeqNo - + "] found; prematurely terminated last_seen_seqno [" - + lastSeenSeqNo - + "]" - ); - } - } else { - final long expectedSeqNo = lastSeenSeqNo + 1; - if (op.seqNo() != expectedSeqNo) { - throw new MissingHistoryOperationsException( - "Not all operations between from_seqno [" - + fromSeqNo - + "] " - + "and to_seqno [" - + toSeqNo - + "] found; expected seqno [" - + expectedSeqNo - + "]; found [" - + op - + "]" - ); - } - } - } - private int nextDocIndex() throws IOException { // we have processed all docs in the current search - fetch the next batch - if (docIndex == scoreDocs.length && docIndex > 0) { - final ScoreDoc prev = scoreDocs[scoreDocs.length - 1]; - scoreDocs = searchOperations((FieldDoc) prev, false).scoreDocs; + if (docIndex == maxDocIndex && docIndex > 0) { + var scoreDocs = nextTopDocs().scoreDocs; fillParallelArray(scoreDocs, parallelArray); docIndex = 0; + maxDocIndex = scoreDocs.length; } - if (docIndex < scoreDocs.length) { + if (docIndex < maxDocIndex) { int idx = docIndex; docIndex++; return idx; @@ -237,14 +146,13 @@ private void fillParallelArray(ScoreDoc[] scoreDocs, ParallelArray parallelArray } int docBase = -1; int maxDoc = 0; - List leaves = indexSearcher.getIndexReader().leaves(); int readerIndex = 0; CombinedDocValues combinedDocValues = null; LeafReaderContext leaf = null; for (ScoreDoc scoreDoc : scoreDocs) { if (scoreDoc.doc >= docBase + maxDoc) { do { - leaf = leaves.get(readerIndex++); + leaf = leaves().get(readerIndex++); docBase = leaf.docBase; maxDoc = leaf.reader().maxDoc(); } while (scoreDoc.doc >= docBase + maxDoc); @@ -253,6 +161,7 @@ private void fillParallelArray(ScoreDoc[] scoreDocs, ParallelArray parallelArray final int segmentDocID = scoreDoc.doc - docBase; final int index = scoreDoc.shardIndex; parallelArray.leafReaderContexts[index] = leaf; + parallelArray.docID[index] = scoreDoc.doc; parallelArray.seqNo[index] = combinedDocValues.docSeqNo(segmentDocID); parallelArray.primaryTerm[index] = combinedDocValues.docPrimaryTerm(segmentDocID); parallelArray.version[index] = combinedDocValues.docVersion(segmentDocID); @@ -275,16 +184,6 @@ private static boolean hasSequentialAccess(ScoreDoc[] scoreDocs) { return true; } - private static IndexSearcher newIndexSearcher(Engine.Searcher engineSearcher) throws IOException { - return new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); - } - - private static Query rangeQuery(long fromSeqNo, long toSeqNo, IndexVersion indexVersionCreated) { - return new BooleanQuery.Builder().add(LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, fromSeqNo, toSeqNo), BooleanClause.Occur.MUST) - .add(Queries.newNonNestedFilter(indexVersionCreated), BooleanClause.Occur.MUST) // exclude non-root nested documents - .build(); - } - static int countOperations(Engine.Searcher engineSearcher, long fromSeqNo, long toSeqNo, IndexVersion indexVersionCreated) throws IOException { if (fromSeqNo < 0 || toSeqNo < 0 || fromSeqNo > toSeqNo) { @@ -293,23 +192,9 @@ static int countOperations(Engine.Searcher engineSearcher, long fromSeqNo, long return newIndexSearcher(engineSearcher).count(rangeQuery(fromSeqNo, toSeqNo, indexVersionCreated)); } - private TopDocs searchOperations(FieldDoc after, boolean accurateTotalHits) throws IOException { - final Query rangeQuery = rangeQuery(Math.max(fromSeqNo, lastSeenSeqNo), toSeqNo, indexVersionCreated); - assert accurateTotalHits == false || after == null : "accurate total hits is required by the first batch only"; - final SortField sortBySeqNo = new SortField(SeqNoFieldMapper.NAME, SortField.Type.LONG); - TopFieldCollectorManager topFieldCollectorManager = new TopFieldCollectorManager( - new Sort(sortBySeqNo), - searchBatchSize, - after, - accurateTotalHits ? Integer.MAX_VALUE : 0, - false - ); - return indexSearcher.search(rangeQuery, topFieldCollectorManager); - } - private Translog.Operation readDocAsOp(int docIndex) throws IOException { final LeafReaderContext leaf = parallelArray.leafReaderContexts[docIndex]; - final int segmentDocID = scoreDocs[docIndex].doc - leaf.docBase; + final int segmentDocID = parallelArray.docID[docIndex] - leaf.docBase; final long primaryTerm = parallelArray.primaryTerm[docIndex]; assert primaryTerm > 0 : "nested child document must be excluded"; final long seqNo = parallelArray.seqNo[docIndex]; @@ -385,19 +270,13 @@ private Translog.Operation readDocAsOp(int docIndex) throws IOException { + "], op [" + op + "]"; + lastSeenSeqNo = op.seqNo(); return op; } - private static boolean assertDocSoftDeleted(LeafReader leafReader, int segmentDocId) throws IOException { - final NumericDocValues ndv = leafReader.getNumericDocValues(Lucene.SOFT_DELETES_FIELD); - if (ndv == null || ndv.advanceExact(segmentDocId) == false) { - throw new IllegalStateException("DocValues for field [" + Lucene.SOFT_DELETES_FIELD + "] is not found"); - } - return ndv.longValue() == 1; - } - private static final class ParallelArray { final LeafReaderContext[] leafReaderContexts; + final int[] docID; final long[] version; final long[] seqNo; final long[] primaryTerm; @@ -406,6 +285,7 @@ private static final class ParallelArray { boolean useSequentialStoredFieldsReader = false; ParallelArray(int size) { + docID = new int[size]; version = new long[size]; seqNo = new long[size]; primaryTerm = new long[size]; diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneSyntheticSourceChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneSyntheticSourceChangesSnapshot.java new file mode 100644 index 0000000000000..3d3d2f6f66d56 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneSyntheticSourceChangesSnapshot.java @@ -0,0 +1,244 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.util.ArrayUtil; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.fieldvisitor.LeafStoredFieldLoader; +import org.elasticsearch.index.fieldvisitor.StoredFieldLoader; +import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.SourceFieldMetrics; +import org.elasticsearch.index.mapper.SourceLoader; +import org.elasticsearch.index.translog.Translog; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Deque; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; + +/** + * A {@link SearchBasedChangesSnapshot} that utilizes a synthetic field loader to rebuild the recovery source. + * This snapshot is activated when {@link IndexSettings#RECOVERY_USE_SYNTHETIC_SOURCE_SETTING} + * is enabled on the underlying index. + * + * The {@code maxMemorySizeInBytes} parameter limits the total size of uncompressed _sources + * loaded into memory during batch retrieval. + */ +public class LuceneSyntheticSourceChangesSnapshot extends SearchBasedChangesSnapshot { + private final long maxMemorySizeInBytes; + private final StoredFieldLoader storedFieldLoader; + private final SourceLoader sourceLoader; + + private int skippedOperations; + private long lastSeenSeqNo; + + private record SearchRecord(FieldDoc doc, boolean isTombstone, long seqNo, long primaryTerm, long version, long size) { + int index() { + return doc.shardIndex; + } + + int docID() { + return doc.doc; + } + + boolean hasRecoverySourceSize() { + return size != -1; + } + } + + private final Deque pendingDocs = new LinkedList<>(); + private final Deque operationQueue = new LinkedList<>(); + + public LuceneSyntheticSourceChangesSnapshot( + MappingLookup mappingLookup, + Engine.Searcher engineSearcher, + int searchBatchSize, + long maxMemorySizeInBytes, + long fromSeqNo, + long toSeqNo, + boolean requiredFullRange, + boolean accessStats, + IndexVersion indexVersionCreated + ) throws IOException { + super(engineSearcher, searchBatchSize, fromSeqNo, toSeqNo, requiredFullRange, accessStats, indexVersionCreated); + assert mappingLookup.isSourceSynthetic(); + // ensure we can buffer at least one document + this.maxMemorySizeInBytes = maxMemorySizeInBytes > 0 ? maxMemorySizeInBytes : 1; + this.sourceLoader = mappingLookup.newSourceLoader(SourceFieldMetrics.NOOP); + Set storedFields = sourceLoader.requiredStoredFields(); + assert mappingLookup.isSourceSynthetic() : "synthetic source must be enabled for proper functionality."; + this.storedFieldLoader = StoredFieldLoader.create(false, storedFields); + this.lastSeenSeqNo = fromSeqNo - 1; + } + + @Override + public int skippedOperations() { + return skippedOperations; + } + + @Override + protected Translog.Operation nextOperation() throws IOException { + while (true) { + if (operationQueue.isEmpty()) { + loadNextBatch(); + } + if (operationQueue.isEmpty()) { + return null; + } + var op = operationQueue.pollFirst(); + if (op.seqNo() == lastSeenSeqNo) { + skippedOperations++; + continue; + } + lastSeenSeqNo = op.seqNo(); + return op; + } + } + + private void loadNextBatch() throws IOException { + List documentsToLoad = new ArrayList<>(); + long accumulatedSize = 0; + while (accumulatedSize < maxMemorySizeInBytes) { + if (pendingDocs.isEmpty()) { + ScoreDoc[] topDocs = nextTopDocs().scoreDocs; + if (topDocs.length == 0) { + break; + } + pendingDocs.addAll(Arrays.asList(transformScoreDocsToRecords(topDocs))); + } + SearchRecord document = pendingDocs.pollFirst(); + document.doc().shardIndex = documentsToLoad.size(); + documentsToLoad.add(document); + accumulatedSize += document.size(); + } + + for (var op : loadDocuments(documentsToLoad)) { + if (op == null) { + skippedOperations++; + continue; + } + operationQueue.add(op); + } + } + + private SearchRecord[] transformScoreDocsToRecords(ScoreDoc[] scoreDocs) throws IOException { + ArrayUtil.introSort(scoreDocs, Comparator.comparingInt(doc -> doc.doc)); + SearchRecord[] documentRecords = new SearchRecord[scoreDocs.length]; + CombinedDocValues combinedDocValues = null; + int docBase = -1; + int maxDoc = 0; + int readerIndex = 0; + LeafReaderContext leafReaderContext; + + for (int i = 0; i < scoreDocs.length; i++) { + ScoreDoc scoreDoc = scoreDocs[i]; + if (scoreDoc.doc >= docBase + maxDoc) { + do { + leafReaderContext = leaves().get(readerIndex++); + docBase = leafReaderContext.docBase; + maxDoc = leafReaderContext.reader().maxDoc(); + } while (scoreDoc.doc >= docBase + maxDoc); + combinedDocValues = new CombinedDocValues(leafReaderContext.reader()); + } + int segmentDocID = scoreDoc.doc - docBase; + int index = scoreDoc.shardIndex; + var primaryTerm = combinedDocValues.docPrimaryTerm(segmentDocID); + assert primaryTerm > 0 : "nested child document must be excluded"; + documentRecords[index] = new SearchRecord( + (FieldDoc) scoreDoc, + combinedDocValues.isTombstone(segmentDocID), + combinedDocValues.docSeqNo(segmentDocID), + primaryTerm, + combinedDocValues.docVersion(segmentDocID), + combinedDocValues.recoverySourceSize(segmentDocID) + ); + } + return documentRecords; + } + + private Translog.Operation[] loadDocuments(List documentRecords) throws IOException { + documentRecords.sort(Comparator.comparingInt(doc -> doc.docID())); + Translog.Operation[] operations = new Translog.Operation[documentRecords.size()]; + + int docBase = -1; + int maxDoc = 0; + int readerIndex = 0; + LeafReaderContext leafReaderContext = null; + LeafStoredFieldLoader leafFieldLoader = null; + SourceLoader.Leaf leafSourceLoader = null; + for (int i = 0; i < documentRecords.size(); i++) { + SearchRecord docRecord = documentRecords.get(i); + if (docRecord.docID() >= docBase + maxDoc) { + do { + leafReaderContext = leaves().get(readerIndex++); + docBase = leafReaderContext.docBase; + maxDoc = leafReaderContext.reader().maxDoc(); + } while (docRecord.docID() >= docBase + maxDoc); + + leafFieldLoader = storedFieldLoader.getLoader(leafReaderContext, null); + leafSourceLoader = sourceLoader.leaf(leafReaderContext.reader(), null); + } + int segmentDocID = docRecord.docID() - docBase; + leafFieldLoader.advanceTo(segmentDocID); + operations[docRecord.index()] = createOperation(docRecord, leafFieldLoader, leafSourceLoader, segmentDocID, leafReaderContext); + } + return operations; + } + + private Translog.Operation createOperation( + SearchRecord docRecord, + LeafStoredFieldLoader fieldLoader, + SourceLoader.Leaf sourceLoader, + int segmentDocID, + LeafReaderContext context + ) throws IOException { + if (docRecord.isTombstone() && fieldLoader.id() == null) { + assert docRecord.version() == 1L : "Noop tombstone should have version 1L; actual version [" + docRecord.version() + "]"; + assert assertDocSoftDeleted(context.reader(), segmentDocID) : "Noop but soft_deletes field is not set [" + docRecord + "]"; + return new Translog.NoOp(docRecord.seqNo(), docRecord.primaryTerm(), "null"); + } else if (docRecord.isTombstone()) { + assert assertDocSoftDeleted(context.reader(), segmentDocID) : "Delete op but soft_deletes field is not set [" + docRecord + "]"; + return new Translog.Delete(fieldLoader.id(), docRecord.seqNo(), docRecord.primaryTerm(), docRecord.version()); + } else { + if (docRecord.hasRecoverySourceSize() == false) { + // TODO: Callers should ask for the range that source should be retained. Thus we should always + // check for the existence source once we make peer-recovery to send ops after the local checkpoint. + if (requiredFullRange) { + throw new MissingHistoryOperationsException( + "source not found for seqno=" + docRecord.seqNo() + " from_seqno=" + fromSeqNo + " to_seqno=" + toSeqNo + ); + } else { + skippedOperations++; + return null; + } + } + BytesReference source = sourceLoader.source(fieldLoader, segmentDocID).internalSourceRef(); + return new Translog.Index( + fieldLoader.id(), + docRecord.seqNo(), + docRecord.primaryTerm(), + docRecord.version(), + source, + fieldLoader.routing(), + -1 // autogenerated timestamp + ); + } + } + +} diff --git a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java index d4a2fe1b57903..c1d11223fa55e 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java @@ -356,7 +356,7 @@ public Closeable acquireHistoryRetentionLock() { @Override public int countChanges(String source, long fromSeqNo, long toSeqNo) throws IOException { - try (Translog.Snapshot snapshot = newChangesSnapshot(source, fromSeqNo, toSeqNo, false, true, true)) { + try (Translog.Snapshot snapshot = newChangesSnapshot(source, fromSeqNo, toSeqNo, false, true, true, -1)) { return snapshot.totalOperations(); } } @@ -368,7 +368,8 @@ public Translog.Snapshot newChangesSnapshot( long toSeqNo, boolean requiredFullRange, boolean singleConsumer, - boolean accessStats + boolean accessStats, + long maxChunkSize ) { return Translog.Snapshot.EMPTY; } diff --git a/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java b/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java index 18b5ba69ca320..c24ad42d1d61a 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java @@ -32,17 +32,18 @@ import org.apache.lucene.search.Weight; import org.apache.lucene.util.BitSet; import org.apache.lucene.util.BitSetIterator; +import org.elasticsearch.core.Nullable; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.search.internal.FilterStoredFieldVisitor; import java.io.IOException; import java.util.Arrays; -import java.util.Objects; import java.util.function.Supplier; final class RecoverySourcePruneMergePolicy extends OneMergeWrappingMergePolicy { RecoverySourcePruneMergePolicy( - String recoverySourceField, + @Nullable String pruneStoredFieldName, + String pruneNumericDVFieldName, boolean pruneIdField, Supplier retainSourceQuerySupplier, MergePolicy in @@ -51,18 +52,19 @@ final class RecoverySourcePruneMergePolicy extends OneMergeWrappingMergePolicy { @Override public CodecReader wrapForMerge(CodecReader reader) throws IOException { CodecReader wrapped = toWrap.wrapForMerge(reader); - return wrapReader(recoverySourceField, pruneIdField, wrapped, retainSourceQuerySupplier); + return wrapReader(pruneStoredFieldName, pruneNumericDVFieldName, pruneIdField, wrapped, retainSourceQuerySupplier); } }); } private static CodecReader wrapReader( - String recoverySourceField, + String pruneStoredFieldName, + String pruneNumericDVFieldName, boolean pruneIdField, CodecReader reader, Supplier retainSourceQuerySupplier ) throws IOException { - NumericDocValues recoverySource = reader.getNumericDocValues(recoverySourceField); + NumericDocValues recoverySource = reader.getNumericDocValues(pruneNumericDVFieldName); if (recoverySource == null || recoverySource.nextDoc() == DocIdSetIterator.NO_MORE_DOCS) { return reader; // early terminate - nothing to do here since non of the docs has a recovery source anymore. } @@ -77,21 +79,35 @@ private static CodecReader wrapReader( if (recoverySourceToKeep.cardinality() == reader.maxDoc()) { return reader; // keep all source } - return new SourcePruningFilterCodecReader(recoverySourceField, pruneIdField, reader, recoverySourceToKeep); + return new SourcePruningFilterCodecReader( + pruneStoredFieldName, + pruneNumericDVFieldName, + pruneIdField, + reader, + recoverySourceToKeep + ); } else { - return new SourcePruningFilterCodecReader(recoverySourceField, pruneIdField, reader, null); + return new SourcePruningFilterCodecReader(pruneStoredFieldName, pruneNumericDVFieldName, pruneIdField, reader, null); } } private static class SourcePruningFilterCodecReader extends FilterCodecReader { private final BitSet recoverySourceToKeep; - private final String recoverySourceField; + private final String pruneStoredFieldName; + private final String pruneNumericDVFieldName; private final boolean pruneIdField; - SourcePruningFilterCodecReader(String recoverySourceField, boolean pruneIdField, CodecReader reader, BitSet recoverySourceToKeep) { + SourcePruningFilterCodecReader( + @Nullable String pruneStoredFieldName, + String pruneNumericDVFieldName, + boolean pruneIdField, + CodecReader reader, + BitSet recoverySourceToKeep + ) { super(reader); - this.recoverySourceField = recoverySourceField; + this.pruneStoredFieldName = pruneStoredFieldName; this.recoverySourceToKeep = recoverySourceToKeep; + this.pruneNumericDVFieldName = pruneNumericDVFieldName; this.pruneIdField = pruneIdField; } @@ -102,8 +118,8 @@ public DocValuesProducer getDocValuesReader() { @Override public NumericDocValues getNumeric(FieldInfo field) throws IOException { NumericDocValues numeric = super.getNumeric(field); - if (recoverySourceField.equals(field.name)) { - assert numeric != null : recoverySourceField + " must have numeric DV but was null"; + if (field.name.equals(pruneNumericDVFieldName)) { + assert numeric != null : pruneNumericDVFieldName + " must have numeric DV but was null"; final DocIdSetIterator intersection; if (recoverySourceToKeep == null) { // we can't return null here lucenes DocIdMerger expects an instance @@ -138,10 +154,14 @@ public boolean advanceExact(int target) { @Override public StoredFieldsReader getFieldsReader() { + if (pruneStoredFieldName == null && pruneIdField == false) { + // nothing to prune, we can use the original fields reader + return super.getFieldsReader(); + } return new RecoverySourcePruningStoredFieldsReader( super.getFieldsReader(), recoverySourceToKeep, - recoverySourceField, + pruneStoredFieldName, pruneIdField ); } @@ -235,12 +255,13 @@ private static class RecoverySourcePruningStoredFieldsReader extends FilterStore RecoverySourcePruningStoredFieldsReader( StoredFieldsReader in, BitSet recoverySourceToKeep, - String recoverySourceField, + @Nullable String recoverySourceField, boolean pruneIdField ) { super(in); + assert recoverySourceField != null || pruneIdField : "nothing to prune"; this.recoverySourceToKeep = recoverySourceToKeep; - this.recoverySourceField = Objects.requireNonNull(recoverySourceField); + this.recoverySourceField = recoverySourceField; this.pruneIdField = pruneIdField; } @@ -252,7 +273,7 @@ public void document(int docID, StoredFieldVisitor visitor) throws IOException { super.document(docID, new FilterStoredFieldVisitor(visitor) { @Override public Status needsField(FieldInfo fieldInfo) throws IOException { - if (recoverySourceField.equals(fieldInfo.name)) { + if (fieldInfo.name.equals(recoverySourceField)) { return Status.NO; } if (pruneIdField && IdFieldMapper.NAME.equals(fieldInfo.name)) { diff --git a/server/src/main/java/org/elasticsearch/index/engine/SearchBasedChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/SearchBasedChangesSnapshot.java new file mode 100644 index 0000000000000..191125c59705e --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/SearchBasedChangesSnapshot.java @@ -0,0 +1,233 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.TopFieldCollectorManager; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.lucene.search.Queries; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.translog.Translog; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Abstract class that provides a snapshot mechanism to retrieve operations from a live Lucene index + * within a specified range of sequence numbers. Subclasses are expected to define the + * method to fetch the next batch of operations. + */ +public abstract class SearchBasedChangesSnapshot implements Translog.Snapshot, Closeable { + public static final int DEFAULT_BATCH_SIZE = 1024; + + private final IndexVersion indexVersionCreated; + private final IndexSearcher indexSearcher; + private final Closeable onClose; + + protected final long fromSeqNo, toSeqNo; + protected final boolean requiredFullRange; + protected final int searchBatchSize; + + private final boolean accessStats; + private final int totalHits; + private FieldDoc afterDoc; + private long lastSeenSeqNo; + + /** + * Constructs a new snapshot for fetching changes within a sequence number range. + * + * @param engineSearcher Engine searcher instance. + * @param searchBatchSize Number of documents to retrieve per batch. + * @param fromSeqNo Starting sequence number. + * @param toSeqNo Ending sequence number. + * @param requiredFullRange Whether the full range is required. + * @param accessStats If true, enable access statistics for counting total operations. + * @param indexVersionCreated Version of the index when it was created. + */ + protected SearchBasedChangesSnapshot( + Engine.Searcher engineSearcher, + int searchBatchSize, + long fromSeqNo, + long toSeqNo, + boolean requiredFullRange, + boolean accessStats, + IndexVersion indexVersionCreated + ) throws IOException { + + if (fromSeqNo < 0 || toSeqNo < 0 || fromSeqNo > toSeqNo) { + throw new IllegalArgumentException("Invalid range; from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "]"); + } + if (searchBatchSize <= 0) { + throw new IllegalArgumentException("Search_batch_size must be positive [" + searchBatchSize + "]"); + } + + final AtomicBoolean closed = new AtomicBoolean(); + this.onClose = () -> { + if (closed.compareAndSet(false, true)) { + IOUtils.close(engineSearcher); + } + }; + + this.indexVersionCreated = indexVersionCreated; + this.fromSeqNo = fromSeqNo; + this.toSeqNo = toSeqNo; + this.lastSeenSeqNo = fromSeqNo - 1; + this.requiredFullRange = requiredFullRange; + this.indexSearcher = newIndexSearcher(engineSearcher); + this.indexSearcher.setQueryCache(null); + + long requestingSize = (toSeqNo - fromSeqNo == Long.MAX_VALUE) ? Long.MAX_VALUE : (toSeqNo - fromSeqNo + 1L); + this.searchBatchSize = (int) Math.min(requestingSize, searchBatchSize); + + this.accessStats = accessStats; + this.totalHits = accessStats ? indexSearcher.count(rangeQuery(fromSeqNo, toSeqNo, indexVersionCreated)) : -1; + } + + /** + * Abstract method for retrieving the next operation. Should be implemented by subclasses. + * + * @return The next Translog.Operation in the snapshot. + * @throws IOException If an I/O error occurs. + */ + protected abstract Translog.Operation nextOperation() throws IOException; + + /** + * Returns the list of index leaf reader contexts. + * + * @return List of LeafReaderContext. + */ + public List leaves() { + return indexSearcher.getIndexReader().leaves(); + } + + @Override + public int totalOperations() { + if (accessStats == false) { + throw new IllegalStateException("Access stats of a snapshot created with [access_stats] is false"); + } + return totalHits; + } + + @Override + public final Translog.Operation next() throws IOException { + Translog.Operation op = nextOperation(); + if (requiredFullRange) { + verifyRange(op); + } + if (op != null) { + assert fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo && lastSeenSeqNo < op.seqNo() + : "Unexpected operation; last_seen_seqno [" + + lastSeenSeqNo + + "], from_seqno [" + + fromSeqNo + + "], to_seqno [" + + toSeqNo + + "], op [" + + op + + "]"; + lastSeenSeqNo = op.seqNo(); + } + return op; + } + + @Override + public void close() throws IOException { + onClose.close(); + } + + /** + * Retrieves the next batch of top documents based on the sequence range. + * + * @return TopDocs instance containing the documents in the current batch. + */ + protected TopDocs nextTopDocs() throws IOException { + Query rangeQuery = rangeQuery(Math.max(fromSeqNo, lastSeenSeqNo), toSeqNo, indexVersionCreated); + SortField sortBySeqNo = new SortField(SeqNoFieldMapper.NAME, SortField.Type.LONG); + + TopFieldCollectorManager collectorManager = new TopFieldCollectorManager( + new Sort(sortBySeqNo), + searchBatchSize, + afterDoc, + 0, + false + ); + TopDocs results = indexSearcher.search(rangeQuery, collectorManager); + + if (results.scoreDocs.length > 0) { + afterDoc = (FieldDoc) results.scoreDocs[results.scoreDocs.length - 1]; + } + for (int i = 0; i < results.scoreDocs.length; i++) { + results.scoreDocs[i].shardIndex = i; + } + return results; + } + + static IndexSearcher newIndexSearcher(Engine.Searcher engineSearcher) throws IOException { + return new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); + } + + static Query rangeQuery(long fromSeqNo, long toSeqNo, IndexVersion indexVersionCreated) { + return new BooleanQuery.Builder().add(LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, fromSeqNo, toSeqNo), BooleanClause.Occur.MUST) + .add(Queries.newNonNestedFilter(indexVersionCreated), BooleanClause.Occur.MUST) + .build(); + } + + private void verifyRange(Translog.Operation op) { + if (op == null && lastSeenSeqNo < toSeqNo) { + throw new MissingHistoryOperationsException( + "Not all operations between from_seqno [" + + fromSeqNo + + "] " + + "and to_seqno [" + + toSeqNo + + "] found; prematurely terminated last_seen_seqno [" + + lastSeenSeqNo + + "]" + ); + } else if (op != null && op.seqNo() != lastSeenSeqNo + 1) { + throw new MissingHistoryOperationsException( + "Not all operations between from_seqno [" + + fromSeqNo + + "] " + + "and to_seqno [" + + toSeqNo + + "] found; expected seqno [" + + lastSeenSeqNo + + 1 + + "]; found [" + + op + + "]" + ); + } + } + + protected static boolean assertDocSoftDeleted(LeafReader leafReader, int segmentDocId) throws IOException { + NumericDocValues docValues = leafReader.getNumericDocValues(Lucene.SOFT_DELETES_FIELD); + if (docValues == null || docValues.advanceExact(segmentDocId) == false) { + throw new IllegalStateException("DocValues for field [" + Lucene.SOFT_DELETES_FIELD + "] is not found"); + } + return docValues.longValue() == 1; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/fieldvisitor/LeafStoredFieldLoader.java b/server/src/main/java/org/elasticsearch/index/fieldvisitor/LeafStoredFieldLoader.java index c8709d3422213..3ed4c856ccc71 100644 --- a/server/src/main/java/org/elasticsearch/index/fieldvisitor/LeafStoredFieldLoader.java +++ b/server/src/main/java/org/elasticsearch/index/fieldvisitor/LeafStoredFieldLoader.java @@ -47,5 +47,4 @@ public interface LeafStoredFieldLoader { * @return stored fields for the current document */ Map> storedFields(); - } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java index 39dea35bfc2d2..efbe1e91ce9c5 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java @@ -59,6 +59,8 @@ public class SourceFieldMapper extends MetadataFieldMapper { public static final String NAME = "_source"; public static final String RECOVERY_SOURCE_NAME = "_recovery_source"; + public static final String RECOVERY_SOURCE_SIZE_NAME = "_recovery_source_size"; + public static final String CONTENT_TYPE = "_source"; public static final String LOSSY_PARAMETERS_ALLOWED_SETTING_NAME = "index.lossy.source-mapping-parameters"; @@ -412,8 +414,19 @@ public void preParse(DocumentParserContext context) throws IOException { if (enableRecoverySource && originalSource != null && adaptedSource != originalSource) { // if we omitted source or modified it we add the _recovery_source to ensure we have it for ops based recovery BytesRef ref = originalSource.toBytesRef(); - context.doc().add(new StoredField(RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); - context.doc().add(new NumericDocValuesField(RECOVERY_SOURCE_NAME, 1)); + if (context.indexSettings().isRecoverySourceSyntheticEnabled()) { + assert isSynthetic() : "recovery source should not be disabled on non-synthetic source"; + /** + * We use synthetic source for recovery, so we omit the recovery source. + * Instead, we record only the size of the uncompressed source. + * This size is used in {@link LuceneSyntheticSourceChangesSnapshot} to control memory + * usage during the recovery process when loading a batch of synthetic sources. + */ + context.doc().add(new NumericDocValuesField(RECOVERY_SOURCE_SIZE_NAME, ref.length)); + } else { + context.doc().add(new StoredField(RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); + context.doc().add(new NumericDocValuesField(RECOVERY_SOURCE_NAME, 1)); + } } } diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index d4ee6eeaf05fb..ebd4bdffb0436 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -2584,7 +2584,7 @@ public long getMinRetainedSeqNo() { * @param source the source of the request * @param fromSeqNo the start sequence number (inclusive) * @param toSeqNo the end sequence number (inclusive) - * @see #newChangesSnapshot(String, long, long, boolean, boolean, boolean) + * @see #newChangesSnapshot(String, long, long, boolean, boolean, boolean, long) */ public int countChanges(String source, long fromSeqNo, long toSeqNo) throws IOException { return getEngine().countChanges(source, fromSeqNo, toSeqNo); @@ -2603,6 +2603,7 @@ public int countChanges(String source, long fromSeqNo, long toSeqNo) throws IOEx * @param singleConsumer true if the snapshot is accessed by only the thread that creates the snapshot. In this case, the * snapshot can enable some optimizations to improve the performance. * @param accessStats true if the stats of the snapshot is accessed via {@link Translog.Snapshot#totalOperations()} + * @param maxChunkSize The maximum allowable size, in bytes, for buffering source documents during recovery. */ public Translog.Snapshot newChangesSnapshot( String source, @@ -2610,9 +2611,10 @@ public Translog.Snapshot newChangesSnapshot( long toSeqNo, boolean requiredFullRange, boolean singleConsumer, - boolean accessStats + boolean accessStats, + long maxChunkSize ) throws IOException { - return getEngine().newChangesSnapshot(source, fromSeqNo, toSeqNo, requiredFullRange, singleConsumer, accessStats); + return getEngine().newChangesSnapshot(source, fromSeqNo, toSeqNo, requiredFullRange, singleConsumer, accessStats, maxChunkSize); } public List segments() { diff --git a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java index f843357e056c4..1143da30c2952 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java +++ b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java @@ -81,7 +81,7 @@ public void resync(final IndexShard indexShard, final ActionListener // Wrap translog snapshot to make it synchronized as it is accessed by different threads through SnapshotSender. // Even though those calls are not concurrent, snapshot.next() uses non-synchronized state and is not multi-thread-compatible // Also fail the resync early if the shard is shutting down - snapshot = indexShard.newChangesSnapshot("resync", startingSeqNo, Long.MAX_VALUE, false, false, true); + snapshot = indexShard.newChangesSnapshot("resync", startingSeqNo, Long.MAX_VALUE, false, false, true, chunkSize.getBytes()); final Translog.Snapshot originalSnapshot = snapshot; final Translog.Snapshot wrappedSnapshot = new Translog.Snapshot() { @Override diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java index 5fb9d95b91271..f5a920e723308 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java @@ -394,6 +394,18 @@ public Iterator> settings() { public static final ByteSizeValue DEFAULT_CHUNK_SIZE = new ByteSizeValue(512, ByteSizeUnit.KB); + /** + * The maximum allowable size, in bytes, for buffering source documents during recovery. + */ + public static final Setting INDICES_RECOVERY_CHUNK_SIZE = Setting.byteSizeSetting( + "indices.recovery.chunk_size", + DEFAULT_CHUNK_SIZE, + ByteSizeValue.ZERO, + ByteSizeValue.ofBytes(Integer.MAX_VALUE), + Property.NodeScope, + Property.Dynamic + ); + private volatile ByteSizeValue maxBytesPerSec; private volatile int maxConcurrentFileChunks; private volatile int maxConcurrentOperations; @@ -412,7 +424,7 @@ public Iterator> settings() { private final AdjustableSemaphore maxSnapshotFileDownloadsPerNodeSemaphore; - private volatile ByteSizeValue chunkSize = DEFAULT_CHUNK_SIZE; + private volatile ByteSizeValue chunkSize; private final ByteSizeValue availableNetworkBandwidth; private final ByteSizeValue availableDiskReadBandwidth; @@ -439,6 +451,7 @@ public RecoverySettings(Settings settings, ClusterSettings clusterSettings) { this.availableNetworkBandwidth = NODE_BANDWIDTH_RECOVERY_NETWORK_SETTING.get(settings); this.availableDiskReadBandwidth = NODE_BANDWIDTH_RECOVERY_DISK_READ_SETTING.get(settings); this.availableDiskWriteBandwidth = NODE_BANDWIDTH_RECOVERY_DISK_WRITE_SETTING.get(settings); + this.chunkSize = INDICES_RECOVERY_CHUNK_SIZE.get(settings); validateNodeBandwidthRecoverySettings(settings); this.nodeBandwidthSettingsExist = hasNodeBandwidthRecoverySettings(settings); computeMaxBytesPerSec(settings); @@ -488,6 +501,7 @@ public RecoverySettings(Settings settings, ClusterSettings clusterSettings) { CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING, this::setMaxConcurrentIncomingRecoveries ); + clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_CHUNK_SIZE, this::setChunkSize); } private void computeMaxBytesPerSec(Settings settings) { @@ -592,7 +606,7 @@ public ByteSizeValue getChunkSize() { return chunkSize; } - public void setChunkSize(ByteSizeValue chunkSize) { // only settable for tests + public void setChunkSize(ByteSizeValue chunkSize) { if (chunkSize.bytesAsInt() <= 0) { throw new IllegalArgumentException("chunkSize must be > 0"); } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 78430438a4f56..4469f44a66f4c 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -322,7 +322,8 @@ && isTargetSameHistory() Long.MAX_VALUE, false, false, - true + true, + chunkSizeInBytes ); resources.add(phase2Snapshot); retentionLock.close(); diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 6c6973c0cd491..41b85e47618d2 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -158,6 +158,7 @@ public final class RestoreService implements ClusterStateApplier { SETTING_HISTORY_UUID, IndexSettings.MODE.getKey(), SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.getKey(), + IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey(), IndexSortConfig.INDEX_SORT_FIELD_SETTING.getKey(), IndexSortConfig.INDEX_SORT_ORDER_SETTING.getKey(), IndexSortConfig.INDEX_SORT_MODE_SETTING.getKey(), diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index a9ae235bd5f82..faff880ae82e5 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -89,6 +89,7 @@ import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; @@ -3453,7 +3454,7 @@ public void testTranslogReplay() throws IOException { assertThat(indexResult.getVersion(), equalTo(1L)); } assertVisibleCount(engine, numDocs); - translogHandler = createTranslogHandler(engine.engineConfig.getIndexSettings()); + translogHandler = createTranslogHandler(mapperService); engine.close(); // we need to reuse the engine config unless the parser.mappingModified won't work @@ -3465,7 +3466,7 @@ public void testTranslogReplay() throws IOException { assertEquals(numDocs, translogHandler.appliedOperations()); engine.close(); - translogHandler = createTranslogHandler(engine.engineConfig.getIndexSettings()); + translogHandler = createTranslogHandler(mapperService); engine = createEngine(store, primaryTranslogDir, inSyncGlobalCheckpointSupplier); engine.refresh("warm_up"); assertVisibleCount(engine, numDocs, false); @@ -3519,7 +3520,7 @@ public void testTranslogReplay() throws IOException { } engine.close(); - translogHandler = createTranslogHandler(engine.engineConfig.getIndexSettings()); + translogHandler = createTranslogHandler(mapperService); engine = createEngine(store, primaryTranslogDir, inSyncGlobalCheckpointSupplier); engine.refresh("warm_up"); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { @@ -6452,7 +6453,8 @@ protected void doRun() throws Exception { max, true, randomBoolean(), - randomBoolean() + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) ) ) {} } else { @@ -7789,7 +7791,7 @@ public void testDisableRecoverySource() throws Exception { ) { IllegalStateException exc = expectThrows( IllegalStateException.class, - () -> engine.newChangesSnapshot("test", 0, 1000, true, true, true) + () -> engine.newChangesSnapshot("test", 0, 1000, true, true, true, randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes())) ); assertThat(exc.getMessage(), containsString("unavailable")); } diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index 85ba368165ceb..5863d2f932968 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -10,289 +10,37 @@ package org.elasticsearch.index.engine; import org.apache.lucene.index.NoMergePolicy; -import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.core.IOUtils; -import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.index.IndexVersion; -import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.translog.SnapshotMatchers; import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.test.IndexSettingsModule; import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.LongSupplier; - -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasSize; - -public class LuceneChangesSnapshotTests extends EngineTestCase { +public class LuceneChangesSnapshotTests extends SearchBasedChangesSnapshotTests { @Override - protected Settings indexSettings() { - return Settings.builder() - .put(super.indexSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) // always enable soft-deletes - .build(); - } - - public void testBasics() throws Exception { - long fromSeqNo = randomNonNegativeLong(); - long toSeqNo = randomLongBetween(fromSeqNo, Long.MAX_VALUE); - // Empty engine - try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", fromSeqNo, toSeqNo, true, randomBoolean(), randomBoolean())) { - IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); - assertThat( - error.getMessage(), - containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found") - ); - } - try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", fromSeqNo, toSeqNo, false, randomBoolean(), randomBoolean())) { - assertThat(snapshot, SnapshotMatchers.size(0)); - } - int numOps = between(1, 100); - int refreshedSeqNo = -1; - for (int i = 0; i < numOps; i++) { - String id = Integer.toString(randomIntBetween(i, i + 5)); - ParsedDocument doc = createParsedDoc(id, null, randomBoolean()); - if (randomBoolean()) { - engine.index(indexForDoc(doc)); - } else { - engine.delete(new Engine.Delete(doc.id(), Uid.encodeId(doc.id()), primaryTerm.get())); - } - if (rarely()) { - if (randomBoolean()) { - engine.flush(); - } else { - engine.refresh("test"); - } - refreshedSeqNo = i; - } - } - if (refreshedSeqNo == -1) { - fromSeqNo = between(0, numOps); - toSeqNo = randomLongBetween(fromSeqNo, numOps * 2); - - Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); - try ( - Translog.Snapshot snapshot = new LuceneChangesSnapshot( - searcher, - between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), - fromSeqNo, - toSeqNo, - false, - randomBoolean(), - randomBoolean(), - IndexVersion.current() - ) - ) { - searcher = null; - assertThat(snapshot, SnapshotMatchers.size(0)); - } finally { - IOUtils.close(searcher); - } - - searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); - try ( - Translog.Snapshot snapshot = new LuceneChangesSnapshot( - searcher, - between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), - fromSeqNo, - toSeqNo, - true, - randomBoolean(), - randomBoolean(), - IndexVersion.current() - ) - ) { - searcher = null; - IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); - assertThat( - error.getMessage(), - containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found") - ); - } finally { - IOUtils.close(searcher); - } - } else { - fromSeqNo = randomLongBetween(0, refreshedSeqNo); - toSeqNo = randomLongBetween(refreshedSeqNo + 1, numOps * 2); - Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); - try ( - Translog.Snapshot snapshot = new LuceneChangesSnapshot( - searcher, - between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), - fromSeqNo, - toSeqNo, - false, - randomBoolean(), - randomBoolean(), - IndexVersion.current() - ) - ) { - searcher = null; - assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, refreshedSeqNo)); - } finally { - IOUtils.close(searcher); - } - searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); - try ( - Translog.Snapshot snapshot = new LuceneChangesSnapshot( - searcher, - between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), - fromSeqNo, - toSeqNo, - true, - randomBoolean(), - randomBoolean(), - IndexVersion.current() - ) - ) { - searcher = null; - IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); - assertThat( - error.getMessage(), - containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found") - ); - } finally { - IOUtils.close(searcher); - } - toSeqNo = randomLongBetween(fromSeqNo, refreshedSeqNo); - searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); - try ( - Translog.Snapshot snapshot = new LuceneChangesSnapshot( - searcher, - between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), - fromSeqNo, - toSeqNo, - true, - randomBoolean(), - randomBoolean(), - IndexVersion.current() - ) - ) { - searcher = null; - assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); - } finally { - IOUtils.close(searcher); - } - } - // Get snapshot via engine will auto refresh - fromSeqNo = randomLongBetween(0, numOps - 1); - toSeqNo = randomLongBetween(fromSeqNo, numOps - 1); - try ( - Translog.Snapshot snapshot = engine.newChangesSnapshot( - "test", - fromSeqNo, - toSeqNo, - randomBoolean(), - randomBoolean(), - randomBoolean() - ) - ) { - assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); - } - } - - /** - * A nested document is indexed into Lucene as multiple documents. While the root document has both sequence number and primary term, - * non-root documents don't have primary term but only sequence numbers. This test verifies that {@link LuceneChangesSnapshot} - * correctly skip non-root documents and returns at most one operation per sequence number. - */ - public void testSkipNonRootOfNestedDocuments() throws Exception { - Map seqNoToTerm = new HashMap<>(); - List operations = generateHistoryOnReplica(between(1, 100), randomBoolean(), randomBoolean(), randomBoolean()); - for (Engine.Operation op : operations) { - if (engine.getLocalCheckpointTracker().hasProcessed(op.seqNo()) == false) { - seqNoToTerm.put(op.seqNo(), op.primaryTerm()); - } - applyOperation(engine, op); - if (rarely()) { - engine.refresh("test"); - } - if (rarely()) { - engine.rollTranslogGeneration(); - } - if (rarely()) { - engine.flush(); - } - } - long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); - engine.refresh("test"); - Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); - final boolean accessStats = randomBoolean(); - try ( - Translog.Snapshot snapshot = new LuceneChangesSnapshot( - searcher, - between(1, 100), - 0, - maxSeqNo, - false, - randomBoolean(), - accessStats, - IndexVersion.current() - ) - ) { - if (accessStats) { - assertThat(snapshot.totalOperations(), equalTo(seqNoToTerm.size())); - } - Translog.Operation op; - while ((op = snapshot.next()) != null) { - assertThat(op.toString(), op.primaryTerm(), equalTo(seqNoToTerm.get(op.seqNo()))); - } - assertThat(snapshot.skippedOperations(), equalTo(0)); - } - } - - public void testUpdateAndReadChangesConcurrently() throws Exception { - Follower[] followers = new Follower[between(1, 3)]; - CountDownLatch readyLatch = new CountDownLatch(followers.length + 1); - AtomicBoolean isDone = new AtomicBoolean(); - for (int i = 0; i < followers.length; i++) { - followers[i] = new Follower(engine, isDone, readyLatch); - followers[i].start(); - } - boolean onPrimary = randomBoolean(); - List operations = new ArrayList<>(); - int numOps = frequently() ? scaledRandomIntBetween(1, 1500) : scaledRandomIntBetween(5000, 20_000); - for (int i = 0; i < numOps; i++) { - String id = Integer.toString(randomIntBetween(0, randomBoolean() ? 10 : numOps * 2)); - ParsedDocument doc = createParsedDoc(id, randomAlphaOfLengthBetween(1, 5), randomBoolean()); - final Engine.Operation op; - if (onPrimary) { - if (randomBoolean()) { - op = new Engine.Index(newUid(doc), primaryTerm.get(), doc); - } else { - op = new Engine.Delete(doc.id(), Uid.encodeId(doc.id()), primaryTerm.get()); - } - } else { - if (randomBoolean()) { - op = replicaIndexForDoc(doc, randomNonNegativeLong(), i, randomBoolean()); - } else { - op = replicaDeleteForDoc(doc.id(), randomNonNegativeLong(), i, randomNonNegativeLong()); - } - } - operations.add(op); - } - readyLatch.countDown(); - readyLatch.await(); - Randomness.shuffle(operations); - concurrentlyApplyOps(operations, engine); - assertThat(engine.getLocalCheckpointTracker().getProcessedCheckpoint(), equalTo(operations.size() - 1L)); - isDone.set(true); - for (Follower follower : followers) { - follower.join(); - IOUtils.close(follower.engine, follower.engine.store); - } + protected Translog.Snapshot newRandomSnapshot( + MappingLookup mappingLookup, + Engine.Searcher engineSearcher, + int searchBatchSize, + long fromSeqNo, + long toSeqNo, + boolean requiredFullRange, + boolean singleConsumer, + boolean accessStats, + IndexVersion indexVersionCreated + ) throws IOException { + return new LuceneChangesSnapshot( + engineSearcher, + searchBatchSize, + fromSeqNo, + toSeqNo, + requiredFullRange, + singleConsumer, + accessStats, + indexVersionCreated + ); } public void testAccessStoredFieldsSequentially() throws Exception { @@ -319,7 +67,8 @@ public void testAccessStoredFieldsSequentially() throws Exception { between(1, smallBatch), false, randomBoolean(), - randomBoolean() + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) ) ) { while ((op = snapshot.next()) != null) { @@ -335,7 +84,8 @@ public void testAccessStoredFieldsSequentially() throws Exception { between(20, 100), false, randomBoolean(), - randomBoolean() + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) ) ) { while ((op = snapshot.next()) != null) { @@ -351,7 +101,8 @@ public void testAccessStoredFieldsSequentially() throws Exception { between(21, 100), false, true, - randomBoolean() + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) ) ) { while ((op = snapshot.next()) != null) { @@ -367,7 +118,8 @@ public void testAccessStoredFieldsSequentially() throws Exception { between(21, 100), false, false, - randomBoolean() + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) ) ) { while ((op = snapshot.next()) != null) { @@ -377,165 +129,4 @@ public void testAccessStoredFieldsSequentially() throws Exception { } } } - - class Follower extends Thread { - private final InternalEngine leader; - private final InternalEngine engine; - private final TranslogHandler translogHandler; - private final AtomicBoolean isDone; - private final CountDownLatch readLatch; - - Follower(InternalEngine leader, AtomicBoolean isDone, CountDownLatch readLatch) throws IOException { - this.leader = leader; - this.isDone = isDone; - this.readLatch = readLatch; - this.translogHandler = new TranslogHandler( - xContentRegistry(), - IndexSettingsModule.newIndexSettings(shardId.getIndexName(), leader.engineConfig.getIndexSettings().getSettings()) - ); - this.engine = createEngine(createStore(), createTempDir()); - } - - void pullOperations(InternalEngine follower) throws IOException { - long leaderCheckpoint = leader.getLocalCheckpointTracker().getProcessedCheckpoint(); - long followerCheckpoint = follower.getLocalCheckpointTracker().getProcessedCheckpoint(); - if (followerCheckpoint < leaderCheckpoint) { - long fromSeqNo = followerCheckpoint + 1; - long batchSize = randomLongBetween(0, 100); - long toSeqNo = Math.min(fromSeqNo + batchSize, leaderCheckpoint); - try ( - Translog.Snapshot snapshot = leader.newChangesSnapshot( - "test", - fromSeqNo, - toSeqNo, - true, - randomBoolean(), - randomBoolean() - ) - ) { - translogHandler.run(follower, snapshot); - } - } - } - - @Override - public void run() { - try { - readLatch.countDown(); - readLatch.await(); - while (isDone.get() == false - || engine.getLocalCheckpointTracker().getProcessedCheckpoint() < leader.getLocalCheckpointTracker() - .getProcessedCheckpoint()) { - pullOperations(engine); - } - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine); - // have to verify without source since we are randomly testing without _source - List docsWithoutSourceOnFollower = getDocIds(engine, true).stream() - .map(d -> new DocIdSeqNoAndSource(d.id(), null, d.seqNo(), d.primaryTerm(), d.version())) - .toList(); - List docsWithoutSourceOnLeader = getDocIds(leader, true).stream() - .map(d -> new DocIdSeqNoAndSource(d.id(), null, d.seqNo(), d.primaryTerm(), d.version())) - .toList(); - assertThat(docsWithoutSourceOnFollower, equalTo(docsWithoutSourceOnLeader)); - } catch (Exception ex) { - throw new AssertionError(ex); - } - } - } - - private List drainAll(Translog.Snapshot snapshot) throws IOException { - List operations = new ArrayList<>(); - Translog.Operation op; - while ((op = snapshot.next()) != null) { - final Translog.Operation newOp = op; - logger.trace("Reading [{}]", op); - assert operations.stream().allMatch(o -> o.seqNo() < newOp.seqNo()) : "Operations [" + operations + "], op [" + op + "]"; - operations.add(newOp); - } - return operations; - } - - public void testOverFlow() throws Exception { - long fromSeqNo = randomLongBetween(0, 5); - long toSeqNo = randomLongBetween(Long.MAX_VALUE - 5, Long.MAX_VALUE); - try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", fromSeqNo, toSeqNo, true, randomBoolean(), randomBoolean())) { - IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); - assertThat( - error.getMessage(), - containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found") - ); - } - } - - public void testStats() throws Exception { - try (Store store = createStore(); Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { - int numOps = between(100, 5000); - long startingSeqNo = randomLongBetween(0, Integer.MAX_VALUE); - List operations = generateHistoryOnReplica( - numOps, - startingSeqNo, - randomBoolean(), - randomBoolean(), - randomBoolean() - ); - applyOperations(engine, operations); - - LongSupplier fromSeqNo = () -> { - if (randomBoolean()) { - return 0L; - } else if (randomBoolean()) { - return startingSeqNo; - } else { - return randomLongBetween(0, startingSeqNo); - } - }; - - LongSupplier toSeqNo = () -> { - final long maxSeqNo = engine.getSeqNoStats(-1).getMaxSeqNo(); - if (randomBoolean()) { - return maxSeqNo; - } else if (randomBoolean()) { - return Long.MAX_VALUE; - } else { - return randomLongBetween(maxSeqNo, Long.MAX_VALUE); - } - }; - // Can't access stats if didn't request it - try ( - Translog.Snapshot snapshot = engine.newChangesSnapshot( - "test", - fromSeqNo.getAsLong(), - toSeqNo.getAsLong(), - false, - randomBoolean(), - false - ) - ) { - IllegalStateException error = expectThrows(IllegalStateException.class, snapshot::totalOperations); - assertThat(error.getMessage(), equalTo("Access stats of a snapshot created with [access_stats] is false")); - final List translogOps = drainAll(snapshot); - assertThat(translogOps, hasSize(numOps)); - error = expectThrows(IllegalStateException.class, snapshot::totalOperations); - assertThat(error.getMessage(), equalTo("Access stats of a snapshot created with [access_stats] is false")); - } - // Access stats and operations - try ( - Translog.Snapshot snapshot = engine.newChangesSnapshot( - "test", - fromSeqNo.getAsLong(), - toSeqNo.getAsLong(), - false, - randomBoolean(), - true - ) - ) { - assertThat(snapshot.totalOperations(), equalTo(numOps)); - final List translogOps = drainAll(snapshot); - assertThat(translogOps, hasSize(numOps)); - assertThat(snapshot.totalOperations(), equalTo(numOps)); - } - // Verify count - assertThat(engine.countChanges("test", fromSeqNo.getAsLong(), toSeqNo.getAsLong()), equalTo(numOps)); - } - } } diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneSyntheticSourceChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneSyntheticSourceChangesSnapshotTests.java new file mode 100644 index 0000000000000..2a6c3428d6d45 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneSyntheticSourceChangesSnapshotTests.java @@ -0,0 +1,58 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.index.engine; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.translog.Translog; + +import java.io.IOException; + +import static org.elasticsearch.index.mapper.SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING; + +public class LuceneSyntheticSourceChangesSnapshotTests extends SearchBasedChangesSnapshotTests { + @Override + protected Settings indexSettings() { + return Settings.builder() + .put(super.indexSettings()) + .put(INDEX_MAPPER_SOURCE_MODE_SETTING.getKey(), SourceFieldMapper.Mode.SYNTHETIC.name()) + .put(IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey(), true) + .build(); + } + + @Override + protected Translog.Snapshot newRandomSnapshot( + MappingLookup mappingLookup, + Engine.Searcher engineSearcher, + int searchBatchSize, + long fromSeqNo, + long toSeqNo, + boolean requiredFullRange, + boolean singleConsumer, + boolean accessStats, + IndexVersion indexVersionCreated + ) throws IOException { + return new LuceneSyntheticSourceChangesSnapshot( + mappingLookup, + engineSearcher, + searchBatchSize, + randomLongBetween(0, ByteSizeValue.ofBytes(Integer.MAX_VALUE).getBytes()), + fromSeqNo, + toSeqNo, + requiredFullRange, + accessStats, + indexVersionCreated + ); + } +} diff --git a/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java index e7e668415cdd4..74d6e83aff266 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java @@ -25,6 +25,7 @@ import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.StandardDirectoryReader; +import org.apache.lucene.index.StoredFields; import org.apache.lucene.index.Term; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.MatchAllDocsQuery; @@ -38,82 +39,99 @@ import java.io.IOException; import java.util.Collections; +import java.util.List; import java.util.Set; import java.util.stream.Collectors; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; + public class RecoverySourcePruneMergePolicyTests extends ESTestCase { public void testPruneAll() throws IOException { - try (Directory dir = newDirectory()) { - boolean pruneIdField = randomBoolean(); - IndexWriterConfig iwc = newIndexWriterConfig(); - RecoverySourcePruneMergePolicy mp = new RecoverySourcePruneMergePolicy( - "extra_source", - pruneIdField, - MatchNoDocsQuery::new, - newLogMergePolicy() - ); - iwc.setMergePolicy(new ShuffleForcedMergePolicy(mp)); - try (IndexWriter writer = new IndexWriter(dir, iwc)) { - for (int i = 0; i < 20; i++) { - if (i > 0 && randomBoolean()) { - writer.flush(); - } - Document doc = new Document(); - doc.add(new StoredField(IdFieldMapper.NAME, "_id")); - doc.add(new StoredField("source", "hello world")); - doc.add(new StoredField("extra_source", "hello world")); - doc.add(new NumericDocValuesField("extra_source", 1)); - writer.addDocument(doc); - } - writer.forceMerge(1); - writer.commit(); - try (DirectoryReader reader = DirectoryReader.open(writer)) { - for (int i = 0; i < reader.maxDoc(); i++) { - Document document = reader.document(i); - if (pruneIdField) { - assertEquals(1, document.getFields().size()); - assertEquals("source", document.getFields().get(0).name()); - } else { - assertEquals(2, document.getFields().size()); - assertEquals(IdFieldMapper.NAME, document.getFields().get(0).name()); - assertEquals("source", document.getFields().get(1).name()); + for (boolean pruneIdField : List.of(true, false)) { + for (boolean syntheticRecoverySource : List.of(true, false)) { + try (Directory dir = newDirectory()) { + IndexWriterConfig iwc = newIndexWriterConfig(); + RecoverySourcePruneMergePolicy mp = new RecoverySourcePruneMergePolicy( + syntheticRecoverySource ? null : "extra_source", + syntheticRecoverySource ? "extra_source_size" : "extra_source", + pruneIdField, + MatchNoDocsQuery::new, + newLogMergePolicy() + ); + iwc.setMergePolicy(new ShuffleForcedMergePolicy(mp)); + try (IndexWriter writer = new IndexWriter(dir, iwc)) { + for (int i = 0; i < 20; i++) { + if (i > 0 && randomBoolean()) { + writer.flush(); + } + Document doc = new Document(); + doc.add(new StoredField(IdFieldMapper.NAME, "_id")); + doc.add(new StoredField("source", "hello world")); + if (syntheticRecoverySource) { + doc.add(new NumericDocValuesField("extra_source_size", randomIntBetween(10, 10000))); + } else { + doc.add(new StoredField("extra_source", "hello world")); + doc.add(new NumericDocValuesField("extra_source", 1)); + } + writer.addDocument(doc); } - } - assertEquals(1, reader.leaves().size()); - LeafReader leafReader = reader.leaves().get(0).reader(); - NumericDocValues extra_source = leafReader.getNumericDocValues("extra_source"); - if (extra_source != null) { - assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); - } - if (leafReader instanceof CodecReader codecReader && reader instanceof StandardDirectoryReader sdr) { - SegmentInfos segmentInfos = sdr.getSegmentInfos(); - MergePolicy.MergeSpecification forcedMerges = mp.findForcedDeletesMerges( - segmentInfos, - new MergePolicy.MergeContext() { - @Override - public int numDeletesToMerge(SegmentCommitInfo info) { - return info.info.maxDoc() - 1; + writer.forceMerge(1); + writer.commit(); + try (DirectoryReader reader = DirectoryReader.open(writer)) { + StoredFields storedFields = reader.storedFields(); + for (int i = 0; i < reader.maxDoc(); i++) { + Document document = storedFields.document(i); + if (pruneIdField) { + assertEquals(1, document.getFields().size()); + assertEquals("source", document.getFields().get(0).name()); + } else { + assertEquals(2, document.getFields().size()); + assertEquals(IdFieldMapper.NAME, document.getFields().get(0).name()); + assertEquals("source", document.getFields().get(1).name()); } + } - @Override - public int numDeletedDocs(SegmentCommitInfo info) { - return info.info.maxDoc() - 1; - } + assertEquals(1, reader.leaves().size()); + LeafReader leafReader = reader.leaves().get(0).reader(); - @Override - public InfoStream getInfoStream() { - return new NullInfoStream(); - } + NumericDocValues extra_source = leafReader.getNumericDocValues( + syntheticRecoverySource ? "extra_source_size" : "extra_source" + ); + if (extra_source != null) { + assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); + } + if (leafReader instanceof CodecReader codecReader && reader instanceof StandardDirectoryReader sdr) { + SegmentInfos segmentInfos = sdr.getSegmentInfos(); + MergePolicy.MergeSpecification forcedMerges = mp.findForcedDeletesMerges( + segmentInfos, + new MergePolicy.MergeContext() { + @Override + public int numDeletesToMerge(SegmentCommitInfo info) { + return info.info.maxDoc() - 1; + } - @Override - public Set getMergingSegments() { - return Collections.emptySet(); - } + @Override + public int numDeletedDocs(SegmentCommitInfo info) { + return info.info.maxDoc() - 1; + } + + @Override + public InfoStream getInfoStream() { + return new NullInfoStream(); + } + + @Override + public Set getMergingSegments() { + return Collections.emptySet(); + } + } + ); + // don't wrap if there is nothing to do + assertSame(codecReader, forcedMerges.merges.get(0).wrapForMerge(codecReader)); } - ); - // don't wrap if there is nothing to do - assertSame(codecReader, forcedMerges.merges.get(0).wrapForMerge(codecReader)); + } } } } @@ -121,85 +139,126 @@ public Set getMergingSegments() { } public void testPruneSome() throws IOException { - try (Directory dir = newDirectory()) { - boolean pruneIdField = randomBoolean(); - IndexWriterConfig iwc = newIndexWriterConfig(); - iwc.setMergePolicy( - new RecoverySourcePruneMergePolicy( - "extra_source", - pruneIdField, - () -> new TermQuery(new Term("even", "true")), - iwc.getMergePolicy() - ) - ); - try (IndexWriter writer = new IndexWriter(dir, iwc)) { - for (int i = 0; i < 20; i++) { - if (i > 0 && randomBoolean()) { - writer.flush(); - } - Document doc = new Document(); - doc.add(new StoredField(IdFieldMapper.NAME, "_id")); - doc.add(new StringField("even", Boolean.toString(i % 2 == 0), Field.Store.YES)); - doc.add(new StoredField("source", "hello world")); - doc.add(new StoredField("extra_source", "hello world")); - doc.add(new NumericDocValuesField("extra_source", 1)); - writer.addDocument(doc); - } - writer.forceMerge(1); - writer.commit(); - try (DirectoryReader reader = DirectoryReader.open(writer)) { - assertEquals(1, reader.leaves().size()); - NumericDocValues extra_source = reader.leaves().get(0).reader().getNumericDocValues("extra_source"); - assertNotNull(extra_source); - for (int i = 0; i < reader.maxDoc(); i++) { - Document document = reader.document(i); - Set collect = document.getFields().stream().map(IndexableField::name).collect(Collectors.toSet()); - assertTrue(collect.contains("source")); - assertTrue(collect.contains("even")); - if (collect.size() == 4) { - assertTrue(collect.contains("extra_source")); - assertTrue(collect.contains(IdFieldMapper.NAME)); - assertEquals("true", document.getField("even").stringValue()); - assertEquals(i, extra_source.nextDoc()); - } else { - assertEquals(pruneIdField ? 2 : 3, document.getFields().size()); + for (boolean pruneIdField : List.of(true, false)) { + for (boolean syntheticRecoverySource : List.of(true, false)) { + try (Directory dir = newDirectory()) { + IndexWriterConfig iwc = newIndexWriterConfig(); + iwc.setMergePolicy( + new RecoverySourcePruneMergePolicy( + syntheticRecoverySource ? null : "extra_source", + syntheticRecoverySource ? "extra_source_size" : "extra_source", + pruneIdField, + () -> new TermQuery(new Term("even", "true")), + iwc.getMergePolicy() + ) + ); + try (IndexWriter writer = new IndexWriter(dir, iwc)) { + for (int i = 0; i < 20; i++) { + if (i > 0 && randomBoolean()) { + writer.flush(); + } + Document doc = new Document(); + doc.add(new StoredField(IdFieldMapper.NAME, "_id")); + doc.add(new StringField("even", Boolean.toString(i % 2 == 0), Field.Store.YES)); + doc.add(new StoredField("source", "hello world")); + if (syntheticRecoverySource) { + doc.add(new NumericDocValuesField("extra_source_size", randomIntBetween(10, 10000))); + } else { + doc.add(new StoredField("extra_source", "hello world")); + doc.add(new NumericDocValuesField("extra_source", 1)); + } + writer.addDocument(doc); + } + writer.forceMerge(1); + writer.commit(); + try (DirectoryReader reader = DirectoryReader.open(writer)) { + assertEquals(1, reader.leaves().size()); + String extraSourceDVName = syntheticRecoverySource ? "extra_source_size" : "extra_source"; + NumericDocValues extra_source = reader.leaves().get(0).reader().getNumericDocValues(extraSourceDVName); + assertNotNull(extra_source); + StoredFields storedFields = reader.storedFields(); + for (int i = 0; i < reader.maxDoc(); i++) { + Document document = storedFields.document(i); + Set collect = document.getFields().stream().map(IndexableField::name).collect(Collectors.toSet()); + assertTrue(collect.contains("source")); + assertTrue(collect.contains("even")); + boolean isEven = Boolean.parseBoolean(document.getField("even").stringValue()); + if (isEven) { + assertTrue(collect.contains(IdFieldMapper.NAME)); + assertThat(collect.contains("extra_source"), equalTo(syntheticRecoverySource == false)); + if (extra_source.docID() < i) { + extra_source.advance(i); + } + assertEquals(i, extra_source.docID()); + if (syntheticRecoverySource) { + assertThat(extra_source.longValue(), greaterThan(10L)); + } else { + assertThat(extra_source.longValue(), equalTo(1L)); + } + } else { + assertThat(collect.contains(IdFieldMapper.NAME), equalTo(pruneIdField == false)); + assertFalse(collect.contains("extra_source")); + if (extra_source.docID() < i) { + extra_source.advance(i); + } + assertNotEquals(i, extra_source.docID()); + } + } + if (extra_source.docID() != DocIdSetIterator.NO_MORE_DOCS) { + assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); + } } } - assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); } } } } public void testPruneNone() throws IOException { - try (Directory dir = newDirectory()) { - IndexWriterConfig iwc = newIndexWriterConfig(); - iwc.setMergePolicy(new RecoverySourcePruneMergePolicy("extra_source", false, MatchAllDocsQuery::new, iwc.getMergePolicy())); - try (IndexWriter writer = new IndexWriter(dir, iwc)) { - for (int i = 0; i < 20; i++) { - if (i > 0 && randomBoolean()) { - writer.flush(); + for (boolean syntheticRecoverySource : List.of(true, false)) { + try (Directory dir = newDirectory()) { + IndexWriterConfig iwc = newIndexWriterConfig(); + iwc.setMergePolicy( + new RecoverySourcePruneMergePolicy( + syntheticRecoverySource ? null : "extra_source", + syntheticRecoverySource ? "extra_source_size" : "extra_source", + false, + MatchAllDocsQuery::new, + iwc.getMergePolicy() + ) + ); + try (IndexWriter writer = new IndexWriter(dir, iwc)) { + for (int i = 0; i < 20; i++) { + if (i > 0 && randomBoolean()) { + writer.flush(); + } + Document doc = new Document(); + doc.add(new StoredField("source", "hello world")); + if (syntheticRecoverySource) { + doc.add(new NumericDocValuesField("extra_source_size", randomIntBetween(10, 10000))); + } else { + doc.add(new StoredField("extra_source", "hello world")); + doc.add(new NumericDocValuesField("extra_source", 1)); + } + writer.addDocument(doc); } - Document doc = new Document(); - doc.add(new StoredField("source", "hello world")); - doc.add(new StoredField("extra_source", "hello world")); - doc.add(new NumericDocValuesField("extra_source", 1)); - writer.addDocument(doc); - } - writer.forceMerge(1); - writer.commit(); - try (DirectoryReader reader = DirectoryReader.open(writer)) { - assertEquals(1, reader.leaves().size()); - NumericDocValues extra_source = reader.leaves().get(0).reader().getNumericDocValues("extra_source"); - assertNotNull(extra_source); - for (int i = 0; i < reader.maxDoc(); i++) { - Document document = reader.document(i); - Set collect = document.getFields().stream().map(IndexableField::name).collect(Collectors.toSet()); - assertTrue(collect.contains("source")); - assertTrue(collect.contains("extra_source")); - assertEquals(i, extra_source.nextDoc()); + writer.forceMerge(1); + writer.commit(); + try (DirectoryReader reader = DirectoryReader.open(writer)) { + assertEquals(1, reader.leaves().size()); + String extraSourceDVName = syntheticRecoverySource ? "extra_source_size" : "extra_source"; + NumericDocValues extra_source = reader.leaves().get(0).reader().getNumericDocValues(extraSourceDVName); + assertNotNull(extra_source); + StoredFields storedFields = reader.storedFields(); + for (int i = 0; i < reader.maxDoc(); i++) { + Document document = storedFields.document(i); + Set collect = document.getFields().stream().map(IndexableField::name).collect(Collectors.toSet()); + assertTrue(collect.contains("source")); + assertThat(collect.contains("extra_source"), equalTo(syntheticRecoverySource == false)); + assertEquals(i, extra_source.nextDoc()); + } + assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); } - assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); } } } diff --git a/server/src/test/java/org/elasticsearch/index/engine/SearchBasedChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/SearchBasedChangesSnapshotTests.java new file mode 100644 index 0000000000000..9cfa7321973a4 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/SearchBasedChangesSnapshotTests.java @@ -0,0 +1,507 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.index.NoMergePolicy; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.SnapshotMatchers; +import org.elasticsearch.index.translog.Translog; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.LongSupplier; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + +public abstract class SearchBasedChangesSnapshotTests extends EngineTestCase { + @Override + protected Settings indexSettings() { + return Settings.builder() + .put(super.indexSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) // always enable soft-deletes + .build(); + } + + protected abstract Translog.Snapshot newRandomSnapshot( + MappingLookup mappingLookup, + Engine.Searcher engineSearcher, + int searchBatchSize, + long fromSeqNo, + long toSeqNo, + boolean requiredFullRange, + boolean singleConsumer, + boolean accessStats, + IndexVersion indexVersionCreated + ) throws IOException; + + public void testBasics() throws Exception { + long fromSeqNo = randomNonNegativeLong(); + long toSeqNo = randomLongBetween(fromSeqNo, Long.MAX_VALUE); + // Empty engine + try ( + Translog.Snapshot snapshot = engine.newChangesSnapshot( + "test", + fromSeqNo, + toSeqNo, + true, + randomBoolean(), + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) + ) { + IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); + assertThat( + error.getMessage(), + containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found") + ); + } + try ( + Translog.Snapshot snapshot = engine.newChangesSnapshot( + "test", + fromSeqNo, + toSeqNo, + false, + randomBoolean(), + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) + ) { + assertThat(snapshot, SnapshotMatchers.size(0)); + } + int numOps = between(1, 100); + int refreshedSeqNo = -1; + for (int i = 0; i < numOps; i++) { + String id = Integer.toString(randomIntBetween(i, i + 5)); + ParsedDocument doc = parseDocument(engine.engineConfig.getMapperService(), id, null); + if (randomBoolean()) { + engine.index(indexForDoc(doc)); + } else { + engine.delete(new Engine.Delete(doc.id(), Uid.encodeId(doc.id()), primaryTerm.get())); + } + if (rarely()) { + if (randomBoolean()) { + engine.flush(); + } else { + engine.refresh("test"); + } + refreshedSeqNo = i; + } + } + if (refreshedSeqNo == -1) { + fromSeqNo = between(0, numOps); + toSeqNo = randomLongBetween(fromSeqNo, numOps * 2); + + Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); + try ( + Translog.Snapshot snapshot = newRandomSnapshot( + engine.engineConfig.getMapperService().mappingLookup(), + searcher, + between(1, SearchBasedChangesSnapshot.DEFAULT_BATCH_SIZE), + fromSeqNo, + toSeqNo, + false, + randomBoolean(), + randomBoolean(), + IndexVersion.current() + ) + ) { + searcher = null; + assertThat(snapshot, SnapshotMatchers.size(0)); + } finally { + IOUtils.close(searcher); + } + + searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); + try ( + Translog.Snapshot snapshot = newRandomSnapshot( + engine.engineConfig.getMapperService().mappingLookup(), + searcher, + between(1, SearchBasedChangesSnapshot.DEFAULT_BATCH_SIZE), + fromSeqNo, + toSeqNo, + true, + randomBoolean(), + randomBoolean(), + IndexVersion.current() + ) + ) { + searcher = null; + IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); + assertThat( + error.getMessage(), + containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found") + ); + } finally { + IOUtils.close(searcher); + } + } else { + fromSeqNo = randomLongBetween(0, refreshedSeqNo); + toSeqNo = randomLongBetween(refreshedSeqNo + 1, numOps * 2); + Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); + try ( + Translog.Snapshot snapshot = newRandomSnapshot( + engine.engineConfig.getMapperService().mappingLookup(), + searcher, + between(1, SearchBasedChangesSnapshot.DEFAULT_BATCH_SIZE), + fromSeqNo, + toSeqNo, + false, + randomBoolean(), + randomBoolean(), + IndexVersion.current() + ) + ) { + searcher = null; + assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, refreshedSeqNo)); + } finally { + IOUtils.close(searcher); + } + searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); + try ( + Translog.Snapshot snapshot = newRandomSnapshot( + engine.engineConfig.getMapperService().mappingLookup(), + searcher, + between(1, SearchBasedChangesSnapshot.DEFAULT_BATCH_SIZE), + fromSeqNo, + toSeqNo, + true, + randomBoolean(), + randomBoolean(), + IndexVersion.current() + ) + ) { + searcher = null; + IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); + assertThat( + error.getMessage(), + containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found") + ); + } finally { + IOUtils.close(searcher); + } + toSeqNo = randomLongBetween(fromSeqNo, refreshedSeqNo); + searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); + try ( + Translog.Snapshot snapshot = newRandomSnapshot( + engine.engineConfig.getMapperService().mappingLookup(), + searcher, + between(1, SearchBasedChangesSnapshot.DEFAULT_BATCH_SIZE), + fromSeqNo, + toSeqNo, + true, + randomBoolean(), + randomBoolean(), + IndexVersion.current() + ) + ) { + searcher = null; + assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); + } finally { + IOUtils.close(searcher); + } + } + // Get snapshot via engine will auto refresh + fromSeqNo = randomLongBetween(0, numOps - 1); + toSeqNo = randomLongBetween(fromSeqNo, numOps - 1); + try ( + Translog.Snapshot snapshot = engine.newChangesSnapshot( + "test", + fromSeqNo, + toSeqNo, + randomBoolean(), + randomBoolean(), + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) + ) { + assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); + } + } + + /** + * A nested document is indexed into Lucene as multiple documents. While the root document has both sequence number and primary term, + * non-root documents don't have primary term but only sequence numbers. This test verifies that {@link LuceneChangesSnapshot} + * correctly skip non-root documents and returns at most one operation per sequence number. + */ + public void testSkipNonRootOfNestedDocuments() throws Exception { + Map seqNoToTerm = new HashMap<>(); + List operations = generateHistoryOnReplica(between(1, 100), randomBoolean(), randomBoolean(), randomBoolean()); + for (Engine.Operation op : operations) { + if (engine.getLocalCheckpointTracker().hasProcessed(op.seqNo()) == false) { + seqNoToTerm.put(op.seqNo(), op.primaryTerm()); + } + applyOperation(engine, op); + if (rarely()) { + engine.refresh("test"); + } + if (rarely()) { + engine.rollTranslogGeneration(); + } + if (rarely()) { + engine.flush(); + } + } + long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); + engine.refresh("test"); + Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); + final boolean accessStats = randomBoolean(); + try ( + Translog.Snapshot snapshot = newRandomSnapshot( + engine.engineConfig.getMapperService().mappingLookup(), + searcher, + between(1, 100), + 0, + maxSeqNo, + false, + randomBoolean(), + accessStats, + IndexVersion.current() + ) + ) { + if (accessStats) { + assertThat(snapshot.totalOperations(), equalTo(seqNoToTerm.size())); + } + Translog.Operation op; + while ((op = snapshot.next()) != null) { + assertThat(op.toString(), op.primaryTerm(), equalTo(seqNoToTerm.get(op.seqNo()))); + } + assertThat(snapshot.skippedOperations(), equalTo(0)); + } + } + + public void testUpdateAndReadChangesConcurrently() throws Exception { + Follower[] followers = new Follower[between(1, 3)]; + CountDownLatch readyLatch = new CountDownLatch(followers.length + 1); + AtomicBoolean isDone = new AtomicBoolean(); + for (int i = 0; i < followers.length; i++) { + followers[i] = new Follower(engine, isDone, readyLatch); + followers[i].start(); + } + boolean onPrimary = randomBoolean(); + List operations = new ArrayList<>(); + int numOps = frequently() ? scaledRandomIntBetween(1, 1500) : scaledRandomIntBetween(5000, 20_000); + for (int i = 0; i < numOps; i++) { + String id = Integer.toString(randomIntBetween(0, randomBoolean() ? 10 : numOps * 2)); + ParsedDocument doc = parseDocument(engine.engineConfig.getMapperService(), id, randomAlphaOfLengthBetween(1, 5)); + final Engine.Operation op; + if (onPrimary) { + if (randomBoolean()) { + op = new Engine.Index(newUid(doc), primaryTerm.get(), doc); + } else { + op = new Engine.Delete(doc.id(), Uid.encodeId(doc.id()), primaryTerm.get()); + } + } else { + if (randomBoolean()) { + op = replicaIndexForDoc(doc, randomNonNegativeLong(), i, randomBoolean()); + } else { + op = replicaDeleteForDoc(doc.id(), randomNonNegativeLong(), i, randomNonNegativeLong()); + } + } + operations.add(op); + } + readyLatch.countDown(); + readyLatch.await(); + Randomness.shuffle(operations); + concurrentlyApplyOps(operations, engine); + assertThat(engine.getLocalCheckpointTracker().getProcessedCheckpoint(), equalTo(operations.size() - 1L)); + isDone.set(true); + for (Follower follower : followers) { + follower.join(); + IOUtils.close(follower.engine, follower.engine.store); + } + } + + class Follower extends Thread { + private final InternalEngine leader; + private final InternalEngine engine; + private final TranslogHandler translogHandler; + private final AtomicBoolean isDone; + private final CountDownLatch readLatch; + + Follower(InternalEngine leader, AtomicBoolean isDone, CountDownLatch readLatch) throws IOException { + this.leader = leader; + this.isDone = isDone; + this.readLatch = readLatch; + this.engine = createEngine(defaultSettings, createStore(), createTempDir(), newMergePolicy()); + this.translogHandler = new TranslogHandler(engine.engineConfig.getMapperService()); + } + + void pullOperations(InternalEngine follower) throws IOException { + long leaderCheckpoint = leader.getLocalCheckpointTracker().getProcessedCheckpoint(); + long followerCheckpoint = follower.getLocalCheckpointTracker().getProcessedCheckpoint(); + if (followerCheckpoint < leaderCheckpoint) { + long fromSeqNo = followerCheckpoint + 1; + long batchSize = randomLongBetween(0, 100); + long toSeqNo = Math.min(fromSeqNo + batchSize, leaderCheckpoint); + try ( + Translog.Snapshot snapshot = leader.newChangesSnapshot( + "test", + fromSeqNo, + toSeqNo, + true, + randomBoolean(), + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) + ) { + translogHandler.run(follower, snapshot); + } + } + } + + @Override + public void run() { + try { + readLatch.countDown(); + readLatch.await(); + while (isDone.get() == false + || engine.getLocalCheckpointTracker().getProcessedCheckpoint() < leader.getLocalCheckpointTracker() + .getProcessedCheckpoint()) { + pullOperations(engine); + } + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine); + // have to verify without source since we are randomly testing without _source + List docsWithoutSourceOnFollower = getDocIds(engine, true).stream() + .map(d -> new DocIdSeqNoAndSource(d.id(), null, d.seqNo(), d.primaryTerm(), d.version())) + .toList(); + List docsWithoutSourceOnLeader = getDocIds(leader, true).stream() + .map(d -> new DocIdSeqNoAndSource(d.id(), null, d.seqNo(), d.primaryTerm(), d.version())) + .toList(); + assertThat(docsWithoutSourceOnFollower, equalTo(docsWithoutSourceOnLeader)); + } catch (Exception ex) { + throw new AssertionError(ex); + } + } + } + + private List drainAll(Translog.Snapshot snapshot) throws IOException { + List operations = new ArrayList<>(); + Translog.Operation op; + while ((op = snapshot.next()) != null) { + final Translog.Operation newOp = op; + logger.trace("Reading [{}]", op); + assert operations.stream().allMatch(o -> o.seqNo() < newOp.seqNo()) : "Operations [" + operations + "], op [" + op + "]"; + operations.add(newOp); + } + return operations; + } + + public void testOverFlow() throws Exception { + long fromSeqNo = randomLongBetween(0, 5); + long toSeqNo = randomLongBetween(Long.MAX_VALUE - 5, Long.MAX_VALUE); + try ( + Translog.Snapshot snapshot = engine.newChangesSnapshot( + "test", + fromSeqNo, + toSeqNo, + true, + randomBoolean(), + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) + ) { + IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); + assertThat( + error.getMessage(), + containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found") + ); + } + } + + public void testStats() throws Exception { + try (Store store = createStore(); Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { + int numOps = between(100, 5000); + long startingSeqNo = randomLongBetween(0, Integer.MAX_VALUE); + List operations = generateHistoryOnReplica( + numOps, + startingSeqNo, + randomBoolean(), + randomBoolean(), + randomBoolean() + ); + applyOperations(engine, operations); + + LongSupplier fromSeqNo = () -> { + if (randomBoolean()) { + return 0L; + } else if (randomBoolean()) { + return startingSeqNo; + } else { + return randomLongBetween(0, startingSeqNo); + } + }; + + LongSupplier toSeqNo = () -> { + final long maxSeqNo = engine.getSeqNoStats(-1).getMaxSeqNo(); + if (randomBoolean()) { + return maxSeqNo; + } else if (randomBoolean()) { + return Long.MAX_VALUE; + } else { + return randomLongBetween(maxSeqNo, Long.MAX_VALUE); + } + }; + // Can't access stats if didn't request it + try ( + Translog.Snapshot snapshot = engine.newChangesSnapshot( + "test", + fromSeqNo.getAsLong(), + toSeqNo.getAsLong(), + false, + randomBoolean(), + false, + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) + ) { + IllegalStateException error = expectThrows(IllegalStateException.class, snapshot::totalOperations); + assertThat(error.getMessage(), equalTo("Access stats of a snapshot created with [access_stats] is false")); + final List translogOps = drainAll(snapshot); + assertThat(translogOps, hasSize(numOps)); + error = expectThrows(IllegalStateException.class, snapshot::totalOperations); + assertThat(error.getMessage(), equalTo("Access stats of a snapshot created with [access_stats] is false")); + } + // Access stats and operations + try ( + Translog.Snapshot snapshot = engine.newChangesSnapshot( + "test", + fromSeqNo.getAsLong(), + toSeqNo.getAsLong(), + false, + randomBoolean(), + true, + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) + ) { + assertThat(snapshot.totalOperations(), equalTo(numOps)); + final List translogOps = drainAll(snapshot); + assertThat(translogOps, hasSize(numOps)); + assertThat(snapshot.totalOperations(), equalTo(numOps)); + } + // Verify count + assertThat(engine.countChanges("test", fromSeqNo.getAsLong(), toSeqNo.getAsLong()), equalTo(numOps)); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java index 4d6a30849e263..e844edd27f1ab 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.List; +import java.util.Locale; import java.util.Map; import static org.elasticsearch.indices.recovery.RecoverySettings.INDICES_RECOVERY_SOURCE_ENABLED_SETTING; @@ -405,16 +406,114 @@ public void testRecoverySourceWithSourceDisabled() throws IOException { } } - public void testRecoverySourceWithSyntheticSource() throws IOException { + public void testRecoverySourceWitInvalidSettings() { { - MapperService mapperService = createMapperService( - topMapping(b -> b.startObject(SourceFieldMapper.NAME).field("mode", "synthetic").endObject()) + Settings settings = Settings.builder().put(IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey(), true).build(); + IllegalArgumentException exc = expectThrows( + IllegalArgumentException.class, + () -> createMapperService(settings, topMapping(b -> {})) + ); + assertThat( + exc.getMessage(), + containsString( + String.format( + Locale.ROOT, + "The setting [%s] is only permitted", + IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey() + ) + ) + ); + } + + { + Settings settings = Settings.builder() + .put(SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.getKey(), SourceFieldMapper.Mode.STORED.toString()) + .put(IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey(), true) + .build(); + IllegalArgumentException exc = expectThrows( + IllegalArgumentException.class, + () -> createMapperService(settings, topMapping(b -> {})) + ); + assertThat( + exc.getMessage(), + containsString( + String.format( + Locale.ROOT, + "The setting [%s] is only permitted", + IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey() + ) + ) + ); + } + { + Settings settings = Settings.builder() + .put(IndexSettings.MODE.getKey(), IndexMode.STANDARD.toString()) + .put(IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey(), true) + .build(); + IllegalArgumentException exc = expectThrows( + IllegalArgumentException.class, + () -> createMapperService(settings, topMapping(b -> {})) + ); + assertThat( + exc.getMessage(), + containsString( + String.format( + Locale.ROOT, + "The setting [%s] is only permitted", + IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey() + ) + ) ); + } + { + Settings settings = Settings.builder() + .put(SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.getKey(), SourceFieldMapper.Mode.SYNTHETIC.toString()) + .put(IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey(), true) + .build(); + IllegalArgumentException exc = expectThrows( + IllegalArgumentException.class, + () -> createMapperService( + IndexVersionUtils.randomPreviousCompatibleVersion(random(), IndexVersions.USE_SYNTHETIC_SOURCE_FOR_RECOVERY_BACKPORT), + settings, + () -> false, + topMapping(b -> {}) + ) + ); + assertThat( + exc.getMessage(), + containsString( + String.format( + Locale.ROOT, + "The setting [%s] is unavailable on this cluster", + IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey() + ) + ) + ); + } + } + + public void testRecoverySourceWithSyntheticSource() throws IOException { + { + Settings settings = Settings.builder() + .put(SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.getKey(), SourceFieldMapper.Mode.SYNTHETIC.toString()) + .build(); + MapperService mapperService = createMapperService(settings, topMapping(b -> {})); DocumentMapper docMapper = mapperService.documentMapper(); - ParsedDocument doc = docMapper.parse(source(b -> { b.field("field1", "value1"); })); + ParsedDocument doc = docMapper.parse(source(b -> b.field("field1", "value1"))); assertNotNull(doc.rootDoc().getField("_recovery_source")); assertThat(doc.rootDoc().getField("_recovery_source").binaryValue(), equalTo(new BytesRef("{\"field1\":\"value1\"}"))); } + { + Settings settings = Settings.builder() + .put(SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.getKey(), SourceFieldMapper.Mode.SYNTHETIC.toString()) + .put(IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey(), true) + .build(); + MapperService mapperService = createMapperService(settings, topMapping(b -> {})); + DocumentMapper docMapper = mapperService.documentMapper(); + ParsedDocument doc = docMapper.parse(source(b -> b.field("field1", "value1"))); + assertNotNull(doc.rootDoc().getField("_recovery_source_size")); + assertThat(doc.rootDoc().getField("_recovery_source_size").numericValue(), equalTo(19L)); + } { Settings settings = Settings.builder().put(INDICES_RECOVERY_SOURCE_ENABLED_SETTING.getKey(), false).build(); MapperService mapperService = createMapperService( @@ -436,6 +535,17 @@ public void testRecoverySourceWithLogs() throws IOException { assertNotNull(doc.rootDoc().getField("_recovery_source")); assertThat(doc.rootDoc().getField("_recovery_source").binaryValue(), equalTo(new BytesRef("{\"@timestamp\":\"2012-02-13\"}"))); } + { + Settings settings = Settings.builder() + .put(IndexSettings.MODE.getKey(), IndexMode.LOGSDB.getName()) + .put(IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey(), true) + .build(); + MapperService mapperService = createMapperService(settings, mapping(b -> {})); + DocumentMapper docMapper = mapperService.documentMapper(); + ParsedDocument doc = docMapper.parse(source(b -> { b.field("@timestamp", "2012-02-13"); })); + assertNotNull(doc.rootDoc().getField("_recovery_source_size")); + assertThat(doc.rootDoc().getField("_recovery_source_size").numericValue(), equalTo(27L)); + } { Settings settings = Settings.builder() .put(IndexSettings.MODE.getKey(), IndexMode.LOGSDB.getName()) diff --git a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index 47c75ee38ee1b..4c6b912e5d3a3 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexSettings; @@ -486,7 +487,8 @@ protected EngineFactory getEngineFactory(ShardRouting routing) { Long.MAX_VALUE, false, randomBoolean(), - randomBoolean() + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) ) ) { assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); @@ -513,7 +515,8 @@ protected EngineFactory getEngineFactory(ShardRouting routing) { Long.MAX_VALUE, false, randomBoolean(), - randomBoolean() + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) ) ) { assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); @@ -608,7 +611,17 @@ public void testSeqNoCollision() throws Exception { shards.promoteReplicaToPrimary(replica2).get(); logger.info("--> Recover replica3 from replica2"); recoverReplica(replica3, replica2, true); - try (Translog.Snapshot snapshot = replica3.newChangesSnapshot("test", 0, Long.MAX_VALUE, false, randomBoolean(), true)) { + try ( + Translog.Snapshot snapshot = replica3.newChangesSnapshot( + "test", + 0, + Long.MAX_VALUE, + false, + randomBoolean(), + true, + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) + ) { assertThat(snapshot.totalOperations(), equalTo(initDocs + 1)); final List expectedOps = new ArrayList<>(initOperations); expectedOps.add(op2); diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index f15506676dc39..bde48da4e5bfd 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -1819,7 +1819,15 @@ public void testShardFieldStats() throws IOException { shard.refresh("test"); } else { // trigger internal refresh - shard.newChangesSnapshot("test", 0, Long.MAX_VALUE, false, randomBoolean(), randomBoolean()).close(); + shard.newChangesSnapshot( + "test", + 0, + Long.MAX_VALUE, + false, + randomBoolean(), + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ).close(); } assertThat(shard.getShardFieldStats(), sameInstance(stats)); // index more docs @@ -1837,7 +1845,15 @@ public void testShardFieldStats() throws IOException { shard.refresh("test"); } else { // trigger internal refresh - shard.newChangesSnapshot("test", 0, Long.MAX_VALUE, false, randomBoolean(), randomBoolean()).close(); + shard.newChangesSnapshot( + "test", + 0, + Long.MAX_VALUE, + false, + randomBoolean(), + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ).close(); } stats = shard.getShardFieldStats(); assertThat(stats.numSegments(), equalTo(2)); diff --git a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index 6fa405c091da1..bb1f47f9db255 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -158,7 +158,7 @@ public void onFailedEngine(String reason, @Nullable Exception e) { System::nanoTime, null, true, - null + EngineTestCase.createMapperService() ); engine = new InternalEngine(config); EngineTestCase.recoverFromTranslog(engine, (e, s) -> 0, Long.MAX_VALUE); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index 315eaaf9ffaf1..aef58cee04899 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.VersionType; @@ -211,7 +212,8 @@ public void testRecoveryWithOutOfOrderDeleteWithSoftDeletes() throws Exception { Long.MAX_VALUE, false, randomBoolean(), - randomBoolean() + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) ) ) { assertThat(snapshot, SnapshotMatchers.size(6)); diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 0b5803e9887d6..9e6d2e94aa599 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -57,13 +57,12 @@ import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.common.CheckedBiFunction; import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.core.CheckedFunction; import org.elasticsearch.core.IOUtils; @@ -141,6 +140,7 @@ import static org.elasticsearch.index.engine.Engine.Operation.Origin.PEER_RECOVERY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -159,6 +159,8 @@ public abstract class EngineTestCase extends ESTestCase { protected Store store; protected Store storeReplica; + protected MapperService mapperService; + protected InternalEngine engine; protected InternalEngine replicaEngine; @@ -197,6 +199,27 @@ protected Settings indexSettings() { .build(); } + protected String defaultMapping() { + return """ + { + "dynamic": false, + "properties": { + "value": { + "type": "keyword" + }, + "nested_field": { + "type": "nested", + "properties": { + "field-0": { + "type": "keyword" + } + } + } + } + } + """; + } + @Override @Before public void setUp() throws Exception { @@ -211,15 +234,16 @@ public void setUp() throws Exception { } else { codecName = "default"; } - defaultSettings = IndexSettingsModule.newIndexSettings("test", indexSettings()); + defaultSettings = IndexSettingsModule.newIndexSettings("index", indexSettings()); threadPool = new TestThreadPool(getClass().getName()); store = createStore(); storeReplica = createStore(); Lucene.cleanLuceneIndex(store.directory()); Lucene.cleanLuceneIndex(storeReplica.directory()); primaryTranslogDir = createTempDir("translog-primary"); - translogHandler = createTranslogHandler(defaultSettings); - engine = createEngine(store, primaryTranslogDir); + mapperService = createMapperService(defaultSettings.getSettings(), defaultMapping()); + translogHandler = createTranslogHandler(mapperService); + engine = createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy()); LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig(); assertEquals(engine.config().getCodec().getName(), codecService.codec(codecName).getName()); @@ -229,7 +253,7 @@ public void setUp() throws Exception { engine.config().setEnableGcDeletes(false); } replicaTranslogDir = createTempDir("translog-replica"); - replicaEngine = createEngine(storeReplica, replicaTranslogDir); + replicaEngine = createEngine(defaultSettings, storeReplica, replicaTranslogDir, newMergePolicy()); currentIndexWriterConfig = replicaEngine.getCurrentIndexWriterConfig(); assertEquals(replicaEngine.config().getCodec().getName(), codecService.codec(codecName).getName()); @@ -432,37 +456,9 @@ protected static ParsedDocument testParsedDocument( ); } - public static CheckedBiFunction nestedParsedDocFactory() throws Exception { - final MapperService mapperService = createMapperService(); - final String nestedMapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("type") - .startObject("properties") - .startObject("nested_field") - .field("type", "nested") - .endObject() - .endObject() - .endObject() - .endObject() - ); - final DocumentMapper nestedMapper = mapperService.merge( - "type", - new CompressedXContent(nestedMapping), - MapperService.MergeReason.MAPPING_UPDATE - ); - return (docId, nestedFieldValues) -> { - final XContentBuilder source = XContentFactory.jsonBuilder().startObject().field("field", "value"); - if (nestedFieldValues > 0) { - XContentBuilder nestedField = source.startObject("nested_field"); - for (int i = 0; i < nestedFieldValues; i++) { - nestedField.field("field-" + i, "value-" + i); - } - source.endObject(); - } - source.endObject(); - return nestedMapper.parse(new SourceToParse(docId, BytesReference.bytes(source), XContentType.JSON)); - }; + public static ParsedDocument parseDocument(MapperService mapperService, String id, String routing) { + SourceToParse sourceToParse = new SourceToParse(id, new BytesArray("{ \"value\" : \"test\" }"), XContentType.JSON, routing); + return mapperService.documentMapper().parse(sourceToParse); } protected Store createStore() throws IOException { @@ -499,8 +495,8 @@ protected Translog createTranslog(Path translogPath, LongSupplier primaryTermSup ); } - protected TranslogHandler createTranslogHandler(IndexSettings indexSettings) { - return new TranslogHandler(xContentRegistry(), indexSettings); + protected TranslogHandler createTranslogHandler(MapperService mapperService) { + return new TranslogHandler(mapperService); } protected InternalEngine createEngine(Store store, Path translogPath) throws IOException { @@ -856,7 +852,7 @@ public EngineConfig config( this::relativeTimeInNanos, indexCommitListener, true, - null + mapperService ); } @@ -1030,6 +1026,22 @@ public static List generateSingleDocHistory( return ops; } + private CheckedBiFunction nestedParsedDocFactory(MapperService mapperService) { + final DocumentMapper nestedMapper = mapperService.documentMapper(); + return (docId, nestedFieldValues) -> { + final XContentBuilder source = XContentFactory.jsonBuilder().startObject().field("value", "test"); + if (nestedFieldValues > 0) { + XContentBuilder nestedField = source.startObject("nested_field"); + for (int i = 0; i < nestedFieldValues; i++) { + nestedField.field("field-" + i, "value-" + i); + } + source.endObject(); + } + source.endObject(); + return nestedMapper.parse(new SourceToParse(docId, BytesReference.bytes(source), XContentType.JSON)); + }; + } + public List generateHistoryOnReplica( int numOps, boolean allowGapInSeqNo, @@ -1049,7 +1061,9 @@ public List generateHistoryOnReplica( long seqNo = startingSeqNo; final int maxIdValue = randomInt(numOps * 2); final List operations = new ArrayList<>(numOps); - CheckedBiFunction nestedParsedDocFactory = nestedParsedDocFactory(); + CheckedBiFunction nestedParsedDocFactory = nestedParsedDocFactory( + engine.engineConfig.getMapperService() + ); for (int i = 0; i < numOps; i++) { final String id = Integer.toString(randomInt(maxIdValue)); final Engine.Operation.TYPE opType = randomFrom(Engine.Operation.TYPE.values()); @@ -1058,7 +1072,9 @@ public List generateHistoryOnReplica( final long startTime = threadPool.relativeTimeInNanos(); final int copies = allowDuplicate && rarely() ? between(2, 4) : 1; for (int copy = 0; copy < copies; copy++) { - final ParsedDocument doc = isNestedDoc ? nestedParsedDocFactory.apply(id, nestedValues) : createParsedDoc(id, null); + final ParsedDocument doc = isNestedDoc + ? nestedParsedDocFactory.apply(id, nestedValues) + : parseDocument(engine.engineConfig.getMapperService(), id, null); switch (opType) { case INDEX -> operations.add( new Engine.Index( @@ -1270,7 +1286,17 @@ public static List getDocIds(Engine engine, boolean refresh */ public static List readAllOperationsInLucene(Engine engine) throws IOException { final List operations = new ArrayList<>(); - try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", 0, Long.MAX_VALUE, false, randomBoolean(), randomBoolean())) { + try ( + Translog.Snapshot snapshot = engine.newChangesSnapshot( + "test", + 0, + Long.MAX_VALUE, + false, + randomBoolean(), + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) + ) { Translog.Operation op; while ((op = snapshot.next()) != null) { operations.add(op); @@ -1341,7 +1367,15 @@ public static void assertConsistentHistoryBetweenTranslogAndLuceneIndex(Engine e assertThat(luceneOp.toString(), luceneOp.primaryTerm(), equalTo(translogOp.primaryTerm())); assertThat(luceneOp.opType(), equalTo(translogOp.opType())); if (luceneOp.opType() == Translog.Operation.Type.INDEX) { - assertThat(((Translog.Index) luceneOp).source(), equalTo(((Translog.Index) translogOp).source())); + if (engine.engineConfig.getIndexSettings().isRecoverySourceSyntheticEnabled()) { + assertToXContentEquivalent( + ((Translog.Index) luceneOp).source(), + ((Translog.Index) translogOp).source(), + XContentFactory.xContentType(((Translog.Index) luceneOp).source().array()) + ); + } else { + assertThat(((Translog.Index) luceneOp).source(), equalTo(((Translog.Index) translogOp).source())); + } } } } @@ -1397,15 +1431,19 @@ public static void assertAtMostOneLuceneDocumentPerSequenceNumber(IndexSettings } public static MapperService createMapperService() throws IOException { - IndexMetadata indexMetadata = IndexMetadata.builder("test") - .settings(indexSettings(1, 1).put(IndexMetadata.SETTING_VERSION_CREATED, IndexVersion.current())) - .putMapping("{\"properties\": {}}") + return createMapperService(Settings.EMPTY, "{}"); + } + + public static MapperService createMapperService(Settings settings, String mappings) throws IOException { + IndexMetadata indexMetadata = IndexMetadata.builder("index") + .settings(indexSettings(1, 1).put(IndexMetadata.SETTING_VERSION_CREATED, IndexVersion.current()).put(settings)) + .putMapping(mappings) .build(); MapperService mapperService = MapperTestUtils.newMapperService( new NamedXContentRegistry(ClusterModule.getNamedXWriteables()), createTempDir(), - Settings.EMPTY, - "test" + indexMetadata.getSettings(), + "index" ); mapperService.merge(indexMetadata, MapperService.MergeReason.MAPPING_UPDATE); return mapperService; diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java b/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java index 57cca12f99c41..33c745de25438 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java @@ -43,6 +43,10 @@ long appliedOperations() { return appliedOperations.get(); } + public TranslogHandler(MapperService mapperService) { + this.mapperService = mapperService; + } + public TranslogHandler(NamedXContentRegistry xContentRegistry, IndexSettings indexSettings) { SimilarityService similarityService = new SimilarityService(indexSettings, null, emptyMap()); MapperRegistry mapperRegistry = new IndicesModule(emptyList()).getMapperRegistry(); diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java index 011d9dd18331d..72950a99822a7 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java @@ -9,9 +9,12 @@ package org.elasticsearch.index.mapper; +import org.apache.lucene.analysis.standard.StandardAnalyzer; +import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.IndexableFieldType; import org.apache.lucene.index.LeafReader; @@ -20,7 +23,11 @@ import org.apache.lucene.search.FieldExistsQuery; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.UsageTrackingQueryCachingPolicy; +import org.apache.lucene.search.similarities.BM25Similarity; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.analysis.MockAnalyzer; import org.apache.lucene.tests.index.RandomIndexWriter; @@ -30,11 +37,14 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.IndexVersions; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.LuceneSyntheticSourceChangesSnapshot; import org.elasticsearch.index.fielddata.FieldDataContext; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.IndexFieldDataCache; @@ -43,6 +53,7 @@ import org.elasticsearch.index.fieldvisitor.StoredFieldLoader; import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.index.termvectors.TermVectorsService; +import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptContext; @@ -1146,6 +1157,11 @@ public final void testSyntheticSource() throws IOException { assertSyntheticSource(syntheticSourceSupport(shouldUseIgnoreMalformed()).example(5)); } + public final void testSyntheticSourceWithTranslogSnapshot() throws IOException { + assertSyntheticSourceWithTranslogSnapshot(syntheticSourceSupport(shouldUseIgnoreMalformed()), true); + assertSyntheticSourceWithTranslogSnapshot(syntheticSourceSupport(shouldUseIgnoreMalformed()), false); + } + public void testSyntheticSourceIgnoreMalformedExamples() throws IOException { assumeTrue("type doesn't support ignore_malformed", supportsIgnoreMalformed()); // We need to call this in order to hit the assumption inside so that @@ -1171,6 +1187,71 @@ private void assertSyntheticSource(SyntheticSourceExample example) throws IOExce assertThat(syntheticSource(mapper, example::buildInput), equalTo(example.expected())); } + private void assertSyntheticSourceWithTranslogSnapshot(SyntheticSourceSupport support, boolean doIndexSort) throws IOException { + var firstExample = support.example(1); + int maxDocs = randomIntBetween(20, 50); + var settings = Settings.builder() + .put(SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.getKey(), SourceFieldMapper.Mode.SYNTHETIC) + .put(IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE_SETTING.getKey(), true) + .build(); + var mapperService = createMapperService(getVersion(), settings, () -> true, mapping(b -> { + b.startObject("field"); + firstExample.mapping().accept(b); + b.endObject(); + })); + var docMapper = mapperService.documentMapper(); + try (var directory = newDirectory()) { + List examples = new ArrayList<>(); + IndexWriterConfig config = newIndexWriterConfig(random(), new StandardAnalyzer()); + config.setIndexSort(new Sort(new SortField("sort", SortField.Type.LONG))); + try (var iw = new RandomIndexWriter(random(), directory, config)) { + for (int seqNo = 0; seqNo < maxDocs; seqNo++) { + var example = support.example(randomIntBetween(1, 5)); + examples.add(example); + var doc = docMapper.parse(source(example::buildInput)); + assertNull(doc.dynamicMappingsUpdate()); + doc.updateSeqID(seqNo, 1); + doc.version().setLongValue(0); + if (doIndexSort) { + doc.rootDoc().add(new NumericDocValuesField("sort", randomLong())); + } + iw.addDocuments(doc.docs()); + if (frequently()) { + iw.flush(); + } + } + } + try (var indexReader = wrapInMockESDirectoryReader(DirectoryReader.open(directory))) { + int start = randomBoolean() ? 0 : randomIntBetween(1, maxDocs - 10); + var snapshot = new LuceneSyntheticSourceChangesSnapshot( + mapperService.mappingLookup(), + new Engine.Searcher( + "recovery", + indexReader, + new BM25Similarity(), + null, + new UsageTrackingQueryCachingPolicy(), + () -> {} + ), + randomIntBetween(1, maxDocs), + randomLongBetween(0, ByteSizeValue.ofBytes(Integer.MAX_VALUE).getBytes()), + start, + maxDocs, + true, + randomBoolean(), + IndexVersion.current() + ); + for (int i = start; i < maxDocs; i++) { + var example = examples.get(i); + var op = snapshot.next(); + if (op instanceof Translog.Index opIndex) { + assertThat(opIndex.source().utf8ToString(), equalTo(example.expected())); + } + } + } + } + } + protected boolean supportsEmptyInputArray() { return true; } diff --git a/test/framework/src/main/java/org/elasticsearch/indices/recovery/AbstractIndexRecoveryIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/indices/recovery/AbstractIndexRecoveryIntegTestCase.java index ec85feb200984..59c44925f920f 100644 --- a/test/framework/src/main/java/org/elasticsearch/indices/recovery/AbstractIndexRecoveryIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/indices/recovery/AbstractIndexRecoveryIntegTestCase.java @@ -26,7 +26,6 @@ import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.MockEngineFactoryPlugin; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.node.RecoverySettingsChunkSizePlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.snapshots.SnapshotState; @@ -75,7 +74,6 @@ protected Collection> nodePlugins() { return Arrays.asList( MockTransportService.TestPlugin.class, MockFSIndexStore.TestPlugin.class, - RecoverySettingsChunkSizePlugin.class, InternalSettingsPlugin.class, MockEngineFactoryPlugin.class ); diff --git a/test/framework/src/main/java/org/elasticsearch/node/MockNode.java b/test/framework/src/main/java/org/elasticsearch/node/MockNode.java index 38c7b1eb04772..25a6a4486378e 100644 --- a/test/framework/src/main/java/org/elasticsearch/node/MockNode.java +++ b/test/framework/src/main/java/org/elasticsearch/node/MockNode.java @@ -28,7 +28,6 @@ import org.elasticsearch.indices.ExecutorSelector; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.CircuitBreakerService; -import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.plugins.MockPluginsService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.PluginsLoader; @@ -201,16 +200,6 @@ protected TransportService newTransportService( } } - @Override - void processRecoverySettings(PluginsService pluginsService, ClusterSettings clusterSettings, RecoverySettings recoverySettings) { - if (pluginsService.filterPlugins(RecoverySettingsChunkSizePlugin.class).findAny().isEmpty() == false) { - clusterSettings.addSettingsUpdateConsumer( - RecoverySettingsChunkSizePlugin.CHUNK_SIZE_SETTING, - recoverySettings::setChunkSize - ); - } - } - @Override protected ClusterInfoService newClusterInfoService( PluginsService pluginsService, diff --git a/test/framework/src/main/java/org/elasticsearch/node/RecoverySettingsChunkSizePlugin.java b/test/framework/src/main/java/org/elasticsearch/node/RecoverySettingsChunkSizePlugin.java deleted file mode 100644 index 489c9f704f419..0000000000000 --- a/test/framework/src/main/java/org/elasticsearch/node/RecoverySettingsChunkSizePlugin.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the "Elastic License - * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side - * Public License v 1"; you may not use this file except in compliance with, at - * your election, the "Elastic License 2.0", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ - -package org.elasticsearch.node; - -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.indices.recovery.RecoverySettings; -import org.elasticsearch.plugins.Plugin; - -import java.util.List; - -import static java.util.Collections.singletonList; - -/** - * Marker plugin that will trigger {@link MockNode} making {@link #CHUNK_SIZE_SETTING} dynamic. - */ -public class RecoverySettingsChunkSizePlugin extends Plugin { - /** - * The chunk size. Only exposed by tests. - */ - public static final Setting CHUNK_SIZE_SETTING = Setting.byteSizeSetting( - "indices.recovery.chunk_size", - RecoverySettings.DEFAULT_CHUNK_SIZE, - Property.Dynamic, - Property.NodeScope - ); - - @Override - public List> getSettings() { - return singletonList(CHUNK_SIZE_SETTING); - } -} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java index 167c5f66300a9..8133702ab5354 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java @@ -564,7 +564,17 @@ static Translog.Operation[] getOperations( long toSeqNo = Math.min(globalCheckpoint, (fromSeqNo + maxOperationCount) - 1); assert fromSeqNo <= toSeqNo : "invalid range from_seqno[" + fromSeqNo + "] > to_seqno[" + toSeqNo + "]"; final List operations = new ArrayList<>(); - try (Translog.Snapshot snapshot = indexShard.newChangesSnapshot("ccr", fromSeqNo, toSeqNo, true, true, false)) { + try ( + Translog.Snapshot snapshot = indexShard.newChangesSnapshot( + "ccr", + fromSeqNo, + toSeqNo, + true, + true, + false, + maxBatchSize.getBytes() + ) + ) { Translog.Operation op; while ((op = snapshot.next()) != null) { operations.add(op); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java index 5cd9f8bc5b78c..573c66cbb614a 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java @@ -755,7 +755,15 @@ private void assertConsistentHistoryBetweenLeaderAndFollower( final Map operationsOnLeader = new HashMap<>(); try ( Translog.Snapshot snapshot = leader.getPrimary() - .newChangesSnapshot("test", 0, Long.MAX_VALUE, false, randomBoolean(), randomBoolean()) + .newChangesSnapshot( + "test", + 0, + Long.MAX_VALUE, + false, + randomBoolean(), + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) ) { Translog.Operation op; while ((op = snapshot.next()) != null) { @@ -780,7 +788,8 @@ private void assertConsistentHistoryBetweenLeaderAndFollower( Long.MAX_VALUE, false, randomBoolean(), - randomBoolean() + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) ) ) { Translog.Operation op; diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsTests.java index 4e3aea2cad205..e3f26eed0c2e9 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/bulk/BulkShardOperationsTests.java @@ -17,6 +17,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.translog.Translog; @@ -84,7 +85,15 @@ public void testPrimaryTermFromFollower() throws IOException { boolean accessStats = randomBoolean(); try ( - Translog.Snapshot snapshot = followerPrimary.newChangesSnapshot("test", 0, Long.MAX_VALUE, false, randomBoolean(), accessStats) + Translog.Snapshot snapshot = followerPrimary.newChangesSnapshot( + "test", + 0, + Long.MAX_VALUE, + false, + randomBoolean(), + accessStats, + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) ) { if (accessStats) { assertThat(snapshot.totalOperations(), equalTo(operations.size())); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java index 150eddf039cec..62dc3313a1172 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java @@ -15,7 +15,11 @@ import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.CheckedBiFunction; import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.Index; @@ -31,7 +35,10 @@ import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.engine.TranslogHandler; +import org.elasticsearch.index.mapper.DocumentMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.seqno.RetentionLeases; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -44,6 +51,9 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xcontent.XContentType; import java.io.IOException; import java.nio.file.Path; @@ -94,7 +104,7 @@ public void tearDown() throws Exception { super.tearDown(); } - public void testFollowingEngineRejectsNonFollowingIndex() { + public void testFollowingEngineRejectsNonFollowingIndex() throws IOException { final Settings.Builder builder = indexSettings(IndexVersion.current(), 1, 0); if (randomBoolean()) { builder.put("index.xpack.ccr.following_index", false); @@ -212,7 +222,7 @@ private EngineConfig engineConfig( final IndexSettings indexSettings, final ThreadPool threadPool, final Store store - ) { + ) throws IOException { final IndexWriterConfig indexWriterConfig = newIndexWriterConfig(); final Path translogPath = createTempDir("translog"); final TranslogConfig translogConfig = new TranslogConfig( @@ -221,6 +231,7 @@ private EngineConfig engineConfig( indexSettings, BigArrays.NON_RECYCLING_INSTANCE ); + final MapperService mapperService = EngineTestCase.createMapperService(); return new EngineConfig( shardIdValue, threadPool, @@ -253,7 +264,7 @@ public void onFailedEngine(String reason, Exception e) { System::nanoTime, null, true, - null + mapperService ); } @@ -641,7 +652,15 @@ private void fetchOperations(AtomicBoolean stopped, AtomicLong lastFetchedSeqNo, final long toSeqNo = randomLongBetween(nextSeqNo, Math.min(nextSeqNo + 5, checkpoint)); try ( Translog.Snapshot snapshot = shuffleSnapshot( - leader.newChangesSnapshot("test", fromSeqNo, toSeqNo, true, randomBoolean(), randomBoolean()) + leader.newChangesSnapshot( + "test", + fromSeqNo, + toSeqNo, + true, + randomBoolean(), + randomBoolean(), + randomLongBetween(1, ByteSizeValue.ofMb(32).getBytes()) + ) ) ) { follower.advanceMaxSeqNoOfUpdatesOrDeletes(leader.getMaxSeqNoOfUpdatesOrDeletes()); @@ -689,6 +708,39 @@ public void close() throws IOException { }; } + private CheckedBiFunction nestedParsedDocFactory() throws Exception { + final MapperService mapperService = EngineTestCase.createMapperService(); + final String nestedMapping = Strings.toString( + XContentFactory.jsonBuilder() + .startObject() + .startObject("type") + .startObject("properties") + .startObject("nested_field") + .field("type", "nested") + .endObject() + .endObject() + .endObject() + .endObject() + ); + final DocumentMapper nestedMapper = mapperService.merge( + "type", + new CompressedXContent(nestedMapping), + MapperService.MergeReason.MAPPING_UPDATE + ); + return (docId, nestedFieldValues) -> { + final XContentBuilder source = XContentFactory.jsonBuilder().startObject().field("field", "value"); + if (nestedFieldValues > 0) { + XContentBuilder nestedField = source.startObject("nested_field"); + for (int i = 0; i < nestedFieldValues; i++) { + nestedField.field("field-" + i, "value-" + i); + } + source.endObject(); + } + source.endObject(); + return nestedMapper.parse(new SourceToParse(docId, BytesReference.bytes(source), XContentType.JSON)); + }; + } + public void testProcessOnceOnPrimary() throws Exception { final Settings.Builder settingsBuilder = indexSettings(IndexVersion.current(), 1, 0).put("index.xpack.ccr.following_index", true); switch (indexMode) { @@ -709,7 +761,7 @@ public void testProcessOnceOnPrimary() throws Exception { final Settings settings = settingsBuilder.build(); final IndexMetadata indexMetadata = IndexMetadata.builder(index.getName()).settings(settings).build(); final IndexSettings indexSettings = new IndexSettings(indexMetadata, settings); - final CheckedBiFunction nestedDocFunc = EngineTestCase.nestedParsedDocFactory(); + final CheckedBiFunction nestedDocFunc = nestedParsedDocFactory(); int numOps = between(10, 100); List operations = new ArrayList<>(numOps); for (int i = 0; i < numOps; i++) {