From 8a2d154bad8976af3c1282b7d5dd0b2ceb1c2454 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Thu, 30 Aug 2018 15:05:38 -0600 Subject: [PATCH 01/16] Update serialization versions for custom IndexMetaData backport --- .../action/admin/indices/create/CreateIndexRequest.java | 4 ++-- .../admin/indices/template/put/PutIndexTemplateRequest.java | 4 ++-- .../org/elasticsearch/cluster/metadata/IndexMetaData.java | 4 ++-- .../elasticsearch/cluster/metadata/IndexTemplateMetaData.java | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java index a186f9b5011..fa2a395f2c9 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java @@ -448,7 +448,7 @@ public class CreateIndexRequest extends AcknowledgedRequest } mappings.put(type, source); } - if (in.getVersion().before(Version.V_7_0_0_alpha1)) { + if (in.getVersion().before(Version.V_6_5_0)) { // This used to be the size of custom metadata classes int customSize = in.readVInt(); assert customSize == 0 : "unexpected custom metadata when none is supported"; @@ -477,7 +477,7 @@ public class CreateIndexRequest extends AcknowledgedRequest out.writeString(entry.getKey()); out.writeString(entry.getValue()); } - if (out.getVersion().before(Version.V_7_0_0_alpha1)) { + if (out.getVersion().before(Version.V_6_5_0)) { // Size of custom index metadata, which is removed out.writeVInt(0); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java index 7b45709f2a0..d254f989d4a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java @@ -474,7 +474,7 @@ public class PutIndexTemplateRequest extends MasterNodeRequest, ToXContentFragmen builder.putAlias(aliasMd); } int customSize = in.readVInt(); - if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + if (in.getVersion().onOrAfter(Version.V_6_5_0)) { for (int i = 0; i < customSize; i++) { String key = in.readString(); DiffableStringMap custom = new DiffableStringMap(in); @@ -752,7 +752,7 @@ public class IndexMetaData implements Diffable, ToXContentFragmen for (ObjectCursor cursor : aliases.values()) { cursor.value.writeTo(out); } - if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + if (out.getVersion().onOrAfter(Version.V_6_5_0)) { out.writeVInt(customData.size()); for (final ObjectObjectCursor cursor : customData) { out.writeString(cursor.key); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java index c3f0f86e3e9..7e2d9256303 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java @@ -210,7 +210,7 @@ public class IndexTemplateMetaData extends AbstractDiffable cursor : aliases.values()) { cursor.value.writeTo(out); } - if (out.getVersion().before(Version.V_7_0_0_alpha1)) { + if (out.getVersion().before(Version.V_6_5_0)) { out.writeVInt(0); } out.writeOptionalVInt(version); From 39839f97ef8fa4562e2c2e0566867943b0e0de96 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 30 Aug 2018 19:37:20 -0400 Subject: [PATCH 02/16] TEST: Access cluster state directly in assertSeqNos (#33277) Some AbstractDisruptionTestCase tests start failing since we enabled assertSeqNos (in #33130). They fail because the assertSeqNos assertion queries cluster stats while the cluster is disrupted or not formed yet. This commit switches to use the cluster state and shard stats directly from the test cluster. Closes #33251 --- .../elasticsearch/test/ESIntegTestCase.java | 74 ++++++++++--------- 1 file changed, 40 insertions(+), 34 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 1f51ad495e1..322e2a128c9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -20,12 +20,15 @@ package org.elasticsearch.test; import com.carrotsearch.hppc.ObjectLongMap; +import com.carrotsearch.hppc.cursors.IntObjectCursor; +import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.carrotsearch.randomizedtesting.RandomizedContext; import com.carrotsearch.randomizedtesting.annotations.TestGroup; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import org.apache.http.HttpHost; import org.apache.lucene.search.Sort; +import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; @@ -48,10 +51,6 @@ import org.elasticsearch.action.admin.indices.segments.IndexSegments; import org.elasticsearch.action.admin.indices.segments.IndexShardSegments; import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse; import org.elasticsearch.action.admin.indices.segments.ShardSegments; -import org.elasticsearch.action.admin.indices.stats.IndexShardStats; -import org.elasticsearch.action.admin.indices.stats.IndexStats; -import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; -import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; @@ -187,7 +186,6 @@ import java.util.IdentityHashMap; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; @@ -2328,40 +2326,48 @@ public abstract class ESIntegTestCase extends ESTestCase { protected void assertSeqNos() throws Exception { assertBusy(() -> { - IndicesStatsResponse stats = client().admin().indices().prepareStats().clear().get(); - for (IndexStats indexStats : stats.getIndices().values()) { - for (IndexShardStats indexShardStats : indexStats.getIndexShards().values()) { - Optional maybePrimary = Stream.of(indexShardStats.getShards()) - .filter(s -> s.getShardRouting().active() && s.getShardRouting().primary()) - .findFirst(); - if (maybePrimary.isPresent() == false) { + final ClusterState state = clusterService().state(); + for (ObjectObjectCursor indexRoutingTable : state.routingTable().indicesRouting()) { + for (IntObjectCursor indexShardRoutingTable : indexRoutingTable.value.shards()) { + ShardRouting primaryShardRouting = indexShardRoutingTable.value.primaryShard(); + if (primaryShardRouting == null || primaryShardRouting.assignedToNode() == false) { continue; } - ShardStats primary = maybePrimary.get(); - final SeqNoStats primarySeqNoStats = primary.getSeqNoStats(); - final ShardRouting primaryShardRouting = primary.getShardRouting(); + DiscoveryNode primaryNode = state.nodes().get(primaryShardRouting.currentNodeId()); + IndexShard primaryShard = internalCluster().getInstance(IndicesService.class, primaryNode.getName()) + .indexServiceSafe(primaryShardRouting.index()).getShard(primaryShardRouting.id()); + final SeqNoStats primarySeqNoStats; + final ObjectLongMap syncGlobalCheckpoints; + try { + primarySeqNoStats = primaryShard.seqNoStats(); + syncGlobalCheckpoints = primaryShard.getInSyncGlobalCheckpoints(); + } catch (AlreadyClosedException ex) { + continue; // shard is closed - just ignore + } assertThat(primaryShardRouting + " should have set the global checkpoint", - primarySeqNoStats.getGlobalCheckpoint(), not(equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO))); - final DiscoveryNode node = clusterService().state().nodes().get(primaryShardRouting.currentNodeId()); - final IndicesService indicesService = - internalCluster().getInstance(IndicesService.class, node.getName()); - final IndexShard indexShard = indicesService.getShardOrNull(primaryShardRouting.shardId()); - final ObjectLongMap globalCheckpoints = indexShard.getInSyncGlobalCheckpoints(); - for (ShardStats shardStats : indexShardStats) { - final SeqNoStats seqNoStats = shardStats.getSeqNoStats(); - if (seqNoStats == null) { - continue; // this shard was closed + primarySeqNoStats.getGlobalCheckpoint(), not(equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO))); + for (ShardRouting replicaShardRouting : indexShardRoutingTable.value.replicaShards()) { + if (replicaShardRouting.assignedToNode() == false) { + continue; } - assertThat(shardStats.getShardRouting() + " local checkpoint mismatch", - seqNoStats.getLocalCheckpoint(), equalTo(primarySeqNoStats.getLocalCheckpoint())); - assertThat(shardStats.getShardRouting() + " global checkpoint mismatch", - seqNoStats.getGlobalCheckpoint(), equalTo(primarySeqNoStats.getGlobalCheckpoint())); - assertThat(shardStats.getShardRouting() + " max seq no mismatch", - seqNoStats.getMaxSeqNo(), equalTo(primarySeqNoStats.getMaxSeqNo())); + DiscoveryNode replicaNode = state.nodes().get(replicaShardRouting.currentNodeId()); + IndexShard replicaShard = internalCluster().getInstance(IndicesService.class, replicaNode.getName()) + .indexServiceSafe(replicaShardRouting.index()).getShard(replicaShardRouting.id()); + final SeqNoStats seqNoStats; + try { + seqNoStats = replicaShard.seqNoStats(); + } catch (AlreadyClosedException e) { + continue; // shard is closed - just ignore + } + assertThat(replicaShardRouting + " local checkpoint mismatch", + seqNoStats.getLocalCheckpoint(), equalTo(primarySeqNoStats.getLocalCheckpoint())); + assertThat(replicaShardRouting + " global checkpoint mismatch", + seqNoStats.getGlobalCheckpoint(), equalTo(primarySeqNoStats.getGlobalCheckpoint())); + assertThat(replicaShardRouting + " max seq no mismatch", + seqNoStats.getMaxSeqNo(), equalTo(primarySeqNoStats.getMaxSeqNo())); // the local knowledge on the primary of the global checkpoint equals the global checkpoint on the shard - assertThat( - seqNoStats.getGlobalCheckpoint(), - equalTo(globalCheckpoints.get(shardStats.getShardRouting().allocationId().getId()))); + assertThat(replicaShardRouting + " global checkpoint syncs mismatch", seqNoStats.getGlobalCheckpoint(), + equalTo(syncGlobalCheckpoints.get(replicaShardRouting.allocationId().getId()))); } } } From 86feb7713b1903c2f1aca1a1de44c5477bbe40ae Mon Sep 17 00:00:00 2001 From: Tal Levy Date: Thu, 30 Aug 2018 18:13:50 -0700 Subject: [PATCH 03/16] [MUTE] SmokeTestWatcherWithSecurityIT flaky tests --- .../elasticsearch/smoketest/SmokeTestWatcherWithSecurityIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityIT.java b/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityIT.java index 665b92bbc0e..538d54416bf 100644 --- a/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityIT.java +++ b/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityIT.java @@ -158,6 +158,7 @@ public class SmokeTestWatcherWithSecurityIT extends ESRestTestCase { assertThat(conditionMet, is(true)); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/29893") public void testSearchInputWithInsufficientPrivileges() throws Exception { String indexName = "index_not_allowed_to_read"; try (XContentBuilder builder = jsonBuilder()) { @@ -213,6 +214,7 @@ public class SmokeTestWatcherWithSecurityIT extends ESRestTestCase { assertThat(value, is("15")); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/33291") public void testSearchTransformInsufficientPermissions() throws Exception { try (XContentBuilder builder = jsonBuilder()) { builder.startObject(); From 6dd0aa54f6d0ad44219b3b416438b1be57cb37e5 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 30 Aug 2018 22:11:23 -0400 Subject: [PATCH 04/16] Integrates soft-deletes into Elasticsearch (#33222) This PR integrates Lucene soft-deletes(LUCENE-8200) into Elasticsearch. Highlight works in this PR include: - Replace hard-deletes by soft-deletes in InternalEngine - Use _recovery_source if _source is disabled or modified (#31106) - Soft-deletes retention policy based on the global checkpoint (#30335) - Read operation history from Lucene instead of translog (#30120) - Use Lucene history in peer-recovery (#30522) Relates #30086 Closes #29530 --- These works have been done by the whole team; however, these individuals (lexical order) have significant contribution in coding and reviewing: Co-authored-by: Adrien Grand jpountz@gmail.com Co-authored-by: Boaz Leskes b.leskes@gmail.com Co-authored-by: Jason Tedor jason@tedor.me Co-authored-by: Martijn van Groningen martijn.v.groningen@gmail.com Co-authored-by: Nhat Nguyen nhat.nguyen@elastic.co Co-authored-by: Simon Willnauer simonw@apache.org --- .../percolator/CandidateQueryTests.java | 8 +- .../PercolatorFieldMapperTests.java | 30 +- .../elasticsearch/common/lucene/Lucene.java | 86 ++- .../uid/PerThreadIDVersionAndSeqNoLookup.java | 21 +- .../common/settings/IndexScopedSettings.java | 2 + .../elasticsearch/index/IndexSettings.java | 38 ++ .../index/engine/CombinedDeletionPolicy.java | 12 +- .../elasticsearch/index/engine/Engine.java | 28 +- .../index/engine/EngineConfig.java | 27 +- .../index/engine/InternalEngine.java | 390 +++++++++-- .../index/engine/LuceneChangesSnapshot.java | 368 +++++++++++ .../RecoverySourcePruneMergePolicy.java | 292 +++++++++ .../index/engine/SoftDeletesPolicy.java | 120 ++++ .../index/fieldvisitor/FieldsVisitor.java | 10 +- .../index/mapper/DocumentMapper.java | 34 +- .../index/mapper/DocumentParser.java | 33 +- .../index/mapper/FieldNamesFieldMapper.java | 5 +- .../index/mapper/ParseContext.java | 20 +- .../index/mapper/ParsedDocument.java | 11 + .../index/mapper/SeqNoFieldMapper.java | 7 +- .../index/mapper/SourceFieldMapper.java | 16 +- .../elasticsearch/index/shard/IndexShard.java | 47 +- .../index/shard/PrimaryReplicaSyncer.java | 2 +- .../index/shard/StoreRecovery.java | 1 + .../org/elasticsearch/index/store/Store.java | 2 +- .../index/translog/Translog.java | 3 + .../index/translog/TranslogWriter.java | 20 +- .../translog/TruncateTranslogCommand.java | 2 + .../recovery/RecoverySourceHandler.java | 59 +- .../blobstore/BlobStoreRepository.java | 1 + .../snapshots/RestoreService.java | 4 +- .../cluster/routing/PrimaryAllocationIT.java | 1 + .../common/lucene/LuceneTests.java | 91 +++ .../discovery/AbstractDisruptionTestCase.java | 1 + .../gateway/RecoveryFromGatewayIT.java | 13 +- .../index/IndexServiceTests.java | 3 +- .../index/IndexSettingsTests.java | 8 + .../engine/CombinedDeletionPolicyTests.java | 69 +- .../index/engine/InternalEngineTests.java | 620 ++++++++++++------ .../engine/LuceneChangesSnapshotTests.java | 289 ++++++++ .../RecoverySourcePruneMergePolicyTests.java | 161 +++++ .../index/engine/SoftDeletesPolicyTests.java | 75 +++ .../index/mapper/DocumentParserTests.java | 10 +- .../index/mapper/DynamicMappingTests.java | 6 +- .../IndexLevelReplicationTests.java | 29 +- .../RecoveryDuringReplicationTests.java | 11 +- .../index/shard/IndexShardTests.java | 58 +- .../shard/PrimaryReplicaSyncerTests.java | 21 +- .../index/shard/RefreshListenersTests.java | 4 +- .../indices/recovery/IndexRecoveryIT.java | 6 + .../PeerRecoveryTargetServiceTests.java | 2 + .../recovery/RecoverySourceHandlerTests.java | 6 - .../indices/recovery/RecoveryTests.java | 80 ++- .../indices/stats/IndexStatsIT.java | 37 +- .../AbstractSnapshotIntegTestCase.java | 6 + .../SharedClusterSnapshotRestoreIT.java | 13 +- .../versioning/SimpleVersioningIT.java | 23 + .../index/engine/EngineTestCase.java | 400 ++++++++++- .../ESIndexLevelReplicationTestCase.java | 27 +- .../index/shard/IndexShardTestCase.java | 131 ++-- .../elasticsearch/test/ESIntegTestCase.java | 4 + .../test/ESSingleNodeTestCase.java | 9 + .../test/InternalTestCluster.java | 20 + 63 files changed, 3433 insertions(+), 500 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java create mode 100644 server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java create mode 100644 server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java index e6d637aabb1..9c8979601e8 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java @@ -77,6 +77,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; @@ -87,6 +88,7 @@ import org.elasticsearch.common.geo.ShapeRelation; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; @@ -1109,7 +1111,11 @@ public class CandidateQueryTests extends ESSingleNodeTestCase { } private void addQuery(Query query, List docs) { - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(query, parseContext); ParseContext.Document queryDocument = parseContext.doc(); diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java index ecff48b344c..80524a2f862 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java @@ -42,6 +42,7 @@ import org.apache.lucene.search.join.ScoreMode; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -58,6 +59,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperParser; import org.elasticsearch.index.mapper.MapperParsingException; @@ -182,7 +184,11 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); ParseContext.Document document = parseContext.doc(); @@ -204,7 +210,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { bq.add(termQuery1, Occur.MUST); bq.add(termQuery2, Occur.MUST); - parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), + parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); document = parseContext.doc(); @@ -232,8 +238,12 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { bq.add(rangeQuery2, Occur.MUST); DocumentMapper documentMapper = mapperService.documentMapper("doc"); + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); ParseContext.Document document = parseContext.doc(); @@ -259,7 +269,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { .rangeQuery(15, 20, true, true, null, null, null, null); bq.add(rangeQuery2, Occur.MUST); - parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); document = parseContext.doc(); @@ -283,7 +293,11 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { TermRangeQuery query = new TermRangeQuery("field1", new BytesRef("a"), new BytesRef("z"), true, true); DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(query, parseContext); ParseContext.Document document = parseContext.doc(); @@ -298,7 +312,11 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { PhraseQuery phraseQuery = new PhraseQuery("field", "term"); DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(phraseQuery, parseContext); ParseContext.Document document = parseContext.doc(); diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index a24a6aea07f..1c1e5687893 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -27,8 +27,10 @@ import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.document.LatLonDocValuesField; +import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FilterDirectoryReader; import org.apache.lucene.index.FilterLeafReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexFileNames; @@ -96,6 +98,8 @@ public class Lucene { assert annotation == null : "DocValuesFormat " + LATEST_DOC_VALUES_FORMAT + " is deprecated" ; } + public static final String SOFT_DELETES_FIELD = "__soft_deletes"; + public static final NamedAnalyzer STANDARD_ANALYZER = new NamedAnalyzer("_standard", AnalyzerScope.GLOBAL, new StandardAnalyzer()); public static final NamedAnalyzer KEYWORD_ANALYZER = new NamedAnalyzer("_keyword", AnalyzerScope.GLOBAL, new KeywordAnalyzer()); @@ -140,7 +144,7 @@ public class Lucene { public static int getNumDocs(SegmentInfos info) { int numDocs = 0; for (SegmentCommitInfo si : info) { - numDocs += si.info.maxDoc() - si.getDelCount(); + numDocs += si.info.maxDoc() - si.getDelCount() - si.getSoftDelCount(); } return numDocs; } @@ -197,6 +201,7 @@ public class Lucene { } final CommitPoint cp = new CommitPoint(si, directory); try (IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(Lucene.STANDARD_ANALYZER) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setIndexCommit(cp) .setCommitOnClose(false) .setMergePolicy(NoMergePolicy.INSTANCE) @@ -220,6 +225,7 @@ public class Lucene { } } try (IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(Lucene.STANDARD_ANALYZER) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setMergePolicy(NoMergePolicy.INSTANCE) // no merges .setCommitOnClose(false) // no commits .setOpenMode(IndexWriterConfig.OpenMode.CREATE))) // force creation - don't append... @@ -829,4 +835,82 @@ public class Lucene { } }; } + + /** + * Wraps a directory reader to make all documents live except those were rolled back + * or hard-deleted due to non-aborting exceptions during indexing. + * The wrapped reader can be used to query all documents. + * + * @param in the input directory reader + * @return the wrapped reader + */ + public static DirectoryReader wrapAllDocsLive(DirectoryReader in) throws IOException { + return new DirectoryReaderWithAllLiveDocs(in); + } + + private static final class DirectoryReaderWithAllLiveDocs extends FilterDirectoryReader { + static final class LeafReaderWithLiveDocs extends FilterLeafReader { + final Bits liveDocs; + final int numDocs; + LeafReaderWithLiveDocs(LeafReader in, Bits liveDocs, int numDocs) { + super(in); + this.liveDocs = liveDocs; + this.numDocs = numDocs; + } + @Override + public Bits getLiveDocs() { + return liveDocs; + } + @Override + public int numDocs() { + return numDocs; + } + @Override + public CacheHelper getCoreCacheHelper() { + return in.getCoreCacheHelper(); + } + @Override + public CacheHelper getReaderCacheHelper() { + return null; // Modifying liveDocs + } + } + + DirectoryReaderWithAllLiveDocs(DirectoryReader in) throws IOException { + super(in, new SubReaderWrapper() { + @Override + public LeafReader wrap(LeafReader leaf) { + SegmentReader segmentReader = segmentReader(leaf); + Bits hardLiveDocs = segmentReader.getHardLiveDocs(); + if (hardLiveDocs == null) { + return new LeafReaderWithLiveDocs(leaf, null, leaf.maxDoc()); + } + // TODO: Can we avoid calculate numDocs by using SegmentReader#getSegmentInfo with LUCENE-8458? + int numDocs = 0; + for (int i = 0; i < hardLiveDocs.length(); i++) { + if (hardLiveDocs.get(i)) { + numDocs++; + } + } + return new LeafReaderWithLiveDocs(segmentReader, hardLiveDocs, numDocs); + } + }); + } + + @Override + protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { + return wrapAllDocsLive(in); + } + + @Override + public CacheHelper getReaderCacheHelper() { + return null; // Modifying liveDocs + } + } + + /** + * Returns a numeric docvalues which can be used to soft-delete documents. + */ + public static NumericDocValuesField newSoftDeletesField() { + return new NumericDocValuesField(SOFT_DELETES_FIELD, 1); + } } diff --git a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java index 38fcdfe5f1b..3a037bed62b 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java @@ -28,6 +28,7 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -66,15 +67,22 @@ final class PerThreadIDVersionAndSeqNoLookup { */ PerThreadIDVersionAndSeqNoLookup(LeafReader reader, String uidField) throws IOException { this.uidField = uidField; - Terms terms = reader.terms(uidField); + final Terms terms = reader.terms(uidField); if (terms == null) { - throw new IllegalArgumentException("reader misses the [" + uidField + "] field"); + // If a segment contains only no-ops, it does not have _uid but has both _soft_deletes and _tombstone fields. + final NumericDocValues softDeletesDV = reader.getNumericDocValues(Lucene.SOFT_DELETES_FIELD); + final NumericDocValues tombstoneDV = reader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); + if (softDeletesDV == null || tombstoneDV == null) { + throw new IllegalArgumentException("reader does not have _uid terms but not a no-op segment; " + + "_soft_deletes [" + softDeletesDV + "], _tombstone [" + tombstoneDV + "]"); + } + termsEnum = null; + } else { + termsEnum = terms.iterator(); } - termsEnum = terms.iterator(); if (reader.getNumericDocValues(VersionFieldMapper.NAME) == null) { - throw new IllegalArgumentException("reader misses the [" + VersionFieldMapper.NAME + "] field"); + throw new IllegalArgumentException("reader misses the [" + VersionFieldMapper.NAME + "] field; _uid terms [" + terms + "]"); } - Object readerKey = null; assert (readerKey = reader.getCoreCacheHelper().getKey()) != null; this.readerKey = readerKey; @@ -111,7 +119,8 @@ final class PerThreadIDVersionAndSeqNoLookup { * {@link DocIdSetIterator#NO_MORE_DOCS} is returned if not found * */ private int getDocID(BytesRef id, Bits liveDocs) throws IOException { - if (termsEnum.seekExact(id)) { + // termsEnum can possibly be null here if this leaf contains only no-ops. + if (termsEnum != null && termsEnum.seekExact(id)) { int docID = DocIdSetIterator.NO_MORE_DOCS; // there may be more than one matching docID, in the case of nested docs, so we want the last one: docsEnum = termsEnum.postings(docsEnum, 0); 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 46e3867f7ae..f3de294046c 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -129,6 +129,8 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.MAX_REGEX_LENGTH_SETTING, ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING, IndexSettings.INDEX_GC_DELETES_SETTING, + IndexSettings.INDEX_SOFT_DELETES_SETTING, + IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING, UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING, diff --git a/server/src/main/java/org/elasticsearch/index/IndexSettings.java b/server/src/main/java/org/elasticsearch/index/IndexSettings.java index 44cd743bbd4..3ea022bbebd 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/server/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -237,6 +237,21 @@ public final class IndexSettings { public static final Setting INDEX_GC_DELETES_SETTING = Setting.timeSetting("index.gc_deletes", DEFAULT_GC_DELETES, new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic, Property.IndexScope); + + /** + * Specifies if the index should use soft-delete instead of hard-delete for update/delete operations. + */ + public static final Setting INDEX_SOFT_DELETES_SETTING = + Setting.boolSetting("index.soft_deletes.enabled", false, Property.IndexScope, Property.Final); + + /** + * Controls how many soft-deleted documents will be kept around before being merged away. Keeping more deleted + * documents increases the chance of operation-based recoveries and allows querying a longer history of documents. + * If soft-deletes is enabled, an engine by default will retain all operations up to the global checkpoint. + **/ + public static final Setting INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING = + Setting.longSetting("index.soft_deletes.retention.operations", 0, 0, Property.IndexScope, Property.Dynamic); + /** * The maximum number of refresh listeners allows on this shard. */ @@ -289,6 +304,8 @@ public final class IndexSettings { private final IndexSortConfig indexSortConfig; private final IndexScopedSettings scopedSettings; private long gcDeletesInMillis = DEFAULT_GC_DELETES.millis(); + private final boolean softDeleteEnabled; + private volatile long softDeleteRetentionOperations; private volatile boolean warmerEnabled; private volatile int maxResultWindow; private volatile int maxInnerResultWindow; @@ -400,6 +417,8 @@ public final class IndexSettings { generationThresholdSize = scopedSettings.get(INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING); mergeSchedulerConfig = new MergeSchedulerConfig(this); gcDeletesInMillis = scopedSettings.get(INDEX_GC_DELETES_SETTING).getMillis(); + softDeleteEnabled = version.onOrAfter(Version.V_7_0_0_alpha1) && scopedSettings.get(INDEX_SOFT_DELETES_SETTING); + softDeleteRetentionOperations = scopedSettings.get(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING); warmerEnabled = scopedSettings.get(INDEX_WARMER_ENABLED_SETTING); maxResultWindow = scopedSettings.get(MAX_RESULT_WINDOW_SETTING); maxInnerResultWindow = scopedSettings.get(MAX_INNER_RESULT_WINDOW_SETTING); @@ -458,6 +477,7 @@ public final class IndexSettings { scopedSettings.addSettingsUpdateConsumer(INDEX_SEARCH_IDLE_AFTER, this::setSearchIdleAfter); scopedSettings.addSettingsUpdateConsumer(MAX_REGEX_LENGTH_SETTING, this::setMaxRegexLength); scopedSettings.addSettingsUpdateConsumer(DEFAULT_PIPELINE, this::setDefaultPipeline); + scopedSettings.addSettingsUpdateConsumer(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, this::setSoftDeleteRetentionOperations); } private void setSearchIdleAfter(TimeValue searchIdleAfter) { this.searchIdleAfter = searchIdleAfter; } @@ -841,4 +861,22 @@ public final class IndexSettings { public void setDefaultPipeline(String defaultPipeline) { this.defaultPipeline = defaultPipeline; } + + /** + * Returns true if soft-delete is enabled. + */ + public boolean isSoftDeleteEnabled() { + return softDeleteEnabled; + } + + private void setSoftDeleteRetentionOperations(long ops) { + this.softDeleteRetentionOperations = ops; + } + + /** + * Returns the number of extra operations (i.e. soft-deleted documents) to be kept for recoveries and history purpose. + */ + public long getSoftDeleteRetentionOperations() { + return this.softDeleteRetentionOperations; + } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index d0575c8a8c9..d10690379ed 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -46,14 +46,17 @@ import java.util.function.LongSupplier; public final class CombinedDeletionPolicy extends IndexDeletionPolicy { private final Logger logger; private final TranslogDeletionPolicy translogDeletionPolicy; + private final SoftDeletesPolicy softDeletesPolicy; private final LongSupplier globalCheckpointSupplier; private final ObjectIntHashMap snapshottedCommits; // Number of snapshots held against each commit point. private volatile IndexCommit safeCommit; // the most recent safe commit point - its max_seqno at most the persisted global checkpoint. private volatile IndexCommit lastCommit; // the most recent commit point - CombinedDeletionPolicy(Logger logger, TranslogDeletionPolicy translogDeletionPolicy, LongSupplier globalCheckpointSupplier) { + CombinedDeletionPolicy(Logger logger, TranslogDeletionPolicy translogDeletionPolicy, + SoftDeletesPolicy softDeletesPolicy, LongSupplier globalCheckpointSupplier) { this.logger = logger; this.translogDeletionPolicy = translogDeletionPolicy; + this.softDeletesPolicy = softDeletesPolicy; this.globalCheckpointSupplier = globalCheckpointSupplier; this.snapshottedCommits = new ObjectIntHashMap<>(); } @@ -80,7 +83,7 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { deleteCommit(commits.get(i)); } } - updateTranslogDeletionPolicy(); + updateRetentionPolicy(); } private void deleteCommit(IndexCommit commit) throws IOException { @@ -90,7 +93,7 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { assert commit.isDeleted() : "Deletion commit [" + commitDescription(commit) + "] was suppressed"; } - private void updateTranslogDeletionPolicy() throws IOException { + private void updateRetentionPolicy() throws IOException { assert Thread.holdsLock(this); logger.debug("Safe commit [{}], last commit [{}]", commitDescription(safeCommit), commitDescription(lastCommit)); assert safeCommit.isDeleted() == false : "The safe commit must not be deleted"; @@ -101,6 +104,9 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { assert minRequiredGen <= lastGen : "minRequiredGen must not be greater than lastGen"; translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastGen); translogDeletionPolicy.setMinTranslogGenerationForRecovery(minRequiredGen); + + softDeletesPolicy.setLocalCheckpointOfSafeCommit( + Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY))); } /** 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 4d95cf89ef0..08724d6e794 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -58,6 +58,7 @@ import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; @@ -97,6 +98,7 @@ public abstract class Engine implements Closeable { public static final String SYNC_COMMIT_ID = "sync_id"; public static final String HISTORY_UUID_KEY = "history_uuid"; + public static final String MIN_RETAINED_SEQNO = "min_retained_seq_no"; protected final ShardId shardId; protected final String allocationId; @@ -585,18 +587,32 @@ public abstract class Engine implements Closeable { public abstract void syncTranslog() throws IOException; - public abstract Closeable acquireTranslogRetentionLock(); + /** + * Acquires a lock on the translog files and Lucene soft-deleted documents to prevent them from being trimmed + */ + public abstract Closeable acquireRetentionLockForPeerRecovery(); /** - * Creates a new translog snapshot from this engine for reading translog operations whose seq# at least the provided seq#. - * The caller has to close the returned snapshot after finishing the reading. + * Creates a new history snapshot from Lucene for reading operations whose seqno in the requesting seqno range (both inclusive) */ - public abstract Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException; + public abstract Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, + long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException; /** - * Returns the estimated number of translog operations in this engine whose seq# at least the provided seq#. + * Creates a new history snapshot for reading operations since {@code startingSeqNo} (inclusive). + * The returned snapshot can be retrieved from either Lucene index or translog files. */ - public abstract int estimateTranslogOperationsFromMinSeq(long minSeqNo); + public abstract Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException; + + /** + * Returns the estimated number of history operations whose seq# at least {@code startingSeqNo}(inclusive) in this engine. + */ + public abstract int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException; + + /** + * Checks if this engine has every operations since {@code startingSeqNo}(inclusive) in its history (either Lucene or translog) + */ + public abstract boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException; public abstract TranslogStats getTranslogStats(); diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index 2deae61bd52..23a90553f60 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.unit.MemorySizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.codec.CodecService; +import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; @@ -80,6 +81,7 @@ public final class EngineConfig { private final CircuitBreakerService circuitBreakerService; private final LongSupplier globalCheckpointSupplier; private final LongSupplier primaryTermSupplier; + private final TombstoneDocSupplier tombstoneDocSupplier; /** * Index setting to change the low level lucene codec used for writing new segments. @@ -126,7 +128,8 @@ public final class EngineConfig { List externalRefreshListener, List internalRefreshListener, Sort indexSort, TranslogRecoveryRunner translogRecoveryRunner, CircuitBreakerService circuitBreakerService, - LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier) { + LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier, + TombstoneDocSupplier tombstoneDocSupplier) { this.shardId = shardId; this.allocationId = allocationId; this.indexSettings = indexSettings; @@ -164,6 +167,7 @@ public final class EngineConfig { this.circuitBreakerService = circuitBreakerService; this.globalCheckpointSupplier = globalCheckpointSupplier; this.primaryTermSupplier = primaryTermSupplier; + this.tombstoneDocSupplier = tombstoneDocSupplier; } /** @@ -373,4 +377,25 @@ public final class EngineConfig { public LongSupplier getPrimaryTermSupplier() { return primaryTermSupplier; } + + /** + * A supplier supplies tombstone documents which will be used in soft-update methods. + * The returned document consists only _uid, _seqno, _term and _version fields; other metadata fields are excluded. + */ + public interface TombstoneDocSupplier { + /** + * Creates a tombstone document for a delete operation. + */ + ParsedDocument newDeleteTombstoneDoc(String type, String id); + + /** + * Creates a tombstone document for a noop operation. + * @param reason the reason of an a noop + */ + ParsedDocument newNoopTombstoneDoc(String reason); + } + + public TombstoneDocSupplier getTombstoneDocSupplier() { + return tombstoneDocSupplier; + } } 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 023e659ffab..da4decc93b1 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -21,16 +21,20 @@ package org.elasticsearch.index.engine; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.ReferenceManager; @@ -42,6 +46,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.InfoStream; +import org.elasticsearch.Assertions; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.Nullable; @@ -61,7 +66,11 @@ import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParseContext; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; import org.elasticsearch.index.seqno.LocalCheckpointTracker; @@ -140,6 +149,10 @@ public class InternalEngine extends Engine { private final CounterMetric numDocDeletes = new CounterMetric(); private final CounterMetric numDocAppends = new CounterMetric(); private final CounterMetric numDocUpdates = new CounterMetric(); + private final NumericDocValuesField softDeletesField = Lucene.newSoftDeletesField(); + private final boolean softDeleteEnabled; + private final SoftDeletesPolicy softDeletesPolicy; + private final LastRefreshedCheckpointListener lastRefreshedCheckpointListener; /** * How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh. IndexingMemoryController polls this @@ -184,8 +197,10 @@ public class InternalEngine extends Engine { assert translog.getGeneration() != null; this.translog = translog; this.localCheckpointTracker = createLocalCheckpointTracker(localCheckpointTrackerSupplier); + this.softDeleteEnabled = engineConfig.getIndexSettings().isSoftDeleteEnabled(); + this.softDeletesPolicy = newSoftDeletesPolicy(); this.combinedDeletionPolicy = - new CombinedDeletionPolicy(logger, translogDeletionPolicy, translog::getLastSyncedGlobalCheckpoint); + new CombinedDeletionPolicy(logger, translogDeletionPolicy, softDeletesPolicy, translog::getLastSyncedGlobalCheckpoint); writer = createWriter(); bootstrapAppendOnlyInfoFromWriter(writer); historyUUID = loadHistoryUUID(writer); @@ -215,6 +230,8 @@ public class InternalEngine extends Engine { for (ReferenceManager.RefreshListener listener: engineConfig.getInternalRefreshListener()) { this.internalSearcherManager.addListener(listener); } + this.lastRefreshedCheckpointListener = new LastRefreshedCheckpointListener(localCheckpointTracker.getCheckpoint()); + this.internalSearcherManager.addListener(lastRefreshedCheckpointListener); success = true; } finally { if (success == false) { @@ -240,6 +257,18 @@ public class InternalEngine extends Engine { return localCheckpointTrackerSupplier.apply(maxSeqNo, localCheckpoint); } + private SoftDeletesPolicy newSoftDeletesPolicy() throws IOException { + final Map commitUserData = store.readLastCommittedSegmentsInfo().userData; + final long lastMinRetainedSeqNo; + if (commitUserData.containsKey(Engine.MIN_RETAINED_SEQNO)) { + lastMinRetainedSeqNo = Long.parseLong(commitUserData.get(Engine.MIN_RETAINED_SEQNO)); + } else { + lastMinRetainedSeqNo = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)) + 1; + } + return new SoftDeletesPolicy(translog::getLastSyncedGlobalCheckpoint, lastMinRetainedSeqNo, + engineConfig.getIndexSettings().getSoftDeleteRetentionOperations()); + } + /** * This reference manager delegates all it's refresh calls to another (internal) SearcherManager * The main purpose for this is that if we have external refreshes happening we don't issue extra @@ -451,19 +480,31 @@ public class InternalEngine extends Engine { revisitIndexDeletionPolicyOnTranslogSynced(); } + /** + * Creates a new history snapshot for reading operations since the provided seqno. + * The returned snapshot can be retrieved from either Lucene index or translog files. + */ @Override - public Closeable acquireTranslogRetentionLock() { - return getTranslog().acquireRetentionLock(); + public Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { + if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + return newChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false); + } else { + return getTranslog().newSnapshotFromMinSeqNo(startingSeqNo); + } } + /** + * Returns the estimated number of history operations whose seq# at least the provided seq# in this engine. + */ @Override - public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { - return getTranslog().newSnapshotFromMinSeqNo(minSeqNo); - } - - @Override - public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { - return getTranslog().estimateTotalOperationsFromMinSeq(minSeqNo); + public int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { + if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + try (Translog.Snapshot snapshot = newChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false)) { + return snapshot.totalOperations(); + } + } else { + return getTranslog().estimateTotalOperationsFromMinSeq(startingSeqNo); + } } @Override @@ -790,7 +831,7 @@ public class InternalEngine extends Engine { if (plan.earlyResultOnPreFlightError.isPresent()) { indexResult = plan.earlyResultOnPreFlightError.get(); assert indexResult.getResultType() == Result.Type.FAILURE : indexResult.getResultType(); - } else if (plan.indexIntoLucene) { + } else if (plan.indexIntoLucene || plan.addStaleOpToLucene) { indexResult = indexIntoLucene(index, plan); } else { indexResult = new IndexResult( @@ -801,8 +842,10 @@ public class InternalEngine extends Engine { if (indexResult.getResultType() == Result.Type.SUCCESS) { location = translog.add(new Translog.Index(index, indexResult)); } else if (indexResult.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - // if we have document failure, record it as a no-op in the translog with the generated seq_no - location = translog.add(new Translog.NoOp(indexResult.getSeqNo(), index.primaryTerm(), indexResult.getFailure().toString())); + // if we have document failure, record it as a no-op in the translog and Lucene with the generated seq_no + final NoOp noOp = new NoOp(indexResult.getSeqNo(), index.primaryTerm(), index.origin(), + index.startTime(), indexResult.getFailure().toString()); + location = innerNoOp(noOp).getTranslogLocation(); } else { location = null; } @@ -854,7 +897,6 @@ public class InternalEngine extends Engine { // unlike the primary, replicas don't really care to about creation status of documents // this allows to ignore the case where a document was found in the live version maps in // a delete state and return false for the created flag in favor of code simplicity - final OpVsLuceneDocStatus opVsLucene; if (index.seqNo() <= localCheckpointTracker.getCheckpoint()){ // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already @@ -863,16 +905,15 @@ public class InternalEngine extends Engine { // question may have been deleted in an out of order op that is not replayed. // See testRecoverFromStoreWithOutOfOrderDelete for an example of local recovery // See testRecoveryWithOutOfOrderDelete for an example of peer recovery - opVsLucene = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; - } else { - opVsLucene = compareOpToLuceneDocBasedOnSeqNo(index); - } - if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { plan = IndexingStrategy.processButSkipLucene(false, index.seqNo(), index.version()); } else { - plan = IndexingStrategy.processNormally( - opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, index.seqNo(), index.version() - ); + final OpVsLuceneDocStatus opVsLucene = compareOpToLuceneDocBasedOnSeqNo(index); + if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { + plan = IndexingStrategy.processAsStaleOp(softDeleteEnabled, index.seqNo(), index.version()); + } else { + plan = IndexingStrategy.processNormally(opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, + index.seqNo(), index.version()); + } } } return plan; @@ -921,7 +962,7 @@ public class InternalEngine extends Engine { throws IOException { assert plan.seqNoForIndexing >= 0 : "ops should have an assigned seq no.; origin: " + index.origin(); assert plan.versionForIndexing >= 0 : "version must be set. got " + plan.versionForIndexing; - assert plan.indexIntoLucene; + assert plan.indexIntoLucene || plan.addStaleOpToLucene; /* Update the document's sequence number and primary term; the sequence number here is derived here from either the sequence * number service if this is on the primary, or the existing document's sequence number if this is on the replica. The * primary term here has already been set, see IndexShard#prepareIndex where the Engine$Index operation is created. @@ -929,7 +970,9 @@ public class InternalEngine extends Engine { index.parsedDoc().updateSeqID(plan.seqNoForIndexing, index.primaryTerm()); index.parsedDoc().version().setLongValue(plan.versionForIndexing); try { - if (plan.useLuceneUpdateDocument) { + if (plan.addStaleOpToLucene) { + addStaleDocs(index.docs(), indexWriter); + } else if (plan.useLuceneUpdateDocument) { updateDocs(index.uid(), index.docs(), indexWriter); } else { // document does not exists, we can optimize for create, but double check if assertions are running @@ -993,16 +1036,29 @@ public class InternalEngine extends Engine { numDocAppends.inc(docs.size()); } - private static final class IndexingStrategy { + private void addStaleDocs(final List docs, final IndexWriter indexWriter) throws IOException { + assert softDeleteEnabled : "Add history documents but soft-deletes is disabled"; + for (ParseContext.Document doc : docs) { + doc.add(softDeletesField); // soft-deleted every document before adding to Lucene + } + if (docs.size() > 1) { + indexWriter.addDocuments(docs); + } else { + indexWriter.addDocument(docs.get(0)); + } + } + + protected static final class IndexingStrategy { final boolean currentNotFoundOrDeleted; final boolean useLuceneUpdateDocument; final long seqNoForIndexing; final long versionForIndexing; final boolean indexIntoLucene; + final boolean addStaleOpToLucene; final Optional earlyResultOnPreFlightError; private IndexingStrategy(boolean currentNotFoundOrDeleted, boolean useLuceneUpdateDocument, - boolean indexIntoLucene, long seqNoForIndexing, + boolean indexIntoLucene, boolean addStaleOpToLucene, long seqNoForIndexing, long versionForIndexing, IndexResult earlyResultOnPreFlightError) { assert useLuceneUpdateDocument == false || indexIntoLucene : "use lucene update is set to true, but we're not indexing into lucene"; @@ -1015,37 +1071,40 @@ public class InternalEngine extends Engine { this.seqNoForIndexing = seqNoForIndexing; this.versionForIndexing = versionForIndexing; this.indexIntoLucene = indexIntoLucene; + this.addStaleOpToLucene = addStaleOpToLucene; this.earlyResultOnPreFlightError = earlyResultOnPreFlightError == null ? Optional.empty() : Optional.of(earlyResultOnPreFlightError); } static IndexingStrategy optimizedAppendOnly(long seqNoForIndexing) { - return new IndexingStrategy(true, false, true, seqNoForIndexing, 1, null); + return new IndexingStrategy(true, false, true, false, seqNoForIndexing, 1, null); } static IndexingStrategy skipDueToVersionConflict( VersionConflictEngineException e, boolean currentNotFoundOrDeleted, long currentVersion, long term) { final IndexResult result = new IndexResult(e, currentVersion, term); return new IndexingStrategy( - currentNotFoundOrDeleted, false, false, SequenceNumbers.UNASSIGNED_SEQ_NO, Versions.NOT_FOUND, result); + currentNotFoundOrDeleted, false, false, false, SequenceNumbers.UNASSIGNED_SEQ_NO, Versions.NOT_FOUND, result); } static IndexingStrategy processNormally(boolean currentNotFoundOrDeleted, long seqNoForIndexing, long versionForIndexing) { return new IndexingStrategy(currentNotFoundOrDeleted, currentNotFoundOrDeleted == false, - true, seqNoForIndexing, versionForIndexing, null); + true, false, seqNoForIndexing, versionForIndexing, null); } static IndexingStrategy overrideExistingAsIfNotThere( long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(true, true, true, seqNoForIndexing, versionForIndexing, null); + return new IndexingStrategy(true, true, true, false, seqNoForIndexing, versionForIndexing, null); } - static IndexingStrategy processButSkipLucene(boolean currentNotFoundOrDeleted, - long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(currentNotFoundOrDeleted, false, - false, seqNoForIndexing, versionForIndexing, null); + static IndexingStrategy processButSkipLucene(boolean currentNotFoundOrDeleted, long seqNoForIndexing, long versionForIndexing) { + return new IndexingStrategy(currentNotFoundOrDeleted, false, false, false, seqNoForIndexing, versionForIndexing, null); + } + + static IndexingStrategy processAsStaleOp(boolean addStaleOpToLucene, long seqNoForIndexing, long versionForIndexing) { + return new IndexingStrategy(false, false, false, addStaleOpToLucene, seqNoForIndexing, versionForIndexing, null); } } @@ -1072,10 +1131,18 @@ public class InternalEngine extends Engine { } private void updateDocs(final Term uid, final List docs, final IndexWriter indexWriter) throws IOException { - if (docs.size() > 1) { - indexWriter.updateDocuments(uid, docs); + if (softDeleteEnabled) { + if (docs.size() > 1) { + indexWriter.softUpdateDocuments(uid, docs, softDeletesField); + } else { + indexWriter.softUpdateDocument(uid, docs.get(0), softDeletesField); + } } else { - indexWriter.updateDocument(uid, docs.get(0)); + if (docs.size() > 1) { + indexWriter.updateDocuments(uid, docs); + } else { + indexWriter.updateDocument(uid, docs.get(0)); + } } numDocUpdates.inc(docs.size()); } @@ -1099,7 +1166,7 @@ public class InternalEngine extends Engine { if (plan.earlyResultOnPreflightError.isPresent()) { deleteResult = plan.earlyResultOnPreflightError.get(); - } else if (plan.deleteFromLucene) { + } else if (plan.deleteFromLucene || plan.addStaleOpToLucene) { deleteResult = deleteInLucene(delete, plan); } else { deleteResult = new DeleteResult( @@ -1110,8 +1177,10 @@ public class InternalEngine extends Engine { if (deleteResult.getResultType() == Result.Type.SUCCESS) { location = translog.add(new Translog.Delete(delete, deleteResult)); } else if (deleteResult.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - location = translog.add(new Translog.NoOp(deleteResult.getSeqNo(), - delete.primaryTerm(), deleteResult.getFailure().toString())); + // if we have document failure, record it as a no-op in the translog and Lucene with the generated seq_no + final NoOp noOp = new NoOp(deleteResult.getSeqNo(), delete.primaryTerm(), delete.origin(), + delete.startTime(), deleteResult.getFailure().toString()); + location = innerNoOp(noOp).getTranslogLocation(); } else { location = null; } @@ -1142,7 +1211,7 @@ public class InternalEngine extends Engine { // unlike the primary, replicas don't really care to about found status of documents // this allows to ignore the case where a document was found in the live version maps in // a delete state and return true for the found flag in favor of code simplicity - final OpVsLuceneDocStatus opVsLucene; + final DeletionStrategy plan; if (delete.seqNo() <= localCheckpointTracker.getCheckpoint()) { // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already @@ -1151,18 +1220,15 @@ public class InternalEngine extends Engine { // question may have been deleted in an out of order op that is not replayed. // See testRecoverFromStoreWithOutOfOrderDelete for an example of local recovery // See testRecoveryWithOutOfOrderDelete for an example of peer recovery - opVsLucene = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; - } else { - opVsLucene = compareOpToLuceneDocBasedOnSeqNo(delete); - } - - final DeletionStrategy plan; - if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { plan = DeletionStrategy.processButSkipLucene(false, delete.seqNo(), delete.version()); } else { - plan = DeletionStrategy.processNormally( - opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, - delete.seqNo(), delete.version()); + final OpVsLuceneDocStatus opVsLucene = compareOpToLuceneDocBasedOnSeqNo(delete); + if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { + plan = DeletionStrategy.processAsStaleOp(softDeleteEnabled, false, delete.seqNo(), delete.version()); + } else { + plan = DeletionStrategy.processNormally(opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, + delete.seqNo(), delete.version()); + } } return plan; } @@ -1197,15 +1263,31 @@ public class InternalEngine extends Engine { private DeleteResult deleteInLucene(Delete delete, DeletionStrategy plan) throws IOException { try { - if (plan.currentlyDeleted == false) { + if (softDeleteEnabled) { + final ParsedDocument tombstone = engineConfig.getTombstoneDocSupplier().newDeleteTombstoneDoc(delete.type(), delete.id()); + assert tombstone.docs().size() == 1 : "Tombstone doc should have single doc [" + tombstone + "]"; + tombstone.updateSeqID(plan.seqNoOfDeletion, delete.primaryTerm()); + tombstone.version().setLongValue(plan.versionOfDeletion); + final ParseContext.Document doc = tombstone.docs().get(0); + assert doc.getField(SeqNoFieldMapper.TOMBSTONE_NAME) != null : + "Delete tombstone document but _tombstone field is not set [" + doc + " ]"; + doc.add(softDeletesField); + if (plan.addStaleOpToLucene || plan.currentlyDeleted) { + indexWriter.addDocument(doc); + } else { + indexWriter.softUpdateDocument(delete.uid(), doc, softDeletesField); + } + } else if (plan.currentlyDeleted == false) { // any exception that comes from this is a either an ACE or a fatal exception there // can't be any document failures coming from this indexWriter.deleteDocuments(delete.uid()); - numDocDeletes.inc(); } - versionMap.putDeleteUnderLock(delete.uid().bytes(), - new DeleteVersionValue(plan.versionOfDeletion, plan.seqNoOfDeletion, delete.primaryTerm(), - engineConfig.getThreadPool().relativeTimeInMillis())); + if (plan.deleteFromLucene) { + numDocDeletes.inc(); + versionMap.putDeleteUnderLock(delete.uid().bytes(), + new DeleteVersionValue(plan.versionOfDeletion, plan.seqNoOfDeletion, delete.primaryTerm(), + engineConfig.getThreadPool().relativeTimeInMillis())); + } return new DeleteResult( plan.versionOfDeletion, getPrimaryTerm(), plan.seqNoOfDeletion, plan.currentlyDeleted == false); } catch (Exception ex) { @@ -1219,15 +1301,16 @@ public class InternalEngine extends Engine { } } - private static final class DeletionStrategy { + protected static final class DeletionStrategy { // of a rare double delete final boolean deleteFromLucene; + final boolean addStaleOpToLucene; final boolean currentlyDeleted; final long seqNoOfDeletion; final long versionOfDeletion; final Optional earlyResultOnPreflightError; - private DeletionStrategy(boolean deleteFromLucene, boolean currentlyDeleted, + private DeletionStrategy(boolean deleteFromLucene, boolean addStaleOpToLucene, boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion, DeleteResult earlyResultOnPreflightError) { assert (deleteFromLucene && earlyResultOnPreflightError != null) == false : @@ -1235,6 +1318,7 @@ public class InternalEngine extends Engine { "deleteFromLucene: " + deleteFromLucene + " earlyResultOnPreFlightError:" + earlyResultOnPreflightError; this.deleteFromLucene = deleteFromLucene; + this.addStaleOpToLucene = addStaleOpToLucene; this.currentlyDeleted = currentlyDeleted; this.seqNoOfDeletion = seqNoOfDeletion; this.versionOfDeletion = versionOfDeletion; @@ -1246,16 +1330,22 @@ public class InternalEngine extends Engine { VersionConflictEngineException e, long currentVersion, long term, boolean currentlyDeleted) { final long unassignedSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; final DeleteResult deleteResult = new DeleteResult(e, currentVersion, term, unassignedSeqNo, currentlyDeleted == false); - return new DeletionStrategy(false, currentlyDeleted, unassignedSeqNo, Versions.NOT_FOUND, deleteResult); + return new DeletionStrategy(false, false, currentlyDeleted, unassignedSeqNo, Versions.NOT_FOUND, deleteResult); } static DeletionStrategy processNormally(boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(true, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); + return new DeletionStrategy(true, false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); } - public static DeletionStrategy processButSkipLucene(boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); + public static DeletionStrategy processButSkipLucene(boolean currentlyDeleted, + long seqNoOfDeletion, long versionOfDeletion) { + return new DeletionStrategy(false, false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); + } + + static DeletionStrategy processAsStaleOp(boolean addStaleOpToLucene, boolean currentlyDeleted, + long seqNoOfDeletion, long versionOfDeletion) { + return new DeletionStrategy(false, addStaleOpToLucene, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); } } @@ -1284,7 +1374,28 @@ public class InternalEngine extends Engine { assert noOp.seqNo() > SequenceNumbers.NO_OPS_PERFORMED; final long seqNo = noOp.seqNo(); try { - final NoOpResult noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo()); + Exception failure = null; + if (softDeleteEnabled) { + try { + final ParsedDocument tombstone = engineConfig.getTombstoneDocSupplier().newNoopTombstoneDoc(noOp.reason()); + tombstone.updateSeqID(noOp.seqNo(), noOp.primaryTerm()); + // A noop tombstone does not require a _version but it's added to have a fully dense docvalues for the version field. + // 1L is selected to optimize the compression because it might probably be the most common value in version field. + tombstone.version().setLongValue(1L); + assert tombstone.docs().size() == 1 : "Tombstone should have a single doc [" + tombstone + "]"; + final ParseContext.Document doc = tombstone.docs().get(0); + assert doc.getField(SeqNoFieldMapper.TOMBSTONE_NAME) != null + : "Noop tombstone document but _tombstone field is not set [" + doc + " ]"; + doc.add(softDeletesField); + indexWriter.addDocument(doc); + } catch (Exception ex) { + if (maybeFailEngine("noop", ex)) { + throw ex; + } + failure = ex; + } + } + final NoOpResult noOpResult = failure != null ? new NoOpResult(getPrimaryTerm(), noOp.seqNo(), failure) : new NoOpResult(getPrimaryTerm(), noOp.seqNo()); if (noOp.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { final Translog.Location location = translog.add(new Translog.NoOp(noOp.seqNo(), noOp.primaryTerm(), noOp.reason())); noOpResult.setTranslogLocation(location); @@ -1309,6 +1420,7 @@ public class InternalEngine extends Engine { // since it flushes the index as well (though, in terms of concurrency, we are allowed to do it) // both refresh types will result in an internal refresh but only the external will also // pass the new reader reference to the external reader manager. + final long localCheckpointBeforeRefresh = getLocalCheckpoint(); // this will also cause version map ram to be freed hence we always account for it. final long bytes = indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh(); @@ -1334,6 +1446,7 @@ public class InternalEngine extends Engine { } finally { store.decRef(); } + lastRefreshedCheckpointListener.updateRefreshedCheckpoint(localCheckpointBeforeRefresh); } } catch (AlreadyClosedException e) { failOnTragicEvent(e); @@ -1348,7 +1461,8 @@ public class InternalEngine extends Engine { } finally { writingBytes.addAndGet(-bytes); } - + assert lastRefreshedCheckpoint() >= localCheckpointBeforeRefresh : "refresh checkpoint was not advanced; " + + "local_checkpoint=" + localCheckpointBeforeRefresh + " refresh_checkpoint=" + lastRefreshedCheckpoint(); // TODO: maybe we should just put a scheduled job in threadPool? // We check for pruning in each delete request, but we also prune here e.g. in case a delete burst comes in and then no more deletes // for a long time: @@ -1930,7 +2044,11 @@ public class InternalEngine extends Engine { // pkg-private for testing IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOException { - return new IndexWriter(directory, iwc); + if (Assertions.ENABLED) { + return new AssertingIndexWriter(directory, iwc); + } else { + return new IndexWriter(directory, iwc); + } } private IndexWriterConfig getIndexWriterConfig() { @@ -1946,11 +2064,15 @@ public class InternalEngine extends Engine { } iwc.setInfoStream(verbose ? InfoStream.getDefault() : new LoggerInfoStream(logger)); iwc.setMergeScheduler(mergeScheduler); - MergePolicy mergePolicy = config().getMergePolicy(); // Give us the opportunity to upgrade old segments while performing // background merges - mergePolicy = new ElasticsearchMergePolicy(mergePolicy); - iwc.setMergePolicy(mergePolicy); + MergePolicy mergePolicy = config().getMergePolicy(); + if (softDeleteEnabled) { + iwc.setSoftDeletesField(Lucene.SOFT_DELETES_FIELD); + mergePolicy = new RecoverySourcePruneMergePolicy(SourceFieldMapper.RECOVERY_SOURCE_NAME, softDeletesPolicy::getRetentionQuery, + new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, softDeletesPolicy::getRetentionQuery, mergePolicy)); + } + iwc.setMergePolicy(new ElasticsearchMergePolicy(mergePolicy)); iwc.setSimilarity(engineConfig.getSimilarity()); iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().getMbFrac()); iwc.setCodec(engineConfig.getCodec()); @@ -2147,6 +2269,9 @@ public class InternalEngine extends Engine { commitData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(localCheckpointTracker.getMaxSeqNo())); commitData.put(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, Long.toString(maxUnsafeAutoIdTimestamp.get())); commitData.put(HISTORY_UUID_KEY, historyUUID); + if (softDeleteEnabled) { + commitData.put(Engine.MIN_RETAINED_SEQNO, Long.toString(softDeletesPolicy.getMinRetainedSeqNo())); + } logger.trace("committing writer with commit data [{}]", commitData); return commitData.entrySet().iterator(); }); @@ -2202,6 +2327,7 @@ public class InternalEngine extends Engine { final IndexSettings indexSettings = engineConfig.getIndexSettings(); translogDeletionPolicy.setRetentionAgeInMillis(indexSettings.getTranslogRetentionAge().getMillis()); translogDeletionPolicy.setRetentionSizeInBytes(indexSettings.getTranslogRetentionSize().getBytes()); + softDeletesPolicy.setRetentionOperations(indexSettings.getSoftDeleteRetentionOperations()); } public MergeStats getMergeStats() { @@ -2296,6 +2422,69 @@ public class InternalEngine extends Engine { return numDocUpdates.count(); } + @Override + public Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, + long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException { + // TODO: Should we defer the refresh until we really need it? + ensureOpen(); + if (lastRefreshedCheckpoint() < toSeqNo) { + refresh(source, SearcherScope.INTERNAL); + } + Searcher searcher = acquireSearcher(source, SearcherScope.INTERNAL); + try { + LuceneChangesSnapshot snapshot = new LuceneChangesSnapshot( + searcher, mapperService, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE, fromSeqNo, toSeqNo, requiredFullRange); + searcher = null; + return snapshot; + } catch (Exception e) { + try { + maybeFailEngine("acquire changes snapshot", e); + } catch (Exception inner) { + e.addSuppressed(inner); + } + throw e; + } finally { + IOUtils.close(searcher); + } + } + + @Override + public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException { + if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + return getMinRetainedSeqNo() <= startingSeqNo; + } else { + final long currentLocalCheckpoint = getLocalCheckpointTracker().getCheckpoint(); + final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1); + try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(startingSeqNo)) { + Translog.Operation operation; + while ((operation = snapshot.next()) != null) { + if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { + tracker.markSeqNoAsCompleted(operation.seqNo()); + } + } + } + return tracker.getCheckpoint() >= currentLocalCheckpoint; + } + } + + /** + * Returns the minimum seqno that is retained in the Lucene index. + * Operations whose seq# are at least this value should exist in the Lucene index. + */ + final long getMinRetainedSeqNo() { + assert softDeleteEnabled : Thread.currentThread().getName(); + return softDeletesPolicy.getMinRetainedSeqNo(); + } + + @Override + public Closeable acquireRetentionLockForPeerRecovery() { + if (softDeleteEnabled) { + return softDeletesPolicy.acquireRetentionLock(); + } else { + return translog.acquireRetentionLock(); + } + } + @Override public boolean isRecovering() { return pendingTranslogRecovery.get(); @@ -2311,4 +2500,69 @@ public class InternalEngine extends Engine { } return commitData; } + + private final class AssertingIndexWriter extends IndexWriter { + AssertingIndexWriter(Directory d, IndexWriterConfig conf) throws IOException { + super(d, conf); + } + @Override + public long updateDocument(Term term, Iterable doc) throws IOException { + assert softDeleteEnabled == false : "Call #updateDocument but soft-deletes is enabled"; + return super.updateDocument(term, doc); + } + @Override + public long updateDocuments(Term delTerm, Iterable> docs) throws IOException { + assert softDeleteEnabled == false : "Call #updateDocuments but soft-deletes is enabled"; + return super.updateDocuments(delTerm, docs); + } + @Override + public long deleteDocuments(Term... terms) throws IOException { + assert softDeleteEnabled == false : "Call #deleteDocuments but soft-deletes is enabled"; + return super.deleteDocuments(terms); + } + @Override + public long softUpdateDocument(Term term, Iterable doc, Field... softDeletes) throws IOException { + assert softDeleteEnabled : "Call #softUpdateDocument but soft-deletes is disabled"; + return super.softUpdateDocument(term, doc, softDeletes); + } + @Override + public long softUpdateDocuments(Term term, Iterable> docs, Field... softDeletes) throws IOException { + assert softDeleteEnabled : "Call #softUpdateDocuments but soft-deletes is disabled"; + return super.softUpdateDocuments(term, docs, softDeletes); + } + } + + /** + * Returned the last local checkpoint value has been refreshed internally. + */ + final long lastRefreshedCheckpoint() { + return lastRefreshedCheckpointListener.refreshedCheckpoint.get(); + } + + private final class LastRefreshedCheckpointListener implements ReferenceManager.RefreshListener { + final AtomicLong refreshedCheckpoint; + private long pendingCheckpoint; + + LastRefreshedCheckpointListener(long initialLocalCheckpoint) { + this.refreshedCheckpoint = new AtomicLong(initialLocalCheckpoint); + } + + @Override + public void beforeRefresh() { + // all changes until this point should be visible after refresh + pendingCheckpoint = localCheckpointTracker.getCheckpoint(); + } + + @Override + public void afterRefresh(boolean didRefresh) { + if (didRefresh) { + updateRefreshedCheckpoint(pendingCheckpoint); + } + } + + void updateRefreshedCheckpoint(long checkpoint) { + refreshedCheckpoint.updateAndGet(curr -> Math.max(curr, checkpoint)); + assert refreshedCheckpoint.get() >= checkpoint : refreshedCheckpoint.get() + " < " + checkpoint; + } + } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java new file mode 100644 index 00000000000..deebfba9ed4 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -0,0 +1,368 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +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.index.Term; +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.util.ArrayUtil; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.index.fieldvisitor.FieldsVisitor; +import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.VersionFieldMapper; +import org.elasticsearch.index.translog.Translog; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +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; + private long lastSeenSeqNo; + private int skippedOperations; + private final boolean requiredFullRange; + + private final IndexSearcher indexSearcher; + private final MapperService mapperService; + private int docIndex = 0; + private final int totalHits; + private ScoreDoc[] scoreDocs; + private final ParallelArray parallelArray; + private final Closeable onClose; + + /** + * Creates a new "translog" snapshot from Lucene for reading operations whose seq# in the specified range. + * + * @param engineSearcher the internal engine searcher which will be taken over if the snapshot is opened successfully + * @param mapperService the mapper service which will be mainly used to resolve the document's type and uid + * @param searchBatchSize the number of documents should be returned by each search + * @param fromSeqNo the min requesting seq# - inclusive + * @param toSeqNo the maximum requesting seq# - inclusive + * @param requiredFullRange if true, the snapshot will strictly check for the existence of operations between fromSeqNo and toSeqNo + */ + LuceneChangesSnapshot(Engine.Searcher engineSearcher, MapperService mapperService, int searchBatchSize, + long fromSeqNo, long toSeqNo, boolean requiredFullRange) 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.mapperService = mapperService; + this.searchBatchSize = searchBatchSize; + this.fromSeqNo = fromSeqNo; + this.toSeqNo = toSeqNo; + this.lastSeenSeqNo = fromSeqNo - 1; + this.requiredFullRange = requiredFullRange; + this.indexSearcher = new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); + this.indexSearcher.setQueryCache(null); + this.parallelArray = new ParallelArray(searchBatchSize); + final TopDocs topDocs = searchOperations(null); + this.totalHits = Math.toIntExact(topDocs.totalHits); + this.scoreDocs = topDocs.scoreDocs; + fillParallelArray(scoreDocs, parallelArray); + } + + @Override + public void close() throws IOException { + onClose.close(); + } + + @Override + public int totalOperations() { + return totalHits; + } + + @Override + public int skippedOperations() { + return skippedOperations; + } + + @Override + public Translog.Operation next() throws IOException { + Translog.Operation op = null; + for (int idx = nextDocIndex(); idx != -1; idx = nextDocIndex()) { + op = readDocAsOp(idx); + if (op != null) { + break; + } + } + if (requiredFullRange) { + rangeCheck(op); + } + if (op != null) { + lastSeenSeqNo = op.seqNo(); + } + return op; + } + + private void rangeCheck(Translog.Operation op) { + if (op == null) { + if (lastSeenSeqNo < toSeqNo) { + throw new IllegalStateException("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 IllegalStateException("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(prev).scoreDocs; + fillParallelArray(scoreDocs, parallelArray); + docIndex = 0; + } + if (docIndex < scoreDocs.length) { + int idx = docIndex; + docIndex++; + return idx; + } + return -1; + } + + private void fillParallelArray(ScoreDoc[] scoreDocs, ParallelArray parallelArray) throws IOException { + if (scoreDocs.length > 0) { + for (int i = 0; i < scoreDocs.length; i++) { + scoreDocs[i].shardIndex = i; + } + // for better loading performance we sort the array by docID and + // then visit all leaves in order. + ArrayUtil.introSort(scoreDocs, Comparator.comparingInt(i -> i.doc)); + int docBase = -1; + int maxDoc = 0; + List leaves = indexSearcher.getIndexReader().leaves(); + int readerIndex = 0; + CombinedDocValues combinedDocValues = null; + LeafReaderContext leaf = null; + for (int i = 0; i < scoreDocs.length; i++) { + ScoreDoc scoreDoc = scoreDocs[i]; + if (scoreDoc.doc >= docBase + maxDoc) { + do { + leaf = leaves.get(readerIndex++); + docBase = leaf.docBase; + maxDoc = leaf.reader().maxDoc(); + } while (scoreDoc.doc >= docBase + maxDoc); + combinedDocValues = new CombinedDocValues(leaf.reader()); + } + final int segmentDocID = scoreDoc.doc - docBase; + final int index = scoreDoc.shardIndex; + parallelArray.leafReaderContexts[index] = leaf; + parallelArray.seqNo[index] = combinedDocValues.docSeqNo(segmentDocID); + parallelArray.primaryTerm[index] = combinedDocValues.docPrimaryTerm(segmentDocID); + parallelArray.version[index] = combinedDocValues.docVersion(segmentDocID); + parallelArray.isTombStone[index] = combinedDocValues.isTombstone(segmentDocID); + parallelArray.hasRecoverySource[index] = combinedDocValues.hasRecoverySource(segmentDocID); + } + // now sort back based on the shardIndex. we use this to store the previous index + ArrayUtil.introSort(scoreDocs, Comparator.comparingInt(i -> i.shardIndex)); + } + } + + private TopDocs searchOperations(ScoreDoc after) throws IOException { + final Query rangeQuery = LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, lastSeenSeqNo + 1, toSeqNo); + final Sort sortedBySeqNoThenByTerm = new Sort( + new SortField(SeqNoFieldMapper.NAME, SortField.Type.LONG), + new SortField(SeqNoFieldMapper.PRIMARY_TERM_NAME, SortField.Type.LONG, true) + ); + return indexSearcher.searchAfter(after, rangeQuery, searchBatchSize, sortedBySeqNoThenByTerm); + } + + private Translog.Operation readDocAsOp(int docIndex) throws IOException { + final LeafReaderContext leaf = parallelArray.leafReaderContexts[docIndex]; + final int segmentDocID = scoreDocs[docIndex].doc - leaf.docBase; + final long primaryTerm = parallelArray.primaryTerm[docIndex]; + // We don't have to read the nested child documents - those docs don't have primary terms. + if (primaryTerm == -1) { + skippedOperations++; + return null; + } + final long seqNo = parallelArray.seqNo[docIndex]; + // Only pick the first seen seq# + if (seqNo == lastSeenSeqNo) { + skippedOperations++; + return null; + } + final long version = parallelArray.version[docIndex]; + final String sourceField = parallelArray.hasRecoverySource[docIndex] ? SourceFieldMapper.RECOVERY_SOURCE_NAME : + SourceFieldMapper.NAME; + final FieldsVisitor fields = new FieldsVisitor(true, sourceField); + leaf.reader().document(segmentDocID, fields); + fields.postProcess(mapperService); + + final Translog.Operation op; + final boolean isTombstone = parallelArray.isTombStone[docIndex]; + if (isTombstone && fields.uid() == null) { + op = new Translog.NoOp(seqNo, primaryTerm, fields.source().utf8ToString()); + assert version == 1L : "Noop tombstone should have version 1L; actual version [" + version + "]"; + assert assertDocSoftDeleted(leaf.reader(), segmentDocID) : "Noop but soft_deletes field is not set [" + op + "]"; + } else { + final String id = fields.uid().id(); + final String type = fields.uid().type(); + final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); + if (isTombstone) { + op = new Translog.Delete(type, id, uid, seqNo, primaryTerm, version); + assert assertDocSoftDeleted(leaf.reader(), segmentDocID) : "Delete op but soft_deletes field is not set [" + op + "]"; + } else { + final BytesReference source = fields.source(); + if (source == null) { + // 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 IllegalStateException("source not found for seqno=" + seqNo + + " from_seqno=" + fromSeqNo + " to_seqno=" + toSeqNo); + } else { + skippedOperations++; + return null; + } + } + // TODO: pass the latest timestamp from engine. + final long autoGeneratedIdTimestamp = -1; + op = new Translog.Index(type, id, seqNo, primaryTerm, version, + source.toBytesRef().bytes, fields.routing(), autoGeneratedIdTimestamp); + } + } + assert fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo && lastSeenSeqNo < op.seqNo() : "Unexpected operation; " + + "last_seen_seqno [" + lastSeenSeqNo + "], from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "], op [" + op + "]"; + return op; + } + + private 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 long[] version; + final long[] seqNo; + final long[] primaryTerm; + final boolean[] isTombStone; + final boolean[] hasRecoverySource; + + ParallelArray(int size) { + version = new long[size]; + seqNo = new long[size]; + primaryTerm = new long[size]; + isTombStone = new boolean[size]; + hasRecoverySource = new boolean[size]; + leafReaderContexts = new LeafReaderContext[size]; + } + } + + private static final class CombinedDocValues { + private final NumericDocValues versionDV; + private final NumericDocValues seqNoDV; + private final NumericDocValues primaryTermDV; + private final NumericDocValues tombstoneDV; + private final NumericDocValues recoverySource; + + CombinedDocValues(LeafReader leafReader) throws IOException { + this.versionDV = Objects.requireNonNull(leafReader.getNumericDocValues(VersionFieldMapper.NAME), "VersionDV is missing"); + this.seqNoDV = Objects.requireNonNull(leafReader.getNumericDocValues(SeqNoFieldMapper.NAME), "SeqNoDV is missing"); + this.primaryTermDV = Objects.requireNonNull( + leafReader.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME), "PrimaryTermDV is missing"); + this.tombstoneDV = leafReader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); + this.recoverySource = leafReader.getNumericDocValues(SourceFieldMapper.RECOVERY_SOURCE_NAME); + } + + long docVersion(int segmentDocId) throws IOException { + assert versionDV.docID() < segmentDocId; + if (versionDV.advanceExact(segmentDocId) == false) { + throw new IllegalStateException("DocValues for field [" + VersionFieldMapper.NAME + "] is not found"); + } + return versionDV.longValue(); + } + + long docSeqNo(int segmentDocId) throws IOException { + assert seqNoDV.docID() < segmentDocId; + if (seqNoDV.advanceExact(segmentDocId) == false) { + throw new IllegalStateException("DocValues for field [" + SeqNoFieldMapper.NAME + "] is not found"); + } + return seqNoDV.longValue(); + } + + long docPrimaryTerm(int segmentDocId) throws IOException { + if (primaryTermDV == null) { + return -1L; + } + assert primaryTermDV.docID() < segmentDocId; + // Use -1 for docs which don't have primary term. The caller considers those docs as nested docs. + if (primaryTermDV.advanceExact(segmentDocId) == false) { + return -1; + } + return primaryTermDV.longValue(); + } + + boolean isTombstone(int segmentDocId) throws IOException { + if (tombstoneDV == null) { + return false; + } + assert tombstoneDV.docID() < segmentDocId; + return tombstoneDV.advanceExact(segmentDocId) && tombstoneDV.longValue() > 0; + } + + boolean hasRecoverySource(int segmentDocId) throws IOException { + if (recoverySource == null) { + return false; + } + assert recoverySource.docID() < segmentDocId; + return recoverySource.advanceExact(segmentDocId); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java b/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java new file mode 100644 index 00000000000..fde97562de8 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java @@ -0,0 +1,292 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.StoredFieldsReader; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.CodecReader; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FilterCodecReader; +import org.apache.lucene.index.FilterNumericDocValues; +import org.apache.lucene.index.MergePolicy; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.OneMergeWrappingMergePolicy; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.index.StoredFieldVisitor; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.ConjunctionDISI; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.DocValuesFieldExistsQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.apache.lucene.util.BitSet; +import org.apache.lucene.util.BitSetIterator; + +import java.io.IOException; +import java.util.Arrays; +import java.util.function.Supplier; + +final class RecoverySourcePruneMergePolicy extends OneMergeWrappingMergePolicy { + RecoverySourcePruneMergePolicy(String recoverySourceField, Supplier retainSourceQuerySupplier, MergePolicy in) { + super(in, toWrap -> new OneMerge(toWrap.segments) { + @Override + public CodecReader wrapForMerge(CodecReader reader) throws IOException { + CodecReader wrapped = toWrap.wrapForMerge(reader); + return wrapReader(recoverySourceField, wrapped, retainSourceQuerySupplier); + } + }); + } + + // pkg private for testing + static CodecReader wrapReader(String recoverySourceField, CodecReader reader, Supplier retainSourceQuerySupplier) + throws IOException { + NumericDocValues recoverySource = reader.getNumericDocValues(recoverySourceField); + 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. + } + BooleanQuery.Builder builder = new BooleanQuery.Builder(); + builder.add(new DocValuesFieldExistsQuery(recoverySourceField), BooleanClause.Occur.FILTER); + builder.add(retainSourceQuerySupplier.get(), BooleanClause.Occur.FILTER); + IndexSearcher s = new IndexSearcher(reader); + s.setQueryCache(null); + Weight weight = s.createWeight(s.rewrite(builder.build()), false, 1.0f); + Scorer scorer = weight.scorer(reader.getContext()); + if (scorer != null) { + return new SourcePruningFilterCodecReader(recoverySourceField, reader, BitSet.of(scorer.iterator(), reader.maxDoc())); + } else { + return new SourcePruningFilterCodecReader(recoverySourceField, reader, null); + } + } + + private static class SourcePruningFilterCodecReader extends FilterCodecReader { + private final BitSet recoverySourceToKeep; + private final String recoverySourceField; + + SourcePruningFilterCodecReader(String recoverySourceField, CodecReader reader, BitSet recoverySourceToKeep) { + super(reader); + this.recoverySourceField = recoverySourceField; + this.recoverySourceToKeep = recoverySourceToKeep; + } + + @Override + public DocValuesProducer getDocValuesReader() { + DocValuesProducer docValuesReader = super.getDocValuesReader(); + return new FilterDocValuesProducer(docValuesReader) { + @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"; + final DocIdSetIterator intersection; + if (recoverySourceToKeep == null) { + // we can't return null here lucenes DocIdMerger expects an instance + intersection = DocIdSetIterator.empty(); + } else { + intersection = ConjunctionDISI.intersectIterators(Arrays.asList(numeric, + new BitSetIterator(recoverySourceToKeep, recoverySourceToKeep.length()))); + } + return new FilterNumericDocValues(numeric) { + @Override + public int nextDoc() throws IOException { + return intersection.nextDoc(); + } + + @Override + public int advance(int target) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean advanceExact(int target) { + throw new UnsupportedOperationException(); + } + }; + + } + return numeric; + } + }; + } + + @Override + public StoredFieldsReader getFieldsReader() { + StoredFieldsReader fieldsReader = super.getFieldsReader(); + return new FilterStoredFieldsReader(fieldsReader) { + @Override + public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException { + if (recoverySourceToKeep != null && recoverySourceToKeep.get(docID)) { + super.visitDocument(docID, visitor); + } else { + super.visitDocument(docID, new FilterStoredFieldVisitor(visitor) { + @Override + public Status needsField(FieldInfo fieldInfo) throws IOException { + if (recoverySourceField.equals(fieldInfo.name)) { + return Status.NO; + } + return super.needsField(fieldInfo); + } + }); + } + } + }; + } + + @Override + public CacheHelper getCoreCacheHelper() { + return null; + } + + @Override + public CacheHelper getReaderCacheHelper() { + return null; + } + + private static class FilterDocValuesProducer extends DocValuesProducer { + private final DocValuesProducer in; + + FilterDocValuesProducer(DocValuesProducer in) { + this.in = in; + } + + @Override + public NumericDocValues getNumeric(FieldInfo field) throws IOException { + return in.getNumeric(field); + } + + @Override + public BinaryDocValues getBinary(FieldInfo field) throws IOException { + return in.getBinary(field); + } + + @Override + public SortedDocValues getSorted(FieldInfo field) throws IOException { + return in.getSorted(field); + } + + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + return in.getSortedNumeric(field); + } + + @Override + public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { + return in.getSortedSet(field); + } + + @Override + public void checkIntegrity() throws IOException { + in.checkIntegrity(); + } + + @Override + public void close() throws IOException { + in.close(); + } + + @Override + public long ramBytesUsed() { + return in.ramBytesUsed(); + } + } + + private static class FilterStoredFieldsReader extends StoredFieldsReader { + + private final StoredFieldsReader fieldsReader; + + FilterStoredFieldsReader(StoredFieldsReader fieldsReader) { + this.fieldsReader = fieldsReader; + } + + @Override + public long ramBytesUsed() { + return fieldsReader.ramBytesUsed(); + } + + @Override + public void close() throws IOException { + fieldsReader.close(); + } + + @Override + public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException { + fieldsReader.visitDocument(docID, visitor); + } + + @Override + public StoredFieldsReader clone() { + return fieldsReader.clone(); + } + + @Override + public void checkIntegrity() throws IOException { + fieldsReader.checkIntegrity(); + } + } + + private static class FilterStoredFieldVisitor extends StoredFieldVisitor { + private final StoredFieldVisitor visitor; + + FilterStoredFieldVisitor(StoredFieldVisitor visitor) { + this.visitor = visitor; + } + + @Override + public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { + visitor.binaryField(fieldInfo, value); + } + + @Override + public void stringField(FieldInfo fieldInfo, byte[] value) throws IOException { + visitor.stringField(fieldInfo, value); + } + + @Override + public void intField(FieldInfo fieldInfo, int value) throws IOException { + visitor.intField(fieldInfo, value); + } + + @Override + public void longField(FieldInfo fieldInfo, long value) throws IOException { + visitor.longField(fieldInfo, value); + } + + @Override + public void floatField(FieldInfo fieldInfo, float value) throws IOException { + visitor.floatField(fieldInfo, value); + } + + @Override + public void doubleField(FieldInfo fieldInfo, double value) throws IOException { + visitor.doubleField(fieldInfo, value); + } + + @Override + public Status needsField(FieldInfo fieldInfo) throws IOException { + return visitor.needsField(fieldInfo); + } + } + } +} diff --git a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java new file mode 100644 index 00000000000..af2ded8c466 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java @@ -0,0 +1,120 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.search.Query; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.translog.Translog; + +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.LongSupplier; + +/** + * A policy that controls how many soft-deleted documents should be retained for peer-recovery and querying history changes purpose. + */ +final class SoftDeletesPolicy { + private final LongSupplier globalCheckpointSupplier; + private long localCheckpointOfSafeCommit; + // This lock count is used to prevent `minRetainedSeqNo` from advancing. + private int retentionLockCount; + // The extra number of operations before the global checkpoint are retained + private long retentionOperations; + // The min seq_no value that is retained - ops after this seq# should exist in the Lucene index. + private long minRetainedSeqNo; + + SoftDeletesPolicy(LongSupplier globalCheckpointSupplier, long minRetainedSeqNo, long retentionOperations) { + this.globalCheckpointSupplier = globalCheckpointSupplier; + this.retentionOperations = retentionOperations; + this.minRetainedSeqNo = minRetainedSeqNo; + this.localCheckpointOfSafeCommit = SequenceNumbers.NO_OPS_PERFORMED; + this.retentionLockCount = 0; + } + + /** + * Updates the number of soft-deleted documents prior to the global checkpoint to be retained + * See {@link org.elasticsearch.index.IndexSettings#INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING} + */ + synchronized void setRetentionOperations(long retentionOperations) { + this.retentionOperations = retentionOperations; + } + + /** + * Sets the local checkpoint of the current safe commit + */ + synchronized void setLocalCheckpointOfSafeCommit(long newCheckpoint) { + if (newCheckpoint < this.localCheckpointOfSafeCommit) { + throw new IllegalArgumentException("Local checkpoint can't go backwards; " + + "new checkpoint [" + newCheckpoint + "]," + "current checkpoint [" + localCheckpointOfSafeCommit + "]"); + } + this.localCheckpointOfSafeCommit = newCheckpoint; + } + + /** + * Acquires a lock on soft-deleted documents to prevent them from cleaning up in merge processes. This is necessary to + * make sure that all operations that are being retained will be retained until the lock is released. + * This is a analogy to the translog's retention lock; see {@link Translog#acquireRetentionLock()} + */ + synchronized Releasable acquireRetentionLock() { + assert retentionLockCount >= 0 : "Invalid number of retention locks [" + retentionLockCount + "]"; + retentionLockCount++; + final AtomicBoolean released = new AtomicBoolean(); + return () -> { + if (released.compareAndSet(false, true)) { + releaseRetentionLock(); + } + }; + } + + private synchronized void releaseRetentionLock() { + assert retentionLockCount > 0 : "Invalid number of retention locks [" + retentionLockCount + "]"; + retentionLockCount--; + } + + /** + * Returns the min seqno that is retained in the Lucene index. + * Operations whose seq# is least this value should exist in the Lucene index. + */ + synchronized long getMinRetainedSeqNo() { + // Do not advance if the retention lock is held + if (retentionLockCount == 0) { + // This policy retains operations for two purposes: peer-recovery and querying changes history. + // - Peer-recovery is driven by the local checkpoint of the safe commit. In peer-recovery, the primary transfers a safe commit, + // then sends ops after the local checkpoint of that commit. This requires keeping all ops after localCheckpointOfSafeCommit; + // - Changes APIs are driven the combination of the global checkpoint and retention ops. Here we prefer using the global + // checkpoint instead of max_seqno because only operations up to the global checkpoint are exposed in the the changes APIs. + final long minSeqNoForQueryingChanges = globalCheckpointSupplier.getAsLong() - retentionOperations; + final long minSeqNoToRetain = Math.min(minSeqNoForQueryingChanges, localCheckpointOfSafeCommit) + 1; + // This can go backward as the retentionOperations value can be changed in settings. + minRetainedSeqNo = Math.max(minRetainedSeqNo, minSeqNoToRetain); + } + return minRetainedSeqNo; + } + + /** + * Returns a soft-deletes retention query that will be used in {@link org.apache.lucene.index.SoftDeletesRetentionMergePolicy} + * Documents including tombstones are soft-deleted and matched this query will be retained and won't cleaned up by merges. + */ + Query getRetentionQuery() { + return LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, getMinRetainedSeqNo(), Long.MAX_VALUE); + } +} diff --git a/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java b/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java index 4c65635c61b..462f8ce8e68 100644 --- a/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java +++ b/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java @@ -54,13 +54,19 @@ public class FieldsVisitor extends StoredFieldVisitor { RoutingFieldMapper.NAME)); private final boolean loadSource; + private final String sourceFieldName; private final Set requiredFields; protected BytesReference source; protected String type, id; protected Map> fieldsValues; public FieldsVisitor(boolean loadSource) { + this(loadSource, SourceFieldMapper.NAME); + } + + public FieldsVisitor(boolean loadSource, String sourceFieldName) { this.loadSource = loadSource; + this.sourceFieldName = sourceFieldName; requiredFields = new HashSet<>(); reset(); } @@ -103,7 +109,7 @@ public class FieldsVisitor extends StoredFieldVisitor { @Override public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { - if (SourceFieldMapper.NAME.equals(fieldInfo.name)) { + if (sourceFieldName.equals(fieldInfo.name)) { source = new BytesArray(value); } else if (IdFieldMapper.NAME.equals(fieldInfo.name)) { id = Uid.decodeId(value); @@ -175,7 +181,7 @@ public class FieldsVisitor extends StoredFieldVisitor { requiredFields.addAll(BASE_REQUIRED_FIELDS); if (loadSource) { - requiredFields.add(SourceFieldMapper.NAME); + requiredFields.add(sourceFieldName); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index a0640ac68a9..663aa7e6f9e 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -19,11 +19,14 @@ package org.elasticsearch.index.mapper; +import org.apache.lucene.document.StoredField; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchGenerationException; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.text.Text; @@ -39,12 +42,15 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.stream.Stream; public class DocumentMapper implements ToXContentFragment { @@ -121,6 +127,8 @@ public class DocumentMapper implements ToXContentFragment { private final Map objectMappers; private final boolean hasNestedObjects; + private final MetadataFieldMapper[] deleteTombstoneMetadataFieldMappers; + private final MetadataFieldMapper[] noopTombstoneMetadataFieldMappers; public DocumentMapper(MapperService mapperService, Mapping mapping) { this.mapperService = mapperService; @@ -171,6 +179,15 @@ public class DocumentMapper implements ToXContentFragment { } catch (Exception e) { throw new ElasticsearchGenerationException("failed to serialize source for type [" + type + "]", e); } + + final Collection deleteTombstoneMetadataFields = Arrays.asList(VersionFieldMapper.NAME, IdFieldMapper.NAME, + TypeFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME); + this.deleteTombstoneMetadataFieldMappers = Stream.of(mapping.metadataMappers) + .filter(field -> deleteTombstoneMetadataFields.contains(field.name())).toArray(MetadataFieldMapper[]::new); + final Collection noopTombstoneMetadataFields = Arrays.asList( + VersionFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME); + this.noopTombstoneMetadataFieldMappers = Stream.of(mapping.metadataMappers) + .filter(field -> noopTombstoneMetadataFields.contains(field.name())).toArray(MetadataFieldMapper[]::new); } public Mapping mapping() { @@ -242,7 +259,22 @@ public class DocumentMapper implements ToXContentFragment { } public ParsedDocument parse(SourceToParse source) throws MapperParsingException { - return documentParser.parseDocument(source); + return documentParser.parseDocument(source, mapping.metadataMappers); + } + + public ParsedDocument createDeleteTombstoneDoc(String index, String type, String id) throws MapperParsingException { + final SourceToParse emptySource = SourceToParse.source(index, type, id, new BytesArray("{}"), XContentType.JSON); + return documentParser.parseDocument(emptySource, deleteTombstoneMetadataFieldMappers).toTombstone(); + } + + public ParsedDocument createNoopTombstoneDoc(String index, String reason) throws MapperParsingException { + final String id = ""; // _id won't be used. + final SourceToParse sourceToParse = SourceToParse.source(index, type, id, new BytesArray("{}"), XContentType.JSON); + final ParsedDocument parsedDoc = documentParser.parseDocument(sourceToParse, noopTombstoneMetadataFieldMappers).toTombstone(); + // Store the reason of a noop as a raw string in the _source field + final BytesRef byteRef = new BytesRef(reason); + parsedDoc.rootDoc().add(new StoredField(SourceFieldMapper.NAME, byteRef.bytes, byteRef.offset, byteRef.length)); + return parsedDoc; } /** diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java index 0fd156c0905..85123f602ed 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -55,7 +55,7 @@ final class DocumentParser { this.docMapper = docMapper; } - ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { + ParsedDocument parseDocument(SourceToParse source, MetadataFieldMapper[] metadataFieldsMappers) throws MapperParsingException { validateType(source); final Mapping mapping = docMapper.mapping(); @@ -64,9 +64,9 @@ final class DocumentParser { try (XContentParser parser = XContentHelper.createParser(docMapperParser.getXContentRegistry(), LoggingDeprecationHandler.INSTANCE, source.source(), xContentType)) { - context = new ParseContext.InternalParseContext(indexSettings.getSettings(), docMapperParser, docMapper, source, parser); + context = new ParseContext.InternalParseContext(indexSettings, docMapperParser, docMapper, source, parser); validateStart(parser); - internalParseDocument(mapping, context, parser); + internalParseDocument(mapping, metadataFieldsMappers, context, parser); validateEnd(parser); } catch (Exception e) { throw wrapInMapperParsingException(source, e); @@ -81,10 +81,11 @@ final class DocumentParser { return parsedDocument(source, context, createDynamicUpdate(mapping, docMapper, context.getDynamicMappers())); } - private static void internalParseDocument(Mapping mapping, ParseContext.InternalParseContext context, XContentParser parser) throws IOException { + private static void internalParseDocument(Mapping mapping, MetadataFieldMapper[] metadataFieldsMappers, + ParseContext.InternalParseContext context, XContentParser parser) throws IOException { final boolean emptyDoc = isEmptyDoc(mapping, parser); - for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { + for (MetadataFieldMapper metadataMapper : metadataFieldsMappers) { metadataMapper.preParse(context); } @@ -95,7 +96,7 @@ final class DocumentParser { parseObjectOrNested(context, mapping.root); } - for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { + for (MetadataFieldMapper metadataMapper : metadataFieldsMappers) { metadataMapper.postParse(context); } } @@ -495,7 +496,7 @@ final class DocumentParser { if (builder == null) { builder = new ObjectMapper.Builder(currentFieldName).enabled(true); } - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); objectMapper = builder.build(builderContext); context.addDynamicMapper(objectMapper); context.path().add(currentFieldName); @@ -538,7 +539,7 @@ final class DocumentParser { if (builder == null) { parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName); } else { - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); mapper = builder.build(builderContext); assert mapper != null; if (mapper instanceof ArrayValueMapperParser) { @@ -696,13 +697,13 @@ final class DocumentParser { if (parseableAsLong && context.root().numericDetection()) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG); if (builder == null) { - builder = newLongBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); + builder = newLongBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } else if (parseableAsDouble && context.root().numericDetection()) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DOUBLE); if (builder == null) { - builder = newFloatBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); + builder = newFloatBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } else if (parseableAsLong == false && parseableAsDouble == false && context.root().dateDetection()) { @@ -718,7 +719,7 @@ final class DocumentParser { } Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DATE); if (builder == null) { - builder = newDateBuilder(currentFieldName, dateTimeFormatter, Version.indexCreated(context.indexSettings())); + builder = newDateBuilder(currentFieldName, dateTimeFormatter, context.indexSettings().getIndexVersionCreated()); } if (builder instanceof DateFieldMapper.Builder) { DateFieldMapper.Builder dateBuilder = (DateFieldMapper.Builder) builder; @@ -741,7 +742,7 @@ final class DocumentParser { if (numberType == XContentParser.NumberType.INT || numberType == XContentParser.NumberType.LONG) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG); if (builder == null) { - builder = newLongBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); + builder = newLongBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } else if (numberType == XContentParser.NumberType.FLOAT || numberType == XContentParser.NumberType.DOUBLE) { @@ -750,7 +751,7 @@ final class DocumentParser { // no templates are defined, we use float by default instead of double // since this is much more space-efficient and should be enough most of // the time - builder = newFloatBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); + builder = newFloatBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } @@ -785,7 +786,7 @@ final class DocumentParser { return; } final String path = context.path().pathAsText(currentFieldName); - final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); final MappedFieldType existingFieldType = context.mapperService().fullName(path); final Mapper.Builder builder; if (existingFieldType != null) { @@ -883,8 +884,8 @@ final class DocumentParser { if (builder == null) { builder = new ObjectMapper.Builder(paths[i]).enabled(true); } - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); - mapper = (ObjectMapper) builder.build(builderContext); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), + context.path()); mapper = (ObjectMapper) builder.build(builderContext); if (mapper.nested() != ObjectMapper.Nested.NO) { throw new MapperParsingException("It is forbidden to create dynamic nested objects ([" + context.path().pathAsText(paths[i]) + "]) through `copy_to` or dots in field names"); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java index 606777392de..8389a306270 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java @@ -24,7 +24,6 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; import org.apache.lucene.search.Query; import org.elasticsearch.Version; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.lucene.Lucene; @@ -205,12 +204,12 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper { } @Override - public void preParse(ParseContext context) throws IOException { + public void preParse(ParseContext context) { } @Override public void postParse(ParseContext context) throws IOException { - if (context.indexSettings().getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).before(Version.V_6_1_0)) { + if (context.indexSettings().getIndexVersionCreated().before(Version.V_6_1_0)) { super.parse(context); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java index b77ffee05ca..cf8cc4022fd 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -24,9 +24,8 @@ import com.carrotsearch.hppc.ObjectObjectMap; import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexableField; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.IndexSettings; import java.util.ArrayList; import java.util.Collection; @@ -196,7 +195,7 @@ public abstract class ParseContext implements Iterable{ } @Override - public Settings indexSettings() { + public IndexSettings indexSettings() { return in.indexSettings(); } @@ -315,8 +314,7 @@ public abstract class ParseContext implements Iterable{ private final List documents; - @Nullable - private final Settings indexSettings; + private final IndexSettings indexSettings; private final SourceToParse sourceToParse; @@ -334,8 +332,8 @@ public abstract class ParseContext implements Iterable{ private final Set ignoredFields = new HashSet<>(); - public InternalParseContext(@Nullable Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, - SourceToParse source, XContentParser parser) { + public InternalParseContext(IndexSettings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, + SourceToParse source, XContentParser parser) { this.indexSettings = indexSettings; this.docMapper = docMapper; this.docMapperParser = docMapperParser; @@ -347,7 +345,7 @@ public abstract class ParseContext implements Iterable{ this.version = null; this.sourceToParse = source; this.dynamicMappers = new ArrayList<>(); - this.maxAllowedNumNestedDocs = MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING.get(indexSettings); + this.maxAllowedNumNestedDocs = indexSettings.getValue(MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING); this.numNestedDocs = 0L; } @@ -357,8 +355,7 @@ public abstract class ParseContext implements Iterable{ } @Override - @Nullable - public Settings indexSettings() { + public IndexSettings indexSettings() { return this.indexSettings; } @@ -565,8 +562,7 @@ public abstract class ParseContext implements Iterable{ return false; } - @Nullable - public abstract Settings indexSettings(); + public abstract IndexSettings indexSettings(); public abstract SourceToParse sourceToParse(); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java b/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java index 414cb3a98ec..d2cf17ddd35 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java @@ -83,6 +83,17 @@ public class ParsedDocument { this.seqID.primaryTerm.setLongValue(primaryTerm); } + /** + * Makes the processing document as a tombstone document rather than a regular document. + * Tombstone documents are stored in Lucene index to represent delete operations or Noops. + */ + ParsedDocument toTombstone() { + assert docs().size() == 1 : "Tombstone should have a single doc [" + docs() + "]"; + this.seqID.tombstoneField.setLongValue(1); + rootDoc().add(this.seqID.tombstoneField); + return this; + } + public String routing() { return this.routing; } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java index ac3ffe46272..5a0db4163bf 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java @@ -69,26 +69,29 @@ public class SeqNoFieldMapper extends MetadataFieldMapper { public final Field seqNo; public final Field seqNoDocValue; public final Field primaryTerm; + public final Field tombstoneField; - public SequenceIDFields(Field seqNo, Field seqNoDocValue, Field primaryTerm) { + public SequenceIDFields(Field seqNo, Field seqNoDocValue, Field primaryTerm, Field tombstoneField) { Objects.requireNonNull(seqNo, "sequence number field cannot be null"); Objects.requireNonNull(seqNoDocValue, "sequence number dv field cannot be null"); Objects.requireNonNull(primaryTerm, "primary term field cannot be null"); this.seqNo = seqNo; this.seqNoDocValue = seqNoDocValue; this.primaryTerm = primaryTerm; + this.tombstoneField = tombstoneField; } public static SequenceIDFields emptySeqID() { return new SequenceIDFields(new LongPoint(NAME, SequenceNumbers.UNASSIGNED_SEQ_NO), new NumericDocValuesField(NAME, SequenceNumbers.UNASSIGNED_SEQ_NO), - new NumericDocValuesField(PRIMARY_TERM_NAME, 0)); + new NumericDocValuesField(PRIMARY_TERM_NAME, 0), new NumericDocValuesField(TOMBSTONE_NAME, 0)); } } public static final String NAME = "_seq_no"; public static final String CONTENT_TYPE = "_seq_no"; public static final String PRIMARY_TERM_NAME = "_primary_term"; + public static final String TOMBSTONE_NAME = "_tombstone"; public static class SeqNoDefaults { public static final String NAME = SeqNoFieldMapper.NAME; 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 f2090613c09..7bfe793bba4 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.mapper; +import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.StoredField; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; @@ -49,6 +50,7 @@ import java.util.function.Function; public class SourceFieldMapper extends MetadataFieldMapper { public static final String NAME = "_source"; + public static final String RECOVERY_SOURCE_NAME = "_recovery_source"; public static final String CONTENT_TYPE = "_source"; private final Function, Map> filter; @@ -224,7 +226,8 @@ public class SourceFieldMapper extends MetadataFieldMapper { @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { - BytesReference source = context.sourceToParse().source(); + BytesReference originalSource = context.sourceToParse().source(); + BytesReference source = originalSource; if (enabled && fieldType().stored() && source != null) { // Percolate and tv APIs may not set the source and that is ok, because these APIs will not index any data if (filter != null) { @@ -240,8 +243,17 @@ public class SourceFieldMapper extends MetadataFieldMapper { } BytesRef ref = source.toBytesRef(); fields.add(new StoredField(fieldType().name(), ref.bytes, ref.offset, ref.length)); + } else { + source = null; } - } + + if (originalSource != null && source != originalSource && context.indexSettings().isSoftDeleteEnabled()) { + // 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(); + fields.add(new StoredField(RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); + fields.add(new NumericDocValuesField(RECOVERY_SOURCE_NAME, 1)); + } + } @Override protected String contentType() { 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 e030c95b56e..ef5f9ab0ef3 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -92,12 +92,14 @@ import org.elasticsearch.index.fielddata.ShardFieldData; import org.elasticsearch.index.flush.FlushStats; import org.elasticsearch.index.get.GetStats; import org.elasticsearch.index.get.ShardGetService; +import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperForType; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; @@ -1620,25 +1622,33 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl } /** - * Acquires a lock on the translog files, preventing them from being trimmed. + * Acquires a lock on the translog files and Lucene soft-deleted documents to prevent them from being trimmed */ - public Closeable acquireTranslogRetentionLock() { - return getEngine().acquireTranslogRetentionLock(); + public Closeable acquireRetentionLockForPeerRecovery() { + return getEngine().acquireRetentionLockForPeerRecovery(); } /** - * Creates a new translog snapshot for reading translog operations whose seq# at least the provided seq#. - * The caller has to close the returned snapshot after finishing the reading. + * Returns the estimated number of history operations whose seq# at least the provided seq# in this shard. */ - public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { - return getEngine().newTranslogSnapshotFromMinSeqNo(minSeqNo); + public int estimateNumberOfHistoryOperations(String source, long startingSeqNo) throws IOException { + return getEngine().estimateNumberOfHistoryOperations(source, mapperService, startingSeqNo); } /** - * Returns the estimated number of operations in translog whose seq# at least the provided seq#. + * Creates a new history snapshot for reading operations since the provided starting seqno (inclusive). + * The returned snapshot can be retrieved from either Lucene index or translog files. */ - public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { - return getEngine().estimateTranslogOperationsFromMinSeq(minSeqNo); + public Translog.Snapshot getHistoryOperations(String source, long startingSeqNo) throws IOException { + return getEngine().readHistoryOperations(source, mapperService, startingSeqNo); + } + + /** + * Checks if we have a completed history of operations since the given starting seqno (inclusive). + * This method should be called after acquiring the retention lock; See {@link #acquireRetentionLockForPeerRecovery()} + */ + public boolean hasCompleteHistoryOperations(String source, long startingSeqNo) throws IOException { + return getEngine().hasCompleteOperationHistory(source, mapperService, startingSeqNo); } public List segments(boolean verbose) { @@ -2209,7 +2219,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), - indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, () -> operationPrimaryTerm); + indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, () -> operationPrimaryTerm, tombstoneDocSupplier()); } /** @@ -2648,4 +2658,19 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl refreshMetric.inc(System.nanoTime() - currentRefreshStartTime); } } + + private EngineConfig.TombstoneDocSupplier tombstoneDocSupplier() { + final RootObjectMapper.Builder noopRootMapper = new RootObjectMapper.Builder("__noop"); + final DocumentMapper noopDocumentMapper = new DocumentMapper.Builder(noopRootMapper, mapperService).build(mapperService); + return new EngineConfig.TombstoneDocSupplier() { + @Override + public ParsedDocument newDeleteTombstoneDoc(String type, String id) { + return docMapper(type).getDocumentMapper().createDeleteTombstoneDoc(shardId.getIndexName(), type, id); + } + @Override + public ParsedDocument newNoopTombstoneDoc(String reason) { + return noopDocumentMapper.createNoopTombstoneDoc(shardId.getIndexName(), reason); + } + }; + } } 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 1edc0eb5dca..016a8afff69 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java +++ b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java @@ -89,7 +89,7 @@ public class PrimaryReplicaSyncer extends AbstractComponent { // 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.newTranslogSnapshotFromMinSeqNo(startingSeqNo); + snapshot = indexShard.getHistoryOperations("resync", startingSeqNo); final Translog.Snapshot originalSnapshot = snapshot; final Translog.Snapshot wrappedSnapshot = new Translog.Snapshot() { @Override diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index e9acfe3d8b0..ae3f90e63e7 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -156,6 +156,7 @@ final class StoreRecovery { final Directory hardLinkOrCopyTarget = new org.apache.lucene.store.HardlinkCopyDirectoryWrapper(target); IndexWriterConfig iwc = new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) // we don't want merges to happen here - we call maybe merge on the engine // later once we stared it up otherwise we would need to wait for it here diff --git a/server/src/main/java/org/elasticsearch/index/store/Store.java b/server/src/main/java/org/elasticsearch/index/store/Store.java index 001e263ea8f..85975bc68c8 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -1009,7 +1009,6 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref } final String segmentId = IndexFileNames.parseSegmentName(meta.name()); final String extension = IndexFileNames.getExtension(meta.name()); - assert FIELD_INFOS_FILE_EXTENSION.equals(extension) == false || IndexFileNames.stripExtension(IndexFileNames.stripSegmentName(meta.name())).isEmpty() : "FieldInfos are generational but updateable DV are not supported in elasticsearch"; if (IndexFileNames.SEGMENTS.equals(segmentId) || DEL_FILE_EXTENSION.equals(extension) || LIV_FILE_EXTENSION.equals(extension)) { // only treat del files as per-commit files fnm files are generational but only for upgradable DV perCommitStoreFiles.add(meta); @@ -1595,6 +1594,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref throws IOException { assert openMode == IndexWriterConfig.OpenMode.APPEND || commit == null : "can't specify create flag with a commit"; IndexWriterConfig iwc = new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) .setIndexCommit(commit) // we don't want merges to happen here - we call maybe merge on the engine diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index 618aa546e42..f17acac3789 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -1261,6 +1261,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC ", type='" + type + '\'' + ", seqNo=" + seqNo + ", primaryTerm=" + primaryTerm + + ", version=" + version + + ", autoGeneratedIdTimestamp=" + autoGeneratedIdTimestamp + '}'; } @@ -1403,6 +1405,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC "uid=" + uid + ", seqNo=" + seqNo + ", primaryTerm=" + primaryTerm + + ", version=" + version + '}'; } } diff --git a/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java b/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java index f48f2ceb792..e0cfe9eaaff 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -40,6 +40,7 @@ import java.nio.file.Path; import java.nio.file.StandardOpenOption; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.LongSupplier; @@ -192,7 +193,24 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable { new BufferedChecksumStreamInput(data.streamInput(), "assertion")); Translog.Operation prvOp = Translog.readOperation( new BufferedChecksumStreamInput(previous.v1().streamInput(), "assertion")); - if (newOp.equals(prvOp) == false) { + // TODO: We haven't had timestamp for Index operations in Lucene yet, we need to loosen this check without timestamp. + final boolean sameOp; + if (newOp instanceof Translog.Index && prvOp instanceof Translog.Index) { + final Translog.Index o1 = (Translog.Index) prvOp; + final Translog.Index o2 = (Translog.Index) newOp; + sameOp = Objects.equals(o1.id(), o2.id()) && Objects.equals(o1.type(), o2.type()) + && Objects.equals(o1.source(), o2.source()) && Objects.equals(o1.routing(), o2.routing()) + && o1.primaryTerm() == o2.primaryTerm() && o1.seqNo() == o2.seqNo() + && o1.version() == o2.version(); + } else if (newOp instanceof Translog.Delete && prvOp instanceof Translog.Delete) { + final Translog.Delete o1 = (Translog.Delete) newOp; + final Translog.Delete o2 = (Translog.Delete) prvOp; + sameOp = Objects.equals(o1.id(), o2.id()) && Objects.equals(o1.type(), o2.type()) + && o1.primaryTerm() == o2.primaryTerm() && o1.seqNo() == o2.seqNo() && o1.version() == o2.version(); + } else { + sameOp = false; + } + if (sameOp == false) { throw new AssertionError( "seqNo [" + seqNo + "] was processed twice in generation [" + generation + "], with different data. " + "prvOp [" + prvOp + "], newOp [" + newOp + "]", previous.v2()); diff --git a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java index 86995ae7c5a..a90f8af0af4 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java @@ -32,6 +32,7 @@ import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.Lock; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.store.NativeFSLockFactory; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cli.EnvironmentAwareCommand; @@ -177,6 +178,7 @@ public class TruncateTranslogCommand extends EnvironmentAwareCommand { terminal.println("Marking index with the new history uuid"); // commit the new histroy id IndexWriterConfig iwc = new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) // we don't want merges to happen here - we call maybe merge on the engine // later once we stared it up otherwise we would need to wait for it here 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 352f07d5764..10f796e5e15 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -146,11 +146,11 @@ public class RecoverySourceHandler { assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; }, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger); - try (Closeable ignored = shard.acquireTranslogRetentionLock()) { + try (Closeable ignored = shard.acquireRetentionLockForPeerRecovery()) { final long startingSeqNo; final long requiredSeqNoRangeStart; final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && - isTargetSameHistory() && isTranslogReadyForSequenceNumberBasedRecovery(); + isTargetSameHistory() && shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo()); if (isSequenceNumberBasedRecovery) { logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); startingSeqNo = request.startingSeqNo(); @@ -162,14 +162,16 @@ public class RecoverySourceHandler { } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e); } - // we set this to 0 to create a translog roughly according to the retention policy - // on the target. Note that it will still filter out legacy operations with no sequence numbers - startingSeqNo = 0; - // but we must have everything above the local checkpoint in the commit + // We must have everything above the local checkpoint in the commit requiredSeqNoRangeStart = Long.parseLong(phase1Snapshot.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1; + // If soft-deletes enabled, we need to transfer only operations after the local_checkpoint of the commit to have + // the same history on the target. However, with translog, we need to set this to 0 to create a translog roughly + // according to the retention policy on the target. Note that it will still filter out legacy operations without seqNo. + startingSeqNo = shard.indexSettings().isSoftDeleteEnabled() ? requiredSeqNoRangeStart : 0; try { - phase1(phase1Snapshot.getIndexCommit(), () -> shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); + final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo); + phase1(phase1Snapshot.getIndexCommit(), () -> estimateNumOps); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e); } finally { @@ -186,7 +188,8 @@ public class RecoverySourceHandler { try { // For a sequence based recovery, the target can keep its local translog - prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); + prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, + shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); } @@ -207,11 +210,13 @@ public class RecoverySourceHandler { */ cancellableThreads.execute(() -> shard.waitForOpsToComplete(endingSeqNo)); - logger.trace("all operations up to [{}] completed, which will be used as an ending sequence number", endingSeqNo); - - logger.trace("snapshot translog for recovery; current size is [{}]", shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); + if (logger.isTraceEnabled()) { + logger.trace("all operations up to [{}] completed, which will be used as an ending sequence number", endingSeqNo); + logger.trace("snapshot translog for recovery; current size is [{}]", + shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); + } final long targetLocalCheckpoint; - try(Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) { + try (Translog.Snapshot snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo)) { targetLocalCheckpoint = phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot); } catch (Exception e) { throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); @@ -268,36 +273,6 @@ public class RecoverySourceHandler { }); } - /** - * Determines if the source translog is ready for a sequence-number-based peer recovery. The main condition here is that the source - * translog contains all operations above the local checkpoint on the target. We already know the that translog contains or will contain - * all ops above the source local checkpoint, so we can stop check there. - * - * @return {@code true} if the source is ready for a sequence-number-based recovery - * @throws IOException if an I/O exception occurred reading the translog snapshot - */ - boolean isTranslogReadyForSequenceNumberBasedRecovery() throws IOException { - final long startingSeqNo = request.startingSeqNo(); - assert startingSeqNo >= 0; - final long localCheckpoint = shard.getLocalCheckpoint(); - logger.trace("testing sequence numbers in range: [{}, {}]", startingSeqNo, localCheckpoint); - // the start recovery request is initialized with the starting sequence number set to the target shard's local checkpoint plus one - if (startingSeqNo - 1 <= localCheckpoint) { - final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1); - try (Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) { - Translog.Operation operation; - while ((operation = snapshot.next()) != null) { - if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - tracker.markSeqNoAsCompleted(operation.seqNo()); - } - } - } - return tracker.getCheckpoint() >= localCheckpoint; - } else { - return false; - } - } - /** * Perform phase1 of the recovery operations. Once this {@link IndexCommit} * snapshot has been performed no commit operations (files being fsync'd) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index a4d6518e9af..9469f657c96 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1492,6 +1492,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp // empty shard would cause exceptions to be thrown. Since there is no data to restore from an empty // shard anyway, we just create the empty shard here and then exit. IndexWriter writer = new IndexWriter(store.directory(), new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setOpenMode(IndexWriterConfig.OpenMode.CREATE) .setCommitOnClose(true)); writer.close(); diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 702d63d0d94..6acdbad2cce 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -64,6 +64,7 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.repositories.IndexId; @@ -120,7 +121,8 @@ public class RestoreService extends AbstractComponent implements ClusterStateApp SETTING_NUMBER_OF_SHARDS, SETTING_VERSION_CREATED, SETTING_INDEX_UUID, - SETTING_CREATION_DATE)); + SETTING_CREATION_DATE, + IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey())); // It's OK to change some settings, but we shouldn't allow simply removing them private static final Set UNREMOVABLE_SETTINGS; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 90173455c3b..9786c0eaf52 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -392,6 +392,7 @@ public class PrimaryAllocationIT extends ESIntegTestCase { assertThat(shard.getLocalCheckpoint(), equalTo(numDocs + moreDocs)); } }, 30, TimeUnit.SECONDS); + internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); } } diff --git a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java index 753aedea01e..890f6ef163b 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java @@ -33,18 +33,23 @@ import org.apache.lucene.index.NoDeletionPolicy; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.Weight; import org.apache.lucene.store.Directory; import org.apache.lucene.store.MMapDirectory; import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.util.Bits; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.test.ESTestCase; import java.io.IOException; +import java.io.StringReader; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -53,6 +58,8 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import static org.hamcrest.Matchers.equalTo; + public class LuceneTests extends ESTestCase { public void testWaitForIndex() throws Exception { final MockDirectoryWrapper dir = newMockDirectory(); @@ -406,4 +413,88 @@ public class LuceneTests extends ESTestCase { // add assume's here if needed for certain platforms, but we should know if it does not work. assertTrue("MMapDirectory does not support unmapping: " + MMapDirectory.UNMAP_NOT_SUPPORTED_REASON, MMapDirectory.UNMAP_SUPPORTED); } + + public void testWrapAllDocsLive() throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) + .setMergePolicy(new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, MatchAllDocsQuery::new, newMergePolicy())); + IndexWriter writer = new IndexWriter(dir, config); + int numDocs = between(1, 10); + Set liveDocs = new HashSet<>(); + for (int i = 0; i < numDocs; i++) { + String id = Integer.toString(i); + Document doc = new Document(); + doc.add(new StringField("id", id, Store.YES)); + writer.addDocument(doc); + liveDocs.add(id); + } + for (int i = 0; i < numDocs; i++) { + if (randomBoolean()) { + String id = Integer.toString(i); + Document doc = new Document(); + doc.add(new StringField("id", "v2-" + id, Store.YES)); + if (randomBoolean()) { + doc.add(Lucene.newSoftDeletesField()); + } + writer.softUpdateDocument(new Term("id", id), doc, Lucene.newSoftDeletesField()); + liveDocs.add("v2-" + id); + } + } + try (DirectoryReader unwrapped = DirectoryReader.open(writer)) { + DirectoryReader reader = Lucene.wrapAllDocsLive(unwrapped); + assertThat(reader.numDocs(), equalTo(liveDocs.size())); + IndexSearcher searcher = new IndexSearcher(reader); + Set actualDocs = new HashSet<>(); + TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), Integer.MAX_VALUE); + for (ScoreDoc scoreDoc : topDocs.scoreDocs) { + actualDocs.add(reader.document(scoreDoc.doc).get("id")); + } + assertThat(actualDocs, equalTo(liveDocs)); + } + IOUtils.close(writer, dir); + } + + public void testWrapLiveDocsNotExposeAbortedDocuments() throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) + .setMergePolicy(new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, MatchAllDocsQuery::new, newMergePolicy())); + IndexWriter writer = new IndexWriter(dir, config); + int numDocs = between(1, 10); + List liveDocs = new ArrayList<>(); + for (int i = 0; i < numDocs; i++) { + String id = Integer.toString(i); + Document doc = new Document(); + doc.add(new StringField("id", id, Store.YES)); + if (randomBoolean()) { + doc.add(Lucene.newSoftDeletesField()); + } + writer.addDocument(doc); + liveDocs.add(id); + } + int abortedDocs = between(1, 10); + for (int i = 0; i < abortedDocs; i++) { + try { + Document doc = new Document(); + doc.add(new StringField("id", "aborted-" + i, Store.YES)); + StringReader reader = new StringReader(""); + doc.add(new TextField("other", reader)); + reader.close(); // mark the indexing hit non-aborting error + writer.addDocument(doc); + fail("index should have failed"); + } catch (Exception ignored) { } + } + try (DirectoryReader unwrapped = DirectoryReader.open(writer)) { + DirectoryReader reader = Lucene.wrapAllDocsLive(unwrapped); + assertThat(reader.maxDoc(), equalTo(numDocs + abortedDocs)); + assertThat(reader.numDocs(), equalTo(liveDocs.size())); + IndexSearcher searcher = new IndexSearcher(reader); + List actualDocs = new ArrayList<>(); + TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), Integer.MAX_VALUE); + for (ScoreDoc scoreDoc : topDocs.scoreDocs) { + actualDocs.add(reader.document(scoreDoc.doc).get("id")); + } + assertThat(actualDocs, equalTo(liveDocs)); + } + IOUtils.close(writer, dir); + } } diff --git a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java index 6bdd8ea3f2e..ac2f2b0d4f3 100644 --- a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java +++ b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java @@ -109,6 +109,7 @@ public abstract class AbstractDisruptionTestCase extends ESIntegTestCase { protected void beforeIndexDeletion() throws Exception { if (disableBeforeIndexDeletion == false) { super.beforeIndexDeletion(); + internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); assertSeqNos(); } } diff --git a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index d098c4918a7..b0b6c35f92a 100644 --- a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -40,6 +40,7 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; +import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -397,7 +398,8 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { .get(); logger.info("--> indexing docs"); - for (int i = 0; i < randomIntBetween(1, 1024); i++) { + int numDocs = randomIntBetween(1, 1024); + for (int i = 0; i < numDocs; i++) { client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); } @@ -419,12 +421,15 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { } logger.info("--> restart replica node"); + boolean softDeleteEnabled = internalCluster().getInstance(IndicesService.class, primaryNode) + .indexServiceSafe(resolveIndex("test")).getShard(0).indexSettings().isSoftDeleteEnabled(); + int moreDocs = randomIntBetween(1, 1024); internalCluster().restartNode(replicaNode, new RestartCallback() { @Override public Settings onNodeStopped(String nodeName) throws Exception { // index some more documents; we expect to reuse the files that already exist on the replica - for (int i = 0; i < randomIntBetween(1, 1024); i++) { + for (int i = 0; i < moreDocs; i++) { client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); } @@ -432,8 +437,12 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { client(primaryNode).admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder() .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) ).get(); client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get(); + if (softDeleteEnabled) { // We need an extra flush to advance the min_retained_seqno of the SoftDeletesPolicy + client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get(); + } return super.onNodeStopped(nodeName); } }); diff --git a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java index 28fa440d96a..b0b4ec3930a 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -306,7 +307,7 @@ public class IndexServiceTests extends ESSingleNodeTestCase { .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), -1)) .get(); IndexShard shard = indexService.getShard(0); - assertBusy(() -> assertThat(shard.estimateTranslogOperationsFromMinSeq(0L), equalTo(0))); + assertBusy(() -> assertThat(IndexShardTestCase.getTranslog(shard).totalOperations(), equalTo(0))); } public void testIllegalFsyncInterval() { diff --git a/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java b/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java index b7da5add2ac..64a2fa69bcb 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java @@ -553,4 +553,12 @@ public class IndexSettingsTests extends ESTestCase { ); assertThat(index.getDefaultFields(), equalTo(Arrays.asList("body", "title"))); } + + public void testUpdateSoftDeletesFails() { + IndexScopedSettings settings = new IndexScopedSettings(Settings.EMPTY, IndexScopedSettings.BUILT_IN_INDEX_SETTINGS); + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, () -> + settings.updateSettings(Settings.builder().put("index.soft_deletes.enabled", randomBoolean()).build(), + Settings.builder(), Settings.builder(), "index")); + assertThat(error.getMessage(), equalTo("final index setting [index.soft_deletes.enabled], not updateable")); + } } diff --git a/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index ea7de50b7b3..3f9fc9a0429 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -51,20 +51,24 @@ public class CombinedDeletionPolicyTests extends ESTestCase { public void testKeepCommitsAfterGlobalCheckpoint() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); + final int extraRetainedOps = between(0, 100); + final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, extraRetainedOps); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); final LongArrayList maxSeqNoList = new LongArrayList(); final LongArrayList translogGenList = new LongArrayList(); final List commitList = new ArrayList<>(); int totalCommits = between(2, 20); long lastMaxSeqNo = 0; + long lastCheckpoint = lastMaxSeqNo; long lastTranslogGen = 0; final UUID translogUUID = UUID.randomUUID(); for (int i = 0; i < totalCommits; i++) { lastMaxSeqNo += between(1, 10000); + lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); lastTranslogGen += between(1, 100); - commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); maxSeqNoList.add(lastMaxSeqNo); translogGenList.add(lastTranslogGen); } @@ -85,14 +89,19 @@ public class CombinedDeletionPolicyTests extends ESTestCase { } assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(translogGenList.get(keptIndex))); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), + equalTo(Math.min(getLocalCheckpoint(commitList.get(keptIndex)) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } public void testAcquireIndexCommit() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); + final int extraRetainedOps = between(0, 100); + final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, extraRetainedOps); final UUID translogUUID = UUID.randomUUID(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); long lastMaxSeqNo = between(1, 1000); + long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); long lastTranslogGen = between(1, 20); int safeIndex = 0; List commitList = new ArrayList<>(); @@ -102,8 +111,9 @@ public class CombinedDeletionPolicyTests extends ESTestCase { int newCommits = between(1, 10); for (int n = 0; n < newCommits; n++) { lastMaxSeqNo += between(1, 1000); + lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); lastTranslogGen += between(1, 20); - commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); } // Advance the global checkpoint to between [safeIndex, safeIndex + 1) safeIndex = randomIntBetween(safeIndex, commitList.size() - 1); @@ -114,6 +124,9 @@ public class CombinedDeletionPolicyTests extends ESTestCase { globalCheckpoint.set(randomLongBetween(lower, upper)); commitList.forEach(this::resetDeletion); indexPolicy.onCommit(commitList); + IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), + equalTo(Math.min(getLocalCheckpoint(safeCommit) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); // Captures and releases some commits int captures = between(0, 5); for (int n = 0; n < captures; n++) { @@ -132,7 +145,7 @@ public class CombinedDeletionPolicyTests extends ESTestCase { snapshottingCommits.remove(snapshot); final long pendingSnapshots = snapshottingCommits.stream().filter(snapshot::equals).count(); final IndexCommit lastCommit = commitList.get(commitList.size() - 1); - final IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); + safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); assertThat(indexPolicy.releaseCommit(snapshot), equalTo(pendingSnapshots == 0 && snapshot.equals(lastCommit) == false && snapshot.equals(safeCommit) == false)); } @@ -143,6 +156,8 @@ public class CombinedDeletionPolicyTests extends ESTestCase { equalTo(Long.parseLong(commitList.get(safeIndex).getUserData().get(Translog.TRANSLOG_GENERATION_KEY)))); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(Long.parseLong(commitList.get(commitList.size() - 1).getUserData().get(Translog.TRANSLOG_GENERATION_KEY)))); + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), + equalTo(Math.min(getLocalCheckpoint(commitList.get(safeIndex)) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } snapshottingCommits.forEach(indexPolicy::releaseCommit); globalCheckpoint.set(randomLongBetween(lastMaxSeqNo, Long.MAX_VALUE)); @@ -154,25 +169,27 @@ public class CombinedDeletionPolicyTests extends ESTestCase { assertThat(commitList.get(commitList.size() - 1).isDeleted(), equalTo(false)); assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(lastTranslogGen)); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); + IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), + equalTo(Math.min(getLocalCheckpoint(safeCommit) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } public void testLegacyIndex() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); + final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); final UUID translogUUID = UUID.randomUUID(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); long legacyTranslogGen = randomNonNegativeLong(); IndexCommit legacyCommit = mockLegacyIndexCommit(translogUUID, legacyTranslogGen); - indexPolicy.onCommit(singletonList(legacyCommit)); - verify(legacyCommit, never()).delete(); - assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(legacyTranslogGen)); - assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(legacyTranslogGen)); + assertThat(CombinedDeletionPolicy.findSafeCommitPoint(singletonList(legacyCommit), globalCheckpoint.get()), + equalTo(legacyCommit)); long safeTranslogGen = randomLongBetween(legacyTranslogGen, Long.MAX_VALUE); long maxSeqNo = randomLongBetween(1, Long.MAX_VALUE); - final IndexCommit freshCommit = mockIndexCommit(maxSeqNo, translogUUID, safeTranslogGen); + final IndexCommit freshCommit = mockIndexCommit(randomLongBetween(-1, maxSeqNo), maxSeqNo, translogUUID, safeTranslogGen); globalCheckpoint.set(randomLongBetween(0, maxSeqNo - 1)); indexPolicy.onCommit(Arrays.asList(legacyCommit, freshCommit)); @@ -189,25 +206,32 @@ public class CombinedDeletionPolicyTests extends ESTestCase { verify(freshCommit, times(0)).delete(); assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(safeTranslogGen)); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(safeTranslogGen)); + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), equalTo(getLocalCheckpoint(freshCommit) + 1)); } public void testDeleteInvalidCommits() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(randomNonNegativeLong()); + final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); final int invalidCommits = between(1, 10); final List commitList = new ArrayList<>(); for (int i = 0; i < invalidCommits; i++) { - commitList.add(mockIndexCommit(randomNonNegativeLong(), UUID.randomUUID(), randomNonNegativeLong())); + long maxSeqNo = randomNonNegativeLong(); + commitList.add(mockIndexCommit(randomLongBetween(-1, maxSeqNo), maxSeqNo, UUID.randomUUID(), randomNonNegativeLong())); } final UUID expectedTranslogUUID = UUID.randomUUID(); long lastTranslogGen = 0; final int validCommits = between(1, 10); + long lastMaxSeqNo = between(1, 1000); + long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); for (int i = 0; i < validCommits; i++) { lastTranslogGen += between(1, 1000); - commitList.add(mockIndexCommit(randomNonNegativeLong(), expectedTranslogUUID, lastTranslogGen)); + lastMaxSeqNo += between(1, 1000); + lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, expectedTranslogUUID, lastTranslogGen)); } // We should never keep invalid commits regardless of the value of the global checkpoint. @@ -215,21 +239,26 @@ public class CombinedDeletionPolicyTests extends ESTestCase { for (int i = 0; i < invalidCommits - 1; i++) { verify(commitList.get(i), times(1)).delete(); } + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), + equalTo(getLocalCheckpoint(CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get())) + 1)); } public void testCheckUnreferencedCommits() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); final UUID translogUUID = UUID.randomUUID(); final TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); final List commitList = new ArrayList<>(); int totalCommits = between(2, 20); long lastMaxSeqNo = between(1, 1000); + long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); long lastTranslogGen = between(1, 50); for (int i = 0; i < totalCommits; i++) { lastMaxSeqNo += between(1, 10000); lastTranslogGen += between(1, 100); - commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); + lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); } IndexCommit safeCommit = randomFrom(commitList); globalCheckpoint.set(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO))); @@ -256,8 +285,9 @@ public class CombinedDeletionPolicyTests extends ESTestCase { } } - IndexCommit mockIndexCommit(long maxSeqNo, UUID translogUUID, long translogGen) throws IOException { + IndexCommit mockIndexCommit(long localCheckpoint, long maxSeqNo, UUID translogUUID, long translogGen) throws IOException { final Map userData = new HashMap<>(); + userData.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(localCheckpoint)); userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); userData.put(Translog.TRANSLOG_UUID_KEY, translogUUID.toString()); userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(translogGen)); @@ -278,6 +308,10 @@ public class CombinedDeletionPolicyTests extends ESTestCase { }).when(commit).delete(); } + private long getLocalCheckpoint(IndexCommit commit) throws IOException { + return Long.parseLong(commit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); + } + IndexCommit mockLegacyIndexCommit(UUID translogUUID, long translogGen) throws IOException { final Map userData = new HashMap<>(); userData.put(Translog.TRANSLOG_UUID_KEY, translogUUID.toString()); @@ -287,4 +321,5 @@ public class CombinedDeletionPolicyTests extends ESTestCase { resetDeletion(commit); return commit; } + } 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 76e05ba1e0b..d3aead9e44e 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.engine; +import java.io.Closeable; import java.io.IOException; import java.io.UncheckedIOException; import java.nio.charset.Charset; @@ -77,10 +78,12 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.LogByteSizeMergePolicy; import org.apache.lucene.index.LogDocMergePolicy; +import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PointValues; import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.index.TieredMergePolicy; import org.apache.lucene.search.IndexSearcher; @@ -114,6 +117,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; @@ -133,6 +137,7 @@ import org.elasticsearch.index.fieldvisitor.FieldsVisitor; import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.Mapper.BuilderContext; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.ParseContext; @@ -172,8 +177,10 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.isIn; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @@ -247,8 +254,13 @@ public class InternalEngineTests extends EngineTestCase { } public void testSegments() throws Exception { + Settings settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings( + IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); try (Store store = createStore(); - InternalEngine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { + InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null))) { List segments = engine.segments(false); assertThat(segments.isEmpty(), equalTo(true)); assertThat(engine.segmentsStats(false).getCount(), equalTo(0L)); @@ -1311,9 +1323,13 @@ public class InternalEngineTests extends EngineTestCase { assertThat(indexResult.getVersion(), equalTo(1L)); } - public void testForceMerge() throws IOException { + public void testForceMergeWithoutSoftDeletes() throws IOException { + Settings settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); + IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); try (Store store = createStore(); - Engine engine = createEngine(config(defaultSettings, store, createTempDir(), + Engine engine = createEngine(config(IndexSettingsModule.newIndexSettings(indexMetaData), store, createTempDir(), new LogByteSizeMergePolicy(), null))) { // use log MP here we test some behavior in ESMP int numDocs = randomIntBetween(10, 100); for (int i = 0; i < numDocs; i++) { @@ -1354,6 +1370,165 @@ public class InternalEngineTests extends EngineTestCase { } } + public void testForceMergeWithSoftDeletesRetention() throws Exception { + final long retainedExtraOps = randomLongBetween(0, 10); + Settings.Builder settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), retainedExtraOps); + final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + final MapperService mapperService = createMapperService("test"); + final Set liveDocs = new HashSet<>(); + try (Store store = createStore(); + InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get))) { + int numDocs = scaledRandomIntBetween(10, 100); + for (int i = 0; i < numDocs; i++) { + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null); + engine.index(indexForDoc(doc)); + liveDocs.add(doc.id()); + } + for (int i = 0; i < numDocs; i++) { + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null); + if (randomBoolean()) { + engine.delete(new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get())); + liveDocs.remove(doc.id()); + } + if (randomBoolean()) { + engine.index(indexForDoc(doc)); + liveDocs.add(doc.id()); + } + if (randomBoolean()) { + engine.flush(randomBoolean(), true); + } + } + engine.flush(); + + long localCheckpoint = engine.getLocalCheckpoint(); + globalCheckpoint.set(randomLongBetween(0, localCheckpoint)); + engine.syncTranslog(); + final long safeCommitCheckpoint; + try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { + safeCommitCheckpoint = Long.parseLong(safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); + } + engine.forceMerge(true, 1, false, false, false); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + Map ops = readAllOperationsInLucene(engine, mapperService) + .stream().collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); + for (long seqno = 0; seqno <= localCheckpoint; seqno++) { + long minSeqNoToRetain = Math.min(globalCheckpoint.get() + 1 - retainedExtraOps, safeCommitCheckpoint + 1); + String msg = "seq# [" + seqno + "], global checkpoint [" + globalCheckpoint + "], retained-ops [" + retainedExtraOps + "]"; + if (seqno < minSeqNoToRetain) { + Translog.Operation op = ops.get(seqno); + if (op != null) { + assertThat(op, instanceOf(Translog.Index.class)); + assertThat(msg, ((Translog.Index) op).id(), isIn(liveDocs)); + assertEquals(msg, ((Translog.Index) op).source(), B_1); + } + } else { + assertThat(msg, ops.get(seqno), notNullValue()); + } + } + settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0); + indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); + engine.onSettingsChanged(); + globalCheckpoint.set(localCheckpoint); + engine.syncTranslog(); + + engine.forceMerge(true, 1, false, false, false); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + assertThat(readAllOperationsInLucene(engine, mapperService), hasSize(liveDocs.size())); + } + } + + public void testForceMergeWithSoftDeletesRetentionAndRecoverySource() throws Exception { + final long retainedExtraOps = randomLongBetween(0, 10); + Settings.Builder settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), retainedExtraOps); + final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + final MapperService mapperService = createMapperService("test"); + final boolean omitSourceAllTheTime = randomBoolean(); + final Set liveDocs = new HashSet<>(); + final Set liveDocsWithSource = new HashSet<>(); + try (Store store = createStore(); + InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, + globalCheckpoint::get))) { + int numDocs = scaledRandomIntBetween(10, 100); + for (int i = 0; i < numDocs; i++) { + boolean useRecoverySource = randomBoolean() || omitSourceAllTheTime; + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null, useRecoverySource); + engine.index(indexForDoc(doc)); + liveDocs.add(doc.id()); + if (useRecoverySource == false) { + liveDocsWithSource.add(Integer.toString(i)); + } + } + for (int i = 0; i < numDocs; i++) { + boolean useRecoverySource = randomBoolean() || omitSourceAllTheTime; + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null, useRecoverySource); + if (randomBoolean()) { + engine.delete(new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get())); + liveDocs.remove(doc.id()); + liveDocsWithSource.remove(doc.id()); + } + if (randomBoolean()) { + engine.index(indexForDoc(doc)); + liveDocs.add(doc.id()); + if (useRecoverySource == false) { + liveDocsWithSource.add(doc.id()); + } else { + liveDocsWithSource.remove(doc.id()); + } + } + if (randomBoolean()) { + engine.flush(randomBoolean(), true); + } + } + engine.flush(); + globalCheckpoint.set(randomLongBetween(0, engine.getLocalCheckpoint())); + engine.syncTranslog(); + final long minSeqNoToRetain; + try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { + long safeCommitLocalCheckpoint = Long.parseLong( + safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); + minSeqNoToRetain = Math.min(globalCheckpoint.get() + 1 - retainedExtraOps, safeCommitLocalCheckpoint + 1); + } + engine.forceMerge(true, 1, false, false, false); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + Map ops = readAllOperationsInLucene(engine, mapperService) + .stream().collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); + for (long seqno = 0; seqno <= engine.getLocalCheckpoint(); seqno++) { + String msg = "seq# [" + seqno + "], global checkpoint [" + globalCheckpoint + "], retained-ops [" + retainedExtraOps + "]"; + if (seqno < minSeqNoToRetain) { + Translog.Operation op = ops.get(seqno); + if (op != null) { + assertThat(op, instanceOf(Translog.Index.class)); + assertThat(msg, ((Translog.Index) op).id(), isIn(liveDocs)); + } + } else { + Translog.Operation op = ops.get(seqno); + assertThat(msg, op, notNullValue()); + if (op instanceof Translog.Index) { + assertEquals(msg, ((Translog.Index) op).source(), B_1); + } + } + } + settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0); + indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); + engine.onSettingsChanged(); + globalCheckpoint.set(engine.getLocalCheckpoint()); + engine.syncTranslog(); + engine.forceMerge(true, 1, false, false, false); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + assertThat(readAllOperationsInLucene(engine, mapperService), hasSize(liveDocsWithSource.size())); + } + } + public void testForceMergeAndClose() throws IOException, InterruptedException { int numIters = randomIntBetween(2, 10); for (int j = 0; j < numIters; j++) { @@ -1422,126 +1597,10 @@ public class InternalEngineTests extends EngineTestCase { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } - protected List generateSingleDocHistory(boolean forReplica, VersionType versionType, - long primaryTerm, - int minOpCount, int maxOpCount, String docId) { - final int numOfOps = randomIntBetween(minOpCount, maxOpCount); - final List ops = new ArrayList<>(); - final Term id = newUid(docId); - final int startWithSeqNo = 0; - final String valuePrefix = (forReplica ? "r_" : "p_" ) + docId + "_"; - final boolean incrementTermWhenIntroducingSeqNo = randomBoolean(); - for (int i = 0; i < numOfOps; i++) { - final Engine.Operation op; - final long version; - switch (versionType) { - case INTERNAL: - version = forReplica ? i : Versions.MATCH_ANY; - break; - case EXTERNAL: - version = i; - break; - case EXTERNAL_GTE: - version = randomBoolean() ? Math.max(i - 1, 0) : i; - break; - case FORCE: - version = randomNonNegativeLong(); - break; - default: - throw new UnsupportedOperationException("unknown version type: " + versionType); - } - if (randomBoolean()) { - op = new Engine.Index(id, testParsedDocument(docId, null, testDocumentWithTextField(valuePrefix + i), B_1, null), - forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, - forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, - version, - forReplica ? null : versionType, - forReplica ? REPLICA : PRIMARY, - System.currentTimeMillis(), -1, false - ); - } else { - op = new Engine.Delete("test", docId, id, - forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, - forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, - version, - forReplica ? null : versionType, - forReplica ? REPLICA : PRIMARY, - System.currentTimeMillis()); - } - ops.add(op); - } - return ops; - } - public void testOutOfOrderDocsOnReplica() throws IOException { final List ops = generateSingleDocHistory(true, randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE, VersionType.FORCE), 2, 2, 20, "1"); - assertOpsOnReplica(ops, replicaEngine, true); - } - - private void assertOpsOnReplica(List ops, InternalEngine replicaEngine, boolean shuffleOps) throws IOException { - final Engine.Operation lastOp = ops.get(ops.size() - 1); - final String lastFieldValue; - if (lastOp instanceof Engine.Index) { - Engine.Index index = (Engine.Index) lastOp; - lastFieldValue = index.docs().get(0).get("value"); - } else { - // delete - lastFieldValue = null; - } - if (shuffleOps) { - int firstOpWithSeqNo = 0; - while (firstOpWithSeqNo < ops.size() && ops.get(firstOpWithSeqNo).seqNo() < 0) { - firstOpWithSeqNo++; - } - // shuffle ops but make sure legacy ops are first - shuffle(ops.subList(0, firstOpWithSeqNo), random()); - shuffle(ops.subList(firstOpWithSeqNo, ops.size()), random()); - } - boolean firstOp = true; - for (Engine.Operation op : ops) { - logger.info("performing [{}], v [{}], seq# [{}], term [{}]", - op.operationType().name().charAt(0), op.version(), op.seqNo(), op.primaryTerm()); - if (op instanceof Engine.Index) { - Engine.IndexResult result = replicaEngine.index((Engine.Index) op); - // replicas don't really care to about creation status of documents - // this allows to ignore the case where a document was found in the live version maps in - // a delete state and return false for the created flag in favor of code simplicity - // as deleted or not. This check is just signal regression so a decision can be made if it's - // intentional - assertThat(result.isCreated(), equalTo(firstOp)); - assertThat(result.getVersion(), equalTo(op.version())); - assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); - - } else { - Engine.DeleteResult result = replicaEngine.delete((Engine.Delete) op); - // Replicas don't really care to about found status of documents - // this allows to ignore the case where a document was found in the live version maps in - // a delete state and return true for the found flag in favor of code simplicity - // his check is just signal regression so a decision can be made if it's - // intentional - assertThat(result.isFound(), equalTo(firstOp == false)); - assertThat(result.getVersion(), equalTo(op.version())); - assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); - } - if (randomBoolean()) { - engine.refresh("test"); - } - if (randomBoolean()) { - engine.flush(); - engine.refresh("test"); - } - firstOp = false; - } - - assertVisibleCount(replicaEngine, lastFieldValue == null ? 0 : 1); - if (lastFieldValue != null) { - try (Searcher searcher = replicaEngine.acquireSearcher("test")) { - final TotalHitCountCollector collector = new TotalHitCountCollector(); - searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); - assertThat(collector.getTotalHits(), equalTo(1)); - } - } + assertOpsOnReplica(ops, replicaEngine, true, logger); } public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, InterruptedException { @@ -1569,11 +1628,12 @@ public class InternalEngineTests extends EngineTestCase { } // randomly interleave final AtomicLong seqNoGenerator = new AtomicLong(); - Function seqNoUpdater = operation -> { - final long newSeqNo = seqNoGenerator.getAndIncrement(); + BiFunction seqNoUpdater = (operation, newSeqNo) -> { if (operation instanceof Engine.Index) { Engine.Index index = (Engine.Index) operation; - return new Engine.Index(index.uid(), index.parsedDoc(), newSeqNo, index.primaryTerm(), index.version(), + Document doc = testDocumentWithTextField(index.docs().get(0).get("value")); + ParsedDocument parsedDocument = testParsedDocument(index.id(), index.routing(), doc, index.source(), null); + return new Engine.Index(index.uid(), parsedDocument, newSeqNo, index.primaryTerm(), index.version(), index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry()); } else { Engine.Delete delete = (Engine.Delete) operation; @@ -1586,12 +1646,12 @@ public class InternalEngineTests extends EngineTestCase { Iterator iter2 = opsDoc2.iterator(); while (iter1.hasNext() && iter2.hasNext()) { final Engine.Operation next = randomBoolean() ? iter1.next() : iter2.next(); - allOps.add(seqNoUpdater.apply(next)); + allOps.add(seqNoUpdater.apply(next, seqNoGenerator.getAndIncrement())); } - iter1.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o))); - iter2.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o))); + iter1.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o, seqNoGenerator.getAndIncrement()))); + iter2.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o, seqNoGenerator.getAndIncrement()))); // insert some duplicates - allOps.addAll(randomSubsetOf(allOps)); + randomSubsetOf(allOps).forEach(op -> allOps.add(seqNoUpdater.apply(op, op.seqNo()))); shuffle(allOps, random()); concurrentlyApplyOps(allOps, engine); @@ -1623,42 +1683,6 @@ public class InternalEngineTests extends EngineTestCase { assertVisibleCount(engine, totalExpectedOps); } - private void concurrentlyApplyOps(List ops, InternalEngine engine) throws InterruptedException { - Thread[] thread = new Thread[randomIntBetween(3, 5)]; - CountDownLatch startGun = new CountDownLatch(thread.length); - AtomicInteger offset = new AtomicInteger(-1); - for (int i = 0; i < thread.length; i++) { - thread[i] = new Thread(() -> { - startGun.countDown(); - try { - startGun.await(); - } catch (InterruptedException e) { - throw new AssertionError(e); - } - int docOffset; - while ((docOffset = offset.incrementAndGet()) < ops.size()) { - try { - final Engine.Operation op = ops.get(docOffset); - if (op instanceof Engine.Index) { - engine.index((Engine.Index) op); - } else { - engine.delete((Engine.Delete) op); - } - if ((docOffset + 1) % 4 == 0) { - engine.refresh("test"); - } - } catch (IOException e) { - throw new AssertionError(e); - } - } - }); - thread[i].start(); - } - for (int i = 0; i < thread.length; i++) { - thread[i].join(); - } - } - public void testInternalVersioningOnPrimary() throws IOException { final List ops = generateSingleDocHistory(false, VersionType.INTERNAL, 2, 2, 20, "1"); assertOpsOnPrimary(ops, Versions.NOT_FOUND, true, engine); @@ -1869,7 +1893,7 @@ public class InternalEngineTests extends EngineTestCase { final boolean deletedOnReplica = lastReplicaOp instanceof Engine.Delete; final long finalReplicaVersion = lastReplicaOp.version(); final long finalReplicaSeqNo = lastReplicaOp.seqNo(); - assertOpsOnReplica(replicaOps, replicaEngine, true); + assertOpsOnReplica(replicaOps, replicaEngine, true, logger); final int opsOnPrimary = assertOpsOnPrimary(primaryOps, finalReplicaVersion, deletedOnReplica, replicaEngine); final long currentSeqNo = getSequenceID(replicaEngine, new Engine.Get(false, false, "type", lastReplicaOp.uid().text(), lastReplicaOp.uid())).v1(); @@ -2674,14 +2698,16 @@ public class InternalEngineTests extends EngineTestCase { Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } + EngineConfig config = engine.config(); assertVisibleCount(engine, numDocs); engine.close(); - trimUnsafeCommits(engine.config()); - engine = new InternalEngine(engine.config()); - engine.skipTranslogRecovery(); - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(0L)); + trimUnsafeCommits(config); + try (InternalEngine engine = new InternalEngine(config)) { + engine.skipTranslogRecovery(); + try (Engine.Searcher searcher = engine.acquireSearcher("test")) { + TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); + assertThat(topDocs.totalHits, equalTo(0L)); + } } } @@ -2811,7 +2837,7 @@ public class InternalEngineTests extends EngineTestCase { new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm::get); + new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm::get, tombstoneDocSupplier()); try { InternalEngine internalEngine = new InternalEngine(brokenConfig); fail("translog belongs to a different engine"); @@ -2940,6 +2966,12 @@ public class InternalEngineTests extends EngineTestCase { } } + @Override + public long softUpdateDocument(Term term, Iterable doc, Field... softDeletes) throws IOException { + maybeThrowFailure(); + return super.softUpdateDocument(term, doc, softDeletes); + } + @Override public long deleteDocuments(Term... terms) throws IOException { maybeThrowFailure(); @@ -3140,10 +3172,10 @@ public class InternalEngineTests extends EngineTestCase { } public void testDoubleDeliveryReplicaAppendingOnly() throws IOException { - final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), + final Supplier doc = () -> testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); - Engine.Index operation = appendOnlyReplica(doc, false, 1, randomIntBetween(0, 5)); - Engine.Index retry = appendOnlyReplica(doc, true, 1, randomIntBetween(0, 5)); + Engine.Index operation = appendOnlyReplica(doc.get(), false, 1, randomIntBetween(0, 5)); + Engine.Index retry = appendOnlyReplica(doc.get(), true, 1, randomIntBetween(0, 5)); // operations with a seq# equal or lower to the local checkpoint are not indexed to lucene // and the version lookup is skipped final boolean belowLckp = operation.seqNo() == 0 && retry.seqNo() == 0; @@ -3182,8 +3214,8 @@ public class InternalEngineTests extends EngineTestCase { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); assertEquals(1, topDocs.totalHits); } - operation = randomAppendOnly(doc, false, 1); - retry = randomAppendOnly(doc, true, 1); + operation = randomAppendOnly(doc.get(), false, 1); + retry = randomAppendOnly(doc.get(), true, 1); if (randomBoolean()) { Engine.IndexResult indexResult = engine.index(operation); assertNotNull(indexResult.getTranslogLocation()); @@ -3248,6 +3280,8 @@ public class InternalEngineTests extends EngineTestCase { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); assertEquals(1, topDocs.totalHits); } + List ops = readAllOperationsInLucene(engine, createMapperService("test")); + assertThat(ops.stream().map(o -> o.seqNo()).collect(Collectors.toList()), hasItem(20L)); } public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOException { @@ -3716,20 +3750,22 @@ public class InternalEngineTests extends EngineTestCase { final List operations = new ArrayList<>(); final int numberOfOperations = randomIntBetween(16, 32); - final Document document = testDocumentWithTextField(); final AtomicLong sequenceNumber = new AtomicLong(); final Engine.Operation.Origin origin = randomFrom(LOCAL_TRANSLOG_RECOVERY, PEER_RECOVERY, PRIMARY, REPLICA); final LongSupplier sequenceNumberSupplier = origin == PRIMARY ? () -> SequenceNumbers.UNASSIGNED_SEQ_NO : sequenceNumber::getAndIncrement; - document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); - final ParsedDocument doc = testParsedDocument("1", null, document, B_1, null); - final Term uid = newUid(doc); + final Supplier doc = () -> { + final Document document = testDocumentWithTextField(); + document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); + return testParsedDocument("1", null, document, B_1, null); + }; + final Term uid = newUid("1"); final BiFunction searcherFactory = engine::acquireSearcher; for (int i = 0; i < numberOfOperations; i++) { if (randomBoolean()) { final Engine.Index index = new Engine.Index( uid, - doc, + doc.get(), sequenceNumberSupplier.getAsLong(), 1, i, @@ -3805,7 +3841,9 @@ public class InternalEngineTests extends EngineTestCase { maxSeqNo, localCheckpoint); trimUnsafeCommits(engine.config()); - noOpEngine = new InternalEngine(engine.config(), supplier) { + EngineConfig noopEngineConfig = copy(engine.config(), new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, + () -> new MatchAllDocsQuery(), engine.config().getMergePolicy())); + noOpEngine = new InternalEngine(noopEngineConfig, supplier) { @Override protected long doGenerateSeqNoForOperation(Operation operation) { throw new UnsupportedOperationException(); @@ -3813,7 +3851,7 @@ public class InternalEngineTests extends EngineTestCase { }; noOpEngine.recoverFromTranslog(Long.MAX_VALUE); final int gapsFilled = noOpEngine.fillSeqNoGaps(primaryTerm.get()); - final String reason = randomAlphaOfLength(16); + final String reason = "filling gaps"; noOpEngine.noOp(new Engine.NoOp(maxSeqNo + 1, primaryTerm.get(), LOCAL_TRANSLOG_RECOVERY, System.nanoTime(), reason)); assertThat(noOpEngine.getLocalCheckpoint(), equalTo((long) (maxSeqNo + 1))); assertThat(noOpEngine.getTranslog().stats().getUncommittedOperations(), equalTo(gapsFilled)); @@ -3835,11 +3873,77 @@ public class InternalEngineTests extends EngineTestCase { assertThat(noOp.seqNo(), equalTo((long) (maxSeqNo + 2))); assertThat(noOp.primaryTerm(), equalTo(primaryTerm.get())); assertThat(noOp.reason(), equalTo(reason)); + if (engine.engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + MapperService mapperService = createMapperService("test"); + List operationsFromLucene = readAllOperationsInLucene(noOpEngine, mapperService); + assertThat(operationsFromLucene, hasSize(maxSeqNo + 2 - localCheckpoint)); // fills n gap and 2 manual noop. + for (int i = 0; i < operationsFromLucene.size(); i++) { + assertThat(operationsFromLucene.get(i), equalTo(new Translog.NoOp(localCheckpoint + 1 + i, primaryTerm.get(), "filling gaps"))); + } + assertConsistentHistoryBetweenTranslogAndLuceneIndex(noOpEngine, mapperService); + } } finally { IOUtils.close(noOpEngine); } } + /** + * Verifies that a segment containing only no-ops can be used to look up _version and _seqno. + */ + public void testSegmentContainsOnlyNoOps() throws Exception { + Engine.NoOpResult noOpResult = engine.noOp(new Engine.NoOp(1, primaryTerm.get(), + randomFrom(Engine.Operation.Origin.values()), randomNonNegativeLong(), "test")); + assertThat(noOpResult.getFailure(), nullValue()); + engine.refresh("test"); + Engine.DeleteResult deleteResult = engine.delete(replicaDeleteForDoc("id", 1, 2, randomNonNegativeLong())); + assertThat(deleteResult.getFailure(), nullValue()); + engine.refresh("test"); + } + + /** + * A simple test to check that random combination of operations can coexist in segments and be lookup. + * This is needed as some fields in Lucene may not exist if a segment misses operation types and this code is to check for that. + * For example, a segment containing only no-ops does not have neither _uid or _version. + */ + public void testRandomOperations() throws Exception { + int numOps = between(10, 100); + for (int i = 0; i < numOps; i++) { + String id = Integer.toString(randomIntBetween(1, 10)); + ParsedDocument doc = createParsedDoc(id, null); + Engine.Operation.TYPE type = randomFrom(Engine.Operation.TYPE.values()); + switch (type) { + case INDEX: + Engine.IndexResult index = engine.index(replicaIndexForDoc(doc, between(1, 100), i, randomBoolean())); + assertThat(index.getFailure(), nullValue()); + break; + case DELETE: + Engine.DeleteResult delete = engine.delete(replicaDeleteForDoc(doc.id(), between(1, 100), i, randomNonNegativeLong())); + assertThat(delete.getFailure(), nullValue()); + break; + case NO_OP: + Engine.NoOpResult noOp = engine.noOp(new Engine.NoOp(i, primaryTerm.get(), + randomFrom(Engine.Operation.Origin.values()), randomNonNegativeLong(), "")); + assertThat(noOp.getFailure(), nullValue()); + break; + default: + throw new IllegalStateException("Invalid op [" + type + "]"); + } + if (randomBoolean()) { + engine.refresh("test"); + } + if (randomBoolean()) { + engine.flush(); + } + if (randomBoolean()) { + engine.forceMerge(randomBoolean(), between(1, 10), randomBoolean(), false, false); + } + } + if (engine.engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + List operations = readAllOperationsInLucene(engine, createMapperService("test")); + assertThat(operations, hasSize(numOps)); + } + } + public void testMinGenerationForSeqNo() throws IOException, BrokenBarrierException, InterruptedException { engine.close(); final int numberOfTriplets = randomIntBetween(1, 32); @@ -4405,7 +4509,7 @@ public class InternalEngineTests extends EngineTestCase { globalCheckpoint.set(randomLongBetween(engine.getLocalCheckpoint(), Long.MAX_VALUE)); engine.syncTranslog(); assertThat(DirectoryReader.listCommits(store.directory()), contains(commits.get(commits.size() - 1))); - assertThat(engine.estimateTranslogOperationsFromMinSeq(0L), equalTo(0)); + assertThat(engine.getTranslog().totalOperations(), equalTo(0)); } } @@ -4768,6 +4872,154 @@ public class InternalEngineTests extends EngineTestCase { } } + public void testLuceneHistoryOnPrimary() throws Exception { + final List operations = generateSingleDocHistory(false, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "1"); + assertOperationHistoryInLucene(operations); + } + + public void testLuceneHistoryOnReplica() throws Exception { + final List operations = generateSingleDocHistory(true, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); + Randomness.shuffle(operations); + assertOperationHistoryInLucene(operations); + } + + private void assertOperationHistoryInLucene(List operations) throws IOException { + final MergePolicy keepSoftDeleteDocsMP = new SoftDeletesRetentionMergePolicy( + Lucene.SOFT_DELETES_FIELD, () -> new MatchAllDocsQuery(), engine.config().getMergePolicy()); + Settings.Builder settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); + final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); + Set expectedSeqNos = new HashSet<>(); + try (Store store = createStore(); + Engine engine = createEngine(config(indexSettings, store, createTempDir(), keepSoftDeleteDocsMP, null))) { + for (Engine.Operation op : operations) { + if (op instanceof Engine.Index) { + Engine.IndexResult indexResult = engine.index((Engine.Index) op); + assertThat(indexResult.getFailure(), nullValue()); + expectedSeqNos.add(indexResult.getSeqNo()); + } else { + Engine.DeleteResult deleteResult = engine.delete((Engine.Delete) op); + assertThat(deleteResult.getFailure(), nullValue()); + expectedSeqNos.add(deleteResult.getSeqNo()); + } + if (rarely()) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(); + } + if (rarely()) { + engine.forceMerge(true); + } + } + MapperService mapperService = createMapperService("test"); + List actualOps = readAllOperationsInLucene(engine, mapperService); + assertThat(actualOps.stream().map(o -> o.seqNo()).collect(Collectors.toList()), containsInAnyOrder(expectedSeqNos.toArray())); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + } + } + + public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException { + IOUtils.close(engine, store); + Settings.Builder settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); + final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + final List operations = generateSingleDocHistory(true, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); + Randomness.shuffle(operations); + Set existingSeqNos = new HashSet<>(); + store = createStore(); + engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get)); + assertThat(engine.getMinRetainedSeqNo(), equalTo(0L)); + long lastMinRetainedSeqNo = engine.getMinRetainedSeqNo(); + for (Engine.Operation op : operations) { + final Engine.Result result; + if (op instanceof Engine.Index) { + result = engine.index((Engine.Index) op); + } else { + result = engine.delete((Engine.Delete) op); + } + existingSeqNos.add(result.getSeqNo()); + if (randomBoolean()) { + globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getCheckpoint())); + } + if (rarely()) { + settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); + indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); + engine.onSettingsChanged(); + } + if (rarely()) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(true, true); + assertThat(Long.parseLong(engine.getLastCommittedSegmentInfos().userData.get(Engine.MIN_RETAINED_SEQNO)), + equalTo(engine.getMinRetainedSeqNo())); + } + if (rarely()) { + engine.forceMerge(randomBoolean()); + } + try (Closeable ignored = engine.acquireRetentionLockForPeerRecovery()) { + long minRetainSeqNos = engine.getMinRetainedSeqNo(); + assertThat(minRetainSeqNos, lessThanOrEqualTo(globalCheckpoint.get() + 1)); + Long[] expectedOps = existingSeqNos.stream().filter(seqno -> seqno >= minRetainSeqNos).toArray(Long[]::new); + Set actualOps = readAllOperationsInLucene(engine, createMapperService("test")).stream() + .map(Translog.Operation::seqNo).collect(Collectors.toSet()); + assertThat(actualOps, containsInAnyOrder(expectedOps)); + } + try (Engine.IndexCommitRef commitRef = engine.acquireSafeIndexCommit()) { + IndexCommit safeCommit = commitRef.getIndexCommit(); + if (safeCommit.getUserData().containsKey(Engine.MIN_RETAINED_SEQNO)) { + lastMinRetainedSeqNo = Long.parseLong(safeCommit.getUserData().get(Engine.MIN_RETAINED_SEQNO)); + } + } + } + if (randomBoolean()) { + engine.close(); + } else { + engine.flushAndClose(); + } + trimUnsafeCommits(engine.config()); + try (InternalEngine recoveringEngine = new InternalEngine(engine.config())) { + assertThat(recoveringEngine.getMinRetainedSeqNo(), equalTo(lastMinRetainedSeqNo)); + } + } + + public void testLastRefreshCheckpoint() throws Exception { + AtomicBoolean done = new AtomicBoolean(); + Thread[] refreshThreads = new Thread[between(1, 8)]; + CountDownLatch latch = new CountDownLatch(refreshThreads.length); + for (int i = 0; i < refreshThreads.length; i++) { + latch.countDown(); + refreshThreads[i] = new Thread(() -> { + while (done.get() == false) { + long checkPointBeforeRefresh = engine.getLocalCheckpoint(); + engine.refresh("test", randomFrom(Engine.SearcherScope.values())); + assertThat(engine.lastRefreshedCheckpoint(), greaterThanOrEqualTo(checkPointBeforeRefresh)); + } + }); + refreshThreads[i].start(); + } + latch.await(); + List ops = generateSingleDocHistory(true, VersionType.EXTERNAL, 1, 10, 1000, "1"); + concurrentlyApplyOps(ops, engine); + done.set(true); + for (Thread thread : refreshThreads) { + thread.join(); + } + engine.refresh("test"); + assertThat(engine.lastRefreshedCheckpoint(), equalTo(engine.getLocalCheckpoint())); + } + private static void trimUnsafeCommits(EngineConfig config) throws IOException { final Store store = config.getStore(); final TranslogConfig translogConfig = config.getTranslogConfig(); diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java new file mode 100644 index 00000000000..2d097366a27 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -0,0 +1,289 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.SnapshotMatchers; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.test.IndexSettingsModule; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class LuceneChangesSnapshotTests extends EngineTestCase { + private MapperService mapperService; + + @Before + public void createMapper() throws Exception { + mapperService = createMapperService("test"); + } + + @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", mapperService, fromSeqNo, toSeqNo, true)) { + 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", mapperService, fromSeqNo, toSeqNo, false)) { + 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.type(), doc.id(), newUid(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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, false)) { + 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { + 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, false)) { + 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { + 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { + 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", mapperService, fromSeqNo, toSeqNo, randomBoolean())) { + assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); + } + } + + public void testDedupByPrimaryTerm() throws Exception { + Map latestOperations = new HashMap<>(); + List terms = Arrays.asList(between(1, 1000), between(1000, 2000)); + int totalOps = 0; + for (long term : terms) { + final List ops = generateSingleDocHistory(true, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE), term, 2, 20, "1"); + primaryTerm.set(Math.max(primaryTerm.get(), term)); + engine.rollTranslogGeneration(); + for (Engine.Operation op : ops) { + // We need to simulate a rollback here as only ops after local checkpoint get into the engine + if (op.seqNo() <= engine.getLocalCheckpointTracker().getCheckpoint()) { + engine.getLocalCheckpointTracker().resetCheckpoint(randomLongBetween(-1, op.seqNo() - 1)); + engine.rollTranslogGeneration(); + } + if (op instanceof Engine.Index) { + engine.index((Engine.Index) op); + } else if (op instanceof Engine.Delete) { + engine.delete((Engine.Delete) op); + } + latestOperations.put(op.seqNo(), op.primaryTerm()); + if (rarely()) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(); + } + totalOps++; + } + } + long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); + try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", mapperService, 0, maxSeqNo, false)) { + Translog.Operation op; + while ((op = snapshot.next()) != null) { + assertThat(op.toString(), op.primaryTerm(), equalTo(latestOperations.get(op.seqNo()))); + } + assertThat(snapshot.skippedOperations(), equalTo(totalOps - latestOperations.size())); + } + } + + 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 = scaledRandomIntBetween(1, 1000); + for (int i = 0; i < numOps; i++) { + String id = Integer.toString(randomIntBetween(1, 10)); + 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.type(), doc.id(), newUid(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(); + concurrentlyApplyOps(operations, engine); + assertThat(engine.getLocalCheckpointTracker().getCheckpoint(), equalTo(operations.size() - 1L)); + isDone.set(true); + for (Follower follower : followers) { + follower.join(); + } + } + + class Follower extends Thread { + private final Engine leader; + private final TranslogHandler translogHandler; + private final AtomicBoolean isDone; + private final CountDownLatch readLatch; + + Follower(Engine leader, AtomicBoolean isDone, CountDownLatch readLatch) { + this.leader = leader; + this.isDone = isDone; + this.readLatch = readLatch; + this.translogHandler = new TranslogHandler(xContentRegistry(), IndexSettingsModule.newIndexSettings(shardId.getIndexName(), + engine.engineConfig.getIndexSettings().getSettings())); + } + + void pullOperations(Engine follower) throws IOException { + long leaderCheckpoint = leader.getLocalCheckpoint(); + long followerCheckpoint = follower.getLocalCheckpoint(); + 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", mapperService, fromSeqNo, toSeqNo, true)) { + translogHandler.run(follower, snapshot); + } + } + } + + @Override + public void run() { + try (Store store = createStore(); + InternalEngine follower = createEngine(store, createTempDir())) { + readLatch.countDown(); + readLatch.await(); + while (isDone.get() == false || + follower.getLocalCheckpointTracker().getCheckpoint() < leader.getLocalCheckpoint()) { + pullOperations(follower); + } + assertConsistentHistoryBetweenTranslogAndLuceneIndex(follower, mapperService); + assertThat(getDocIds(follower, true), equalTo(getDocIds(leader, true))); + } 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.error("Reading [{}]", op); + assert operations.stream().allMatch(o -> o.seqNo() < newOp.seqNo()) : "Operations [" + operations + "], op [" + op + "]"; + operations.add(newOp); + } + return operations; + } +} diff --git a/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java new file mode 100644 index 00000000000..c46b47b87d0 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java @@ -0,0 +1,161 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.CodecReader; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.MergePolicy; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SegmentCommitInfo; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.StandardDirectoryReader; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.NullInfoStream; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.Collections; +import java.util.Set; +import java.util.stream.Collectors; + +public class RecoverySourcePruneMergePolicyTests extends ESTestCase { + + public void testPruneAll() throws IOException { + try (Directory dir = newDirectory()) { + IndexWriterConfig iwc = newIndexWriterConfig(); + RecoverySourcePruneMergePolicy mp = new RecoverySourcePruneMergePolicy("extra_source", MatchNoDocsQuery::new, + newLogMergePolicy()); + iwc.setMergePolicy(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("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); + assertEquals(1, document.getFields().size()); + assertEquals("source", document.getFields().get(0).name()); + } + 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 && reader instanceof StandardDirectoryReader) { + CodecReader codecReader = (CodecReader) leafReader; + StandardDirectoryReader sdr = (StandardDirectoryReader) reader; + 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 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)); + } + } + } + } + } + + + public void testPruneSome() throws IOException { + try (Directory dir = newDirectory()) { + IndexWriterConfig iwc = newIndexWriterConfig(); + iwc.setMergePolicy(new RecoverySourcePruneMergePolicy("extra_source", + () -> 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 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() == 3) { + assertTrue(collect.contains("extra_source")); + assertEquals("true", document.getField("even").stringValue()); + assertEquals(i, extra_source.nextDoc()); + } else { + assertEquals(2, document.getFields().size()); + } + } + assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); + } + } + } + } +} diff --git a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java new file mode 100644 index 00000000000..f3590100382 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java @@ -0,0 +1,75 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + +import static org.hamcrest.Matchers.equalTo; + +public class SoftDeletesPolicyTests extends ESTestCase { + /** + * Makes sure we won't advance the retained seq# if the retention lock is held + */ + public void testSoftDeletesRetentionLock() { + long retainedOps = between(0, 10000); + AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + long safeCommitCheckpoint = globalCheckpoint.get(); + SoftDeletesPolicy policy = new SoftDeletesPolicy(globalCheckpoint::get, between(1, 10000), retainedOps); + long minRetainedSeqNo = policy.getMinRetainedSeqNo(); + List locks = new ArrayList<>(); + int iters = scaledRandomIntBetween(10, 1000); + for (int i = 0; i < iters; i++) { + if (randomBoolean()) { + locks.add(policy.acquireRetentionLock()); + } + // Advances the global checkpoint and the local checkpoint of a safe commit + globalCheckpoint.addAndGet(between(0, 1000)); + safeCommitCheckpoint = randomLongBetween(safeCommitCheckpoint, globalCheckpoint.get()); + policy.setLocalCheckpointOfSafeCommit(safeCommitCheckpoint); + if (rarely()) { + retainedOps = between(0, 10000); + policy.setRetentionOperations(retainedOps); + } + // Release some locks + List releasingLocks = randomSubsetOf(locks); + locks.removeAll(releasingLocks); + releasingLocks.forEach(Releasable::close); + + // We only expose the seqno to the merge policy if the retention lock is not held. + policy.getRetentionQuery(); + if (locks.isEmpty()) { + long retainedSeqNo = Math.min(safeCommitCheckpoint, globalCheckpoint.get() - retainedOps) + 1; + minRetainedSeqNo = Math.max(minRetainedSeqNo, retainedSeqNo); + } + assertThat(policy.getMinRetainedSeqNo(), equalTo(minRetainedSeqNo)); + } + + locks.forEach(Releasable::close); + long retainedSeqNo = Math.min(safeCommitCheckpoint, globalCheckpoint.get() - retainedOps) + 1; + minRetainedSeqNo = Math.max(minRetainedSeqNo, retainedSeqNo); + assertThat(policy.getMinRetainedSeqNo(), equalTo(minRetainedSeqNo)); + } +} diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java index 76ca6aa7ea8..5a46b9a889f 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -311,15 +312,18 @@ public class DocumentParserTests extends ESSingleNodeTestCase { // creates an object mapper, which is about 100x harder than it should be.... ObjectMapper createObjectMapper(MapperService mapperService, String name) throws Exception { - ParseContext context = new ParseContext.InternalParseContext( - Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(), + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + ParseContext context = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), mapperService.documentMapper("type"), null, null); String[] nameParts = name.split("\\."); for (int i = 0; i < nameParts.length - 1; ++i) { context.path().add(nameParts[i]); } Mapper.Builder builder = new ObjectMapper.Builder(nameParts[nameParts.length - 1]).enabled(true); - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); return (ObjectMapper)builder.build(builderContext); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java index cb2ed785699..b11e4876f9e 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.BooleanFieldMapper.BooleanFieldType; import org.elasticsearch.index.mapper.DateFieldMapper.DateFieldType; import org.elasticsearch.index.mapper.NumberFieldMapper.NumberFieldType; @@ -215,7 +216,10 @@ public class DynamicMappingTests extends ESSingleNodeTestCase { } private Mapper parse(DocumentMapper mapper, DocumentMapperParser parser, XContentBuilder builder) throws Exception { - Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); SourceToParse source = SourceToParse.source("test", mapper.type(), "some_id", BytesReference.bytes(builder), builder.contentType()); try (XContentParser xContentParser = createParser(JsonXContent.jsonXContent, source.source())) { ParseContext.InternalParseContext ctx = new ParseContext.InternalParseContext(settings, parser, mapper, source, xContentParser); 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 1d1e423afc1..fba71dd1e52 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.index.replication; +import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; @@ -41,6 +42,7 @@ import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.engine.InternalEngineTests; import org.elasticsearch.index.engine.SegmentsStats; import org.elasticsearch.index.engine.VersionConflictEngineException; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -140,7 +142,9 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase } public void testInheritMaxValidAutoIDTimestampOnRecovery() throws Exception { - try (ReplicationGroup shards = createGroup(0)) { + //TODO: Enables this test with soft-deletes once we have timestamp + Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); + try (ReplicationGroup shards = createGroup(0, settings)) { shards.startAll(); final IndexRequest indexRequest = new IndexRequest(index.getName(), "type").source("{}", XContentType.JSON); indexRequest.onRetry(); // force an update of the timestamp @@ -346,7 +350,13 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase final AtomicBoolean throwAfterIndexedOneDoc = new AtomicBoolean(); // need one document to trigger delete in IW. @Override public long addDocument(Iterable doc) throws IOException { - if (throwAfterIndexedOneDoc.getAndSet(true)) { + boolean isTombstone = false; + for (IndexableField field : doc) { + if (SeqNoFieldMapper.TOMBSTONE_NAME.equals(field.name())) { + isTombstone = true; + } + } + if (isTombstone == false && throwAfterIndexedOneDoc.getAndSet(true)) { throw indexException; } else { return super.addDocument(doc); @@ -356,6 +366,10 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase public long deleteDocuments(Term... terms) throws IOException { throw deleteException; } + @Override + public long softUpdateDocument(Term term, Iterable doc, Field...fields) throws IOException { + throw deleteException; // a delete uses softUpdateDocument API if soft-deletes enabled + } }, null, null, config); try (ReplicationGroup shards = new ReplicationGroup(buildIndexMetaData(0)) { @Override @@ -390,6 +404,9 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase try (Translog.Snapshot snapshot = getTranslog(shard).newSnapshot()) { assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); } + try (Translog.Snapshot snapshot = shard.getHistoryOperations("test", 0)) { + assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); + } } // unlike previous failures, these two failures replicated directly from the replication channel. indexResp = shards.index(new IndexRequest(index.getName(), "type", "any").source("{}", XContentType.JSON)); @@ -404,6 +421,9 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase try (Translog.Snapshot snapshot = getTranslog(shard).newSnapshot()) { assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); } + try (Translog.Snapshot snapshot = shard.getHistoryOperations("test", 0)) { + assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); + } } shards.assertAllEqual(1); } @@ -501,8 +521,9 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase recoverReplica(replica3, replica2, true); try (Translog.Snapshot snapshot = getTranslog(replica3).newSnapshot()) { assertThat(snapshot.totalOperations(), equalTo(initDocs + 1)); - assertThat(snapshot.next(), equalTo(op2)); - assertThat("Remaining of snapshot should contain init operations", snapshot, containsOperationsInAnyOrder(initOperations)); + final List expectedOps = new ArrayList<>(initOperations); + expectedOps.add(op2); + assertThat(snapshot, containsOperationsInAnyOrder(expectedOps)); assertThat("Peer-recovery should not send overridden operations", snapshot.skippedOperations(), equalTo(0)); } // TODO: We should assert the content of shards in the ReplicationGroup. diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index 2d198c32ba7..28122665e9b 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -98,7 +98,8 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC } public void testRecoveryOfDisconnectedReplica() throws Exception { - try (ReplicationGroup shards = createGroup(1)) { + Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); + try (ReplicationGroup shards = createGroup(1, settings)) { shards.startAll(); int docs = shards.indexDocs(randomInt(50)); shards.flush(); @@ -266,6 +267,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC builder.settings(Settings.builder().put(newPrimary.indexSettings().getSettings()) .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) ); newPrimary.indexSettings().updateIndexMetaData(builder.build()); newPrimary.onSettingsChanged(); @@ -275,7 +277,12 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC shards.syncGlobalCheckpoint(); assertThat(newPrimary.getLastSyncedGlobalCheckpoint(), equalTo(newPrimary.seqNoStats().getMaxSeqNo())); }); - newPrimary.flush(new FlushRequest()); + newPrimary.flush(new FlushRequest().force(true)); + if (replica.indexSettings().isSoftDeleteEnabled()) { + // We need an extra flush to advance the min_retained_seqno on the new primary so ops-based won't happen. + // The min_retained_seqno only advances when a merge asks for the retention query. + newPrimary.flush(new FlushRequest().force(true)); + } uncommittedOpsOnPrimary = shards.indexDocs(randomIntBetween(0, 10)); totalDocs += uncommittedOpsOnPrimary; } 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 2228e1b017f..50f95bf4d47 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.TermQuery; @@ -30,6 +31,7 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; +import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Constants; import org.elasticsearch.Assertions; import org.elasticsearch.Version; @@ -89,8 +91,13 @@ import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.ParseContext; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; @@ -160,6 +167,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.hasToString; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.lessThan; @@ -237,7 +245,8 @@ public class IndexShardTests extends IndexShardTestCase { assertNotNull(shardPath); // fail shard shard.failShard("test shard fail", new CorruptIndexException("", "")); - closeShards(shard); + shard.close("do not assert history", false); + shard.store().close(); // check state file still exists ShardStateMetaData shardStateMetaData = load(logger, shardPath.getShardStatePath()); assertEquals(shardStateMetaData, getShardStateMetadata(shard)); @@ -2394,7 +2403,8 @@ public class IndexShardTests extends IndexShardTestCase { public void testDocStats() throws IOException, InterruptedException { IndexShard indexShard = null; try { - indexShard = newStartedShard(); + indexShard = newStartedShard( + Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0).build()); final long numDocs = randomIntBetween(2, 32); // at least two documents so we have docs to delete final long numDocsToDelete = randomLongBetween(1, numDocs); for (int i = 0; i < numDocs; i++) { @@ -2424,7 +2434,16 @@ public class IndexShardTests extends IndexShardTestCase { deleteDoc(indexShard, "_doc", id); indexDoc(indexShard, "_doc", id); } - + // Need to update and sync the global checkpoint as the soft-deletes retention MergePolicy depends on it. + if (indexShard.indexSettings.isSoftDeleteEnabled()) { + if (indexShard.routingEntry().primary()) { + indexShard.updateGlobalCheckpointForShard(indexShard.routingEntry().allocationId().getId(), + indexShard.getLocalCheckpoint()); + } else { + indexShard.updateGlobalCheckpointOnReplica(indexShard.getLocalCheckpoint(), "test"); + } + indexShard.sync(); + } // flush the buffered deletes final FlushRequest flushRequest = new FlushRequest(); flushRequest.force(false); @@ -2962,6 +2981,7 @@ public class IndexShardTests extends IndexShardTestCase { assertThat(breaker.getUsed(), greaterThan(preRefreshBytes)); indexDoc(primary, "_doc", "4", "{\"foo\": \"potato\"}"); + indexDoc(primary, "_doc", "5", "{\"foo\": \"potato\"}"); // Forces a refresh with the INTERNAL scope ((InternalEngine) primary.getEngine()).writeIndexingBuffer(); @@ -2973,6 +2993,13 @@ public class IndexShardTests extends IndexShardTestCase { // Deleting a doc causes its memory to be freed from the breaker deleteDoc(primary, "_doc", "0"); + // Here we are testing that a fully deleted segment should be dropped and its memory usage is freed. + // In order to instruct the merge policy not to keep a fully deleted segment, + // we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything. + if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { + primary.sync(); + flushShard(primary); + } primary.refresh("force refresh"); ss = primary.segmentStats(randomBoolean()); @@ -3064,6 +3091,7 @@ public class IndexShardTests extends IndexShardTestCase { // Close remaining searchers IOUtils.close(searchers); + primary.refresh("test"); SegmentsStats ss = primary.segmentStats(randomBoolean()); CircuitBreaker breaker = primary.circuitBreakerService.getBreaker(CircuitBreaker.ACCOUNTING); @@ -3181,4 +3209,28 @@ public class IndexShardTests extends IndexShardTestCase { } + public void testSupplyTombstoneDoc() throws Exception { + IndexShard shard = newStartedShard(); + String id = randomRealisticUnicodeOfLengthBetween(1, 10); + ParsedDocument deleteTombstone = shard.getEngine().config().getTombstoneDocSupplier().newDeleteTombstoneDoc("doc", id); + assertThat(deleteTombstone.docs(), hasSize(1)); + ParseContext.Document deleteDoc = deleteTombstone.docs().get(0); + assertThat(deleteDoc.getFields().stream().map(IndexableField::name).collect(Collectors.toList()), + containsInAnyOrder(IdFieldMapper.NAME, VersionFieldMapper.NAME, + SeqNoFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME)); + assertThat(deleteDoc.getField(IdFieldMapper.NAME).binaryValue(), equalTo(Uid.encodeId(id))); + assertThat(deleteDoc.getField(SeqNoFieldMapper.TOMBSTONE_NAME).numericValue().longValue(), equalTo(1L)); + + final String reason = randomUnicodeOfLength(200); + ParsedDocument noopTombstone = shard.getEngine().config().getTombstoneDocSupplier().newNoopTombstoneDoc(reason); + assertThat(noopTombstone.docs(), hasSize(1)); + ParseContext.Document noopDoc = noopTombstone.docs().get(0); + assertThat(noopDoc.getFields().stream().map(IndexableField::name).collect(Collectors.toList()), + containsInAnyOrder(VersionFieldMapper.NAME, SourceFieldMapper.NAME, SeqNoFieldMapper.TOMBSTONE_NAME, + SeqNoFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME)); + assertThat(noopDoc.getField(SeqNoFieldMapper.TOMBSTONE_NAME).numericValue().longValue(), equalTo(1L)); + assertThat(noopDoc.getField(SourceFieldMapper.NAME).binaryValue(), equalTo(new BytesRef(reason))); + + closeShards(shard); + } } diff --git a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java index ae2cc84e487..29b16ca28f4 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java @@ -106,17 +106,22 @@ public class PrimaryReplicaSyncerTests extends IndexShardTestCase { .isPresent(), is(false)); } - - assertEquals(globalCheckPoint == numDocs - 1 ? 0 : numDocs, resyncTask.getTotalOperations()); if (syncNeeded && globalCheckPoint < numDocs - 1) { - long skippedOps = globalCheckPoint + 1; // everything up to global checkpoint included - assertEquals(skippedOps, resyncTask.getSkippedOperations()); - assertEquals(numDocs - skippedOps, resyncTask.getResyncedOperations()); + if (shard.indexSettings.isSoftDeleteEnabled()) { + assertThat(resyncTask.getSkippedOperations(), equalTo(0)); + assertThat(resyncTask.getResyncedOperations(), equalTo(resyncTask.getTotalOperations())); + assertThat(resyncTask.getTotalOperations(), equalTo(Math.toIntExact(numDocs - 1 - globalCheckPoint))); + } else { + int skippedOps = Math.toIntExact(globalCheckPoint + 1); // everything up to global checkpoint included + assertThat(resyncTask.getSkippedOperations(), equalTo(skippedOps)); + assertThat(resyncTask.getResyncedOperations(), equalTo(numDocs - skippedOps)); + assertThat(resyncTask.getTotalOperations(), equalTo(globalCheckPoint == numDocs - 1 ? 0 : numDocs)); + } } else { - assertEquals(0, resyncTask.getSkippedOperations()); - assertEquals(0, resyncTask.getResyncedOperations()); + assertThat(resyncTask.getSkippedOperations(), equalTo(0)); + assertThat(resyncTask.getResyncedOperations(), equalTo(0)); + assertThat(resyncTask.getTotalOperations(), equalTo(0)); } - closeShards(shard); } 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 774b272121a..b93f170174c 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; +import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; import org.elasticsearch.index.mapper.IdFieldMapper; @@ -130,7 +131,8 @@ public class RefreshListenersTests extends ESTestCase { indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), Collections.singletonList(listeners), Collections.emptyList(), null, - (e, s) -> 0, new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, () -> primaryTerm); + (e, s) -> 0, new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, () -> primaryTerm, + EngineTestCase.tombstoneDocSupplier()); engine = new InternalEngine(config); engine.recoverFromTranslog(Long.MAX_VALUE); listeners.setCurrentRefreshLocationSupplier(engine::getTranslogLastWriteLocation); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index 89a8813e3e0..81afab4bb8f 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -67,6 +67,7 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; +import org.junit.After; import java.io.IOException; import java.util.ArrayList; @@ -110,6 +111,11 @@ public class IndexRecoveryIT extends ESIntegTestCase { RecoverySettingsChunkSizePlugin.class); } + @After + public void assertConsistentHistoryInLuceneIndex() throws Exception { + internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); + } + private void assertRecoveryStateWithoutStage(RecoveryState state, int shardId, RecoverySource recoverySource, boolean primary, String sourceNode, String targetNode) { assertThat(state.getShardId().getId(), equalTo(shardId)); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java index 4b1419375e6..b6f5a7b6451 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -25,6 +25,7 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.NoMergePolicy; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; @@ -91,6 +92,7 @@ public class PeerRecoveryTargetServiceTests extends IndexShardTestCase { replica.close("test", false); final List commits = DirectoryReader.listCommits(replica.store().directory()); IndexWriterConfig iwc = new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) .setMergePolicy(NoMergePolicy.INSTANCE) .setOpenMode(IndexWriterConfig.OpenMode.APPEND); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index f0644b029c3..0351111c305 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -411,12 +411,6 @@ public class RecoverySourceHandlerTests extends ESTestCase { recoverySettings.getChunkSize().bytesAsInt(), Settings.EMPTY) { - - @Override - boolean isTranslogReadyForSequenceNumberBasedRecovery() throws IOException { - return randomBoolean(); - } - @Override public void phase1(final IndexCommit snapshot, final Supplier translogOps) { phase1Called.set(true); 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 5547a629ab2..45535e19672 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.SourceToParse; @@ -63,13 +64,13 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { int docs = shards.indexDocs(10); getTranslog(shards.getPrimary()).rollGeneration(); shards.flush(); - if (randomBoolean()) { - docs += shards.indexDocs(10); - } + int moreDocs = shards.indexDocs(randomInt(10)); shards.addReplica(); shards.startAll(); final IndexShard replica = shards.getReplicas().get(0); - assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(docs)); + boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); + assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? moreDocs : docs + moreDocs)); + shards.assertAllEqual(docs + moreDocs); } } @@ -101,12 +102,12 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { // rolling/flushing is async assertBusy(() -> { assertThat(replica.getLastSyncedGlobalCheckpoint(), equalTo(19L)); - assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(0)); + assertThat(getTranslog(replica).totalOperations(), equalTo(0)); }); } } - public void testRecoveryWithOutOfOrderDelete() throws Exception { + public void testRecoveryWithOutOfOrderDeleteWithTranslog() throws Exception { /* * The flow of this test: * - delete #1 @@ -118,7 +119,8 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { * - index #5 * - If flush and the translog retention disabled, delete #1 will be removed while index #0 is still retained and replayed. */ - try (ReplicationGroup shards = createGroup(1)) { + Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); + try (ReplicationGroup shards = createGroup(1, settings)) { shards.startAll(); // create out of order delete and index op on replica final IndexShard orgReplica = shards.getReplicas().get(0); @@ -170,7 +172,63 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(newReplica); shards.assertAllEqual(3); - assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(translogOps)); + assertThat(getTranslog(newReplica).totalOperations(), equalTo(translogOps)); + } + } + + public void testRecoveryWithOutOfOrderDeleteWithSoftDeletes() throws Exception { + Settings settings = Settings.builder() + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 10) + // If soft-deletes is enabled, delete#1 will be reclaimed because its segment (segment_1) is fully deleted + // index#0 will be retained if merge is disabled; otherwise it will be reclaimed because gcp=3 and retained_ops=0 + .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false).build(); + try (ReplicationGroup shards = createGroup(1, settings)) { + shards.startAll(); + // create out of order delete and index op on replica + final IndexShard orgReplica = shards.getReplicas().get(0); + final String indexName = orgReplica.shardId().getIndexName(); + + // delete #1 + orgReplica.applyDeleteOperationOnReplica(1, 2, "type", "id"); + orgReplica.flush(new FlushRequest().force(true)); // isolate delete#1 in its own translog generation and lucene segment + // index #0 + orgReplica.applyIndexOperationOnReplica(0, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(indexName, "type", "id", new BytesArray("{}"), XContentType.JSON)); + // index #3 + orgReplica.applyIndexOperationOnReplica(3, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(indexName, "type", "id-3", new BytesArray("{}"), XContentType.JSON)); + // Flushing a new commit with local checkpoint=1 allows to delete the translog gen #1. + orgReplica.flush(new FlushRequest().force(true).waitIfOngoing(true)); + // index #2 + orgReplica.applyIndexOperationOnReplica(2, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(indexName, "type", "id-2", new BytesArray("{}"), XContentType.JSON)); + orgReplica.updateGlobalCheckpointOnReplica(3L, "test"); + // index #5 -> force NoOp #4. + orgReplica.applyIndexOperationOnReplica(5, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(indexName, "type", "id-5", new BytesArray("{}"), XContentType.JSON)); + + if (randomBoolean()) { + if (randomBoolean()) { + logger.info("--> flushing shard (translog/soft-deletes will be trimmed)"); + IndexMetaData.Builder builder = IndexMetaData.builder(orgReplica.indexSettings().getIndexMetaData()); + builder.settings(Settings.builder().put(orgReplica.indexSettings().getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0)); + orgReplica.indexSettings().updateIndexMetaData(builder.build()); + orgReplica.onSettingsChanged(); + } + flushShard(orgReplica); + } + + final IndexShard orgPrimary = shards.getPrimary(); + shards.promoteReplicaToPrimary(orgReplica).get(); // wait for primary/replica sync to make sure seq# gap is closed. + + IndexShard newReplica = shards.addReplicaWithExistingPath(orgPrimary.shardPath(), orgPrimary.routingEntry().currentNodeId()); + shards.recoverReplica(newReplica); + shards.assertAllEqual(3); + try (Translog.Snapshot snapshot = newReplica.getHistoryOperations("test", 0)) { + assertThat(snapshot, SnapshotMatchers.size(6)); + } } } @@ -222,7 +280,8 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(newReplica); // file based recovery should be made assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty())); - assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); + boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); + assertThat(getTranslog(newReplica).totalOperations(), equalTo(softDeletesEnabled ? nonFlushedDocs : numDocs)); // history uuid was restored assertThat(newReplica.getHistoryUUID(), equalTo(historyUUID)); @@ -326,7 +385,8 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(replica); // Make sure the flushing will eventually be completed (eg. `shouldPeriodicallyFlush` is false) assertBusy(() -> assertThat(getEngine(replica).shouldPeriodicallyFlush(), equalTo(false))); - assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); + boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); + assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? 0 : numDocs)); shards.assertAllEqual(numDocs); } } diff --git a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java index fa591411bba..ce162b9600c 100644 --- a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java @@ -43,6 +43,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexModule; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.MergeSchedulerConfig; @@ -50,6 +51,7 @@ import org.elasticsearch.index.VersionType; import org.elasticsearch.index.cache.query.QueryCacheStats; import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesQueryCache; import org.elasticsearch.indices.IndicesRequestCache; @@ -69,6 +71,7 @@ import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.Random; +import java.util.Set; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; @@ -115,6 +118,7 @@ public class IndexStatsIT extends ESIntegTestCase { return Settings.builder().put(super.indexSettings()) .put(IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.getKey(), true) .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) .build(); } @@ -1006,10 +1010,15 @@ public class IndexStatsIT extends ESIntegTestCase { @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/32506") public void testFilterCacheStats() throws Exception { - assertAcked(prepareCreate("index").setSettings(Settings.builder().put(indexSettings()).put("number_of_replicas", 0).build()).get()); - indexRandom(true, + Settings settings = Settings.builder().put(indexSettings()).put("number_of_replicas", 0).build(); + assertAcked(prepareCreate("index").setSettings(settings).get()); + indexRandom(false, true, client().prepareIndex("index", "type", "1").setSource("foo", "bar"), client().prepareIndex("index", "type", "2").setSource("foo", "baz")); + if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { + persistGlobalCheckpoint("index"); // Need to persist the global checkpoint for the soft-deletes retention MP. + } + refresh(); ensureGreen(); IndicesStatsResponse response = client().admin().indices().prepareStats("index").setQueryCache(true).get(); @@ -1040,6 +1049,13 @@ public class IndexStatsIT extends ESIntegTestCase { assertEquals(DocWriteResponse.Result.DELETED, client().prepareDelete("index", "type", "1").get().getResult()); assertEquals(DocWriteResponse.Result.DELETED, client().prepareDelete("index", "type", "2").get().getResult()); + // Here we are testing that a fully deleted segment should be dropped and its cached is evicted. + // In order to instruct the merge policy not to keep a fully deleted segment, + // we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything. + if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { + persistGlobalCheckpoint("index"); + flush("index"); + } refresh(); response = client().admin().indices().prepareStats("index").setQueryCache(true).get(); assertCumulativeQueryCacheStats(response); @@ -1173,4 +1189,21 @@ public class IndexStatsIT extends ESIntegTestCase { assertThat(executionFailures.get(), emptyCollectionOf(Exception.class)); } + + /** + * Persist the global checkpoint on all shards of the given index into disk. + * This makes sure that the persisted global checkpoint on those shards will equal to the in-memory value. + */ + private void persistGlobalCheckpoint(String index) throws Exception { + final Set nodes = internalCluster().nodesInclude(index); + for (String node : nodes) { + final IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node); + for (IndexService indexService : indexServices) { + for (IndexShard indexShard : indexService) { + indexShard.sync(); + assertThat(indexShard.getLastSyncedGlobalCheckpoint(), equalTo(indexShard.getGlobalCheckpoint())); + } + } + } + } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java b/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java index 23c56688e00..c25cad61e07 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -27,6 +27,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.snapshots.mockstore.MockRepository; import org.elasticsearch.test.ESIntegTestCase; +import org.junit.After; import java.io.IOException; import java.nio.file.FileVisitResult; @@ -58,6 +59,11 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase { return Arrays.asList(MockRepository.Plugin.class); } + @After + public void assertConsistentHistoryInLuceneIndex() throws Exception { + internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); + } + public static long getFailureCount(String repository) { long failureCount = 0; for (RepositoriesService repositoriesService : diff --git a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index 1230d594b98..632a1ecbee1 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -122,6 +122,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF import static org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.IndexSettings.INDEX_REFRESH_INTERVAL_SETTING; +import static org.elasticsearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; import static org.elasticsearch.index.query.QueryBuilders.matchQuery; import static org.elasticsearch.index.shard.IndexShardTests.getEngineFromShard; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -2048,7 +2049,9 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES))); // only one shard - assertAcked(prepareCreate("test").setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1))); + final Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).build(); + assertAcked(prepareCreate("test").setSettings(indexSettings)); ensureGreen(); logger.info("--> indexing"); @@ -2094,7 +2097,13 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas SnapshotStatus snapshotStatus = client.admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("test-2").get().getSnapshots().get(0); List shards = snapshotStatus.getShards(); for (SnapshotIndexShardStatus status : shards) { - assertThat(status.getStats().getProcessedFileCount(), equalTo(2)); // we flush before the snapshot such that we have to process the segments_N files plus the .del file + // we flush before the snapshot such that we have to process the segments_N files plus the .del file + if (INDEX_SOFT_DELETES_SETTING.get(indexSettings)) { + // soft-delete generates DV files. + assertThat(status.getStats().getProcessedFileCount(), greaterThan(2)); + } else { + assertThat(status.getStats().getProcessedFileCount(), equalTo(2)); + } } } } diff --git a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java index caf4f725fa4..588118db4ae 100644 --- a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java +++ b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.VersionType; @@ -785,4 +786,26 @@ public class SimpleVersioningIT extends ESIntegTestCase { .getVersion(), equalTo(-1L)); } + + public void testSpecialVersioning() { + internalCluster().ensureAtLeastNumDataNodes(2); + createIndex("test", Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).build()); + IndexResponse doc1 = client().prepareIndex("test", "type", "1").setSource("field", "value1") + .setVersion(0).setVersionType(VersionType.EXTERNAL).execute().actionGet(); + assertThat(doc1.getVersion(), equalTo(0L)); + IndexResponse doc2 = client().prepareIndex("test", "type", "1").setSource("field", "value2") + .setVersion(Versions.MATCH_ANY).setVersionType(VersionType.INTERNAL).execute().actionGet(); + assertThat(doc2.getVersion(), equalTo(1L)); + client().prepareDelete("test", "type", "1").get(); //v2 + IndexResponse doc3 = client().prepareIndex("test", "type", "1").setSource("field", "value3") + .setVersion(Versions.MATCH_DELETED).setVersionType(VersionType.INTERNAL).execute().actionGet(); + assertThat(doc3.getVersion(), equalTo(3L)); + IndexResponse doc4 = client().prepareIndex("test", "type", "1").setSource("field", "value4") + .setVersion(4L).setVersionType(VersionType.EXTERNAL_GTE).execute().actionGet(); + assertThat(doc4.getVersion(), equalTo(4L)); + // Make sure that these versions are replicated correctly + client().admin().indices().prepareUpdateSettings("test") + .setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)).get(); + ensureGreen("test"); + } } 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 b5ba5f18b39..b558cd1ba90 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 @@ -19,14 +19,18 @@ package org.elasticsearch.index.engine; +import org.apache.logging.log4j.Logger; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.StoredField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.Term; @@ -34,32 +38,41 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.search.Sort; +import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TotalHitCountCollector; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.MapperTestUtils; +import org.elasticsearch.index.VersionType; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -80,17 +93,30 @@ import org.junit.Before; import java.io.IOException; import java.nio.charset.Charset; import java.nio.file.Path; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; +import java.util.function.Function; import java.util.function.LongSupplier; import java.util.function.ToLongBiFunction; +import java.util.stream.Collectors; import static java.util.Collections.emptyList; +import static java.util.Collections.shuffle; +import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; +import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; public abstract class EngineTestCase extends ESTestCase { @@ -128,6 +154,20 @@ public abstract class EngineTestCase extends ESTestCase { } } + protected Settings indexSettings() { + // TODO randomize more settings + return Settings.builder() + .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), "1h") // make sure this doesn't kick in on us + .put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName) + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), + between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), + randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) + .build(); + } + @Override @Before public void setUp() throws Exception { @@ -142,13 +182,7 @@ public abstract class EngineTestCase extends ESTestCase { } else { codecName = "default"; } - defaultSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder() - .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), "1h") // make sure this doesn't kick in on us - .put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName) - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), - between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) - .build()); // TODO randomize more settings + defaultSettings = IndexSettingsModule.newIndexSettings("test", indexSettings()); threadPool = new TestThreadPool(getClass().getName()); store = createStore(); storeReplica = createStore(); @@ -180,7 +214,7 @@ public abstract class EngineTestCase extends ESTestCase { new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService(), globalCheckpointSupplier, config.getPrimaryTermSupplier()); + config.getCircuitBreakerService(), globalCheckpointSupplier, config.getPrimaryTermSupplier(), tombstoneDocSupplier()); } public EngineConfig copy(EngineConfig config, Analyzer analyzer) { @@ -189,7 +223,18 @@ public abstract class EngineTestCase extends ESTestCase { new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier()); + config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier(), + config.getTombstoneDocSupplier()); + } + + public EngineConfig copy(EngineConfig config, MergePolicy mergePolicy) { + return new EngineConfig(config.getShardId(), config.getAllocationId(), config.getThreadPool(), config.getIndexSettings(), + config.getWarmer(), config.getStore(), mergePolicy, config.getAnalyzer(), config.getSimilarity(), + new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), + config.getTranslogConfig(), config.getFlushMergesAfter(), + config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), + config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier(), + config.getTombstoneDocSupplier()); } @Override @@ -198,9 +243,11 @@ public abstract class EngineTestCase extends ESTestCase { super.tearDown(); if (engine != null && engine.isClosed.get() == false) { engine.getTranslog().getDeletionPolicy().assertNoOpenTranslogRefs(); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, createMapperService("test")); } if (replicaEngine != null && replicaEngine.isClosed.get() == false) { replicaEngine.getTranslog().getDeletionPolicy().assertNoOpenTranslogRefs(); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(replicaEngine, createMapperService("test")); } IOUtils.close( replicaEngine, storeReplica, @@ -228,8 +275,18 @@ public abstract class EngineTestCase extends ESTestCase { return testParsedDocument(id, routing, testDocumentWithTextField(), new BytesArray("{ \"value\" : \"test\" }"), null); } + public static ParsedDocument createParsedDoc(String id, String routing, boolean recoverySource) { + return testParsedDocument(id, routing, testDocumentWithTextField(), new BytesArray("{ \"value\" : \"test\" }"), null, + recoverySource); + } + protected static ParsedDocument testParsedDocument( String id, String routing, ParseContext.Document document, BytesReference source, Mapping mappingUpdate) { + return testParsedDocument(id, routing, document, source, mappingUpdate, false); + } + protected static ParsedDocument testParsedDocument( + String id, String routing, ParseContext.Document document, BytesReference source, Mapping mappingUpdate, + boolean recoverySource) { Field uidField = new Field("_id", Uid.encodeId(id), IdFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); @@ -239,11 +296,57 @@ public abstract class EngineTestCase extends ESTestCase { document.add(seqID.seqNoDocValue); document.add(seqID.primaryTerm); BytesRef ref = source.toBytesRef(); - document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length)); + if (recoverySource) { + document.add(new StoredField(SourceFieldMapper.RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); + document.add(new NumericDocValuesField(SourceFieldMapper.RECOVERY_SOURCE_NAME, 1)); + } else { + document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length)); + } return new ParsedDocument(versionField, seqID, id, "test", routing, Arrays.asList(document), source, XContentType.JSON, mappingUpdate); } + /** + * Creates a tombstone document that only includes uid, seq#, term and version fields. + */ + public static EngineConfig.TombstoneDocSupplier tombstoneDocSupplier(){ + return new EngineConfig.TombstoneDocSupplier() { + @Override + public ParsedDocument newDeleteTombstoneDoc(String type, String id) { + final ParseContext.Document doc = new ParseContext.Document(); + Field uidField = new Field(IdFieldMapper.NAME, Uid.encodeId(id), IdFieldMapper.Defaults.FIELD_TYPE); + doc.add(uidField); + Field versionField = new NumericDocValuesField(VersionFieldMapper.NAME, 0); + doc.add(versionField); + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); + doc.add(seqID.seqNo); + doc.add(seqID.seqNoDocValue); + doc.add(seqID.primaryTerm); + seqID.tombstoneField.setLongValue(1); + doc.add(seqID.tombstoneField); + return new ParsedDocument(versionField, seqID, id, type, null, + Collections.singletonList(doc), new BytesArray("{}"), XContentType.JSON, null); + } + + @Override + public ParsedDocument newNoopTombstoneDoc(String reason) { + final ParseContext.Document doc = new ParseContext.Document(); + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); + doc.add(seqID.seqNo); + doc.add(seqID.seqNoDocValue); + doc.add(seqID.primaryTerm); + seqID.tombstoneField.setLongValue(1); + doc.add(seqID.tombstoneField); + Field versionField = new NumericDocValuesField(VersionFieldMapper.NAME, 0); + doc.add(versionField); + BytesRef byteRef = new BytesRef(reason); + doc.add(new StoredField(SourceFieldMapper.NAME, byteRef.bytes, byteRef.offset, byteRef.length)); + return new ParsedDocument(versionField, seqID, null, null, null, + Collections.singletonList(doc), null, XContentType.JSON, null); + } + }; + } + protected Store createStore() throws IOException { return createStore(newDirectory()); } @@ -461,7 +564,7 @@ public abstract class EngineTestCase extends ESTestCase { new NoneCircuitBreakerService(), globalCheckpointSupplier == null ? new ReplicationTracker(shardId, allocationId.getId(), indexSettings, SequenceNumbers.NO_OPS_PERFORMED, update -> {}) : - globalCheckpointSupplier, primaryTerm::get); + globalCheckpointSupplier, primaryTerm::get, tombstoneDocSupplier()); return config; } @@ -474,7 +577,7 @@ public abstract class EngineTestCase extends ESTestCase { return new BytesArray(string.getBytes(Charset.defaultCharset())); } - protected Term newUid(String id) { + protected static Term newUid(String id) { return new Term("_id", Uid.encodeId(id)); } @@ -499,6 +602,279 @@ public abstract class EngineTestCase extends ESTestCase { protected Engine.Delete replicaDeleteForDoc(String id, long version, long seqNo, long startTime) { return new Engine.Delete("test", id, newUid(id), seqNo, 1, version, null, Engine.Operation.Origin.REPLICA, startTime); } + protected static void assertVisibleCount(InternalEngine engine, int numDocs) throws IOException { + assertVisibleCount(engine, numDocs, true); + } + + protected static void assertVisibleCount(InternalEngine engine, int numDocs, boolean refresh) throws IOException { + if (refresh) { + engine.refresh("test"); + } + try (Engine.Searcher searcher = engine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new MatchAllDocsQuery(), collector); + assertThat(collector.getTotalHits(), equalTo(numDocs)); + } + } + + public static List generateSingleDocHistory(boolean forReplica, VersionType versionType, + long primaryTerm, int minOpCount, int maxOpCount, String docId) { + final int numOfOps = randomIntBetween(minOpCount, maxOpCount); + final List ops = new ArrayList<>(); + final Term id = newUid(docId); + final int startWithSeqNo = 0; + final String valuePrefix = (forReplica ? "r_" : "p_" ) + docId + "_"; + final boolean incrementTermWhenIntroducingSeqNo = randomBoolean(); + for (int i = 0; i < numOfOps; i++) { + final Engine.Operation op; + final long version; + switch (versionType) { + case INTERNAL: + version = forReplica ? i : Versions.MATCH_ANY; + break; + case EXTERNAL: + version = i; + break; + case EXTERNAL_GTE: + version = randomBoolean() ? Math.max(i - 1, 0) : i; + break; + case FORCE: + version = randomNonNegativeLong(); + break; + default: + throw new UnsupportedOperationException("unknown version type: " + versionType); + } + if (randomBoolean()) { + op = new Engine.Index(id, testParsedDocument(docId, null, testDocumentWithTextField(valuePrefix + i), B_1, null), + forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, + version, + forReplica ? null : versionType, + forReplica ? REPLICA : PRIMARY, + System.currentTimeMillis(), -1, false + ); + } else { + op = new Engine.Delete("test", docId, id, + forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, + version, + forReplica ? null : versionType, + forReplica ? REPLICA : PRIMARY, + System.currentTimeMillis()); + } + ops.add(op); + } + return ops; + } + + public static void assertOpsOnReplica( + final List ops, + final InternalEngine replicaEngine, + boolean shuffleOps, + final Logger logger) throws IOException { + final Engine.Operation lastOp = ops.get(ops.size() - 1); + final String lastFieldValue; + if (lastOp instanceof Engine.Index) { + Engine.Index index = (Engine.Index) lastOp; + lastFieldValue = index.docs().get(0).get("value"); + } else { + // delete + lastFieldValue = null; + } + if (shuffleOps) { + int firstOpWithSeqNo = 0; + while (firstOpWithSeqNo < ops.size() && ops.get(firstOpWithSeqNo).seqNo() < 0) { + firstOpWithSeqNo++; + } + // shuffle ops but make sure legacy ops are first + shuffle(ops.subList(0, firstOpWithSeqNo), random()); + shuffle(ops.subList(firstOpWithSeqNo, ops.size()), random()); + } + boolean firstOp = true; + for (Engine.Operation op : ops) { + logger.info("performing [{}], v [{}], seq# [{}], term [{}]", + op.operationType().name().charAt(0), op.version(), op.seqNo(), op.primaryTerm()); + if (op instanceof Engine.Index) { + Engine.IndexResult result = replicaEngine.index((Engine.Index) op); + // replicas don't really care to about creation status of documents + // this allows to ignore the case where a document was found in the live version maps in + // a delete state and return false for the created flag in favor of code simplicity + // as deleted or not. This check is just signal regression so a decision can be made if it's + // intentional + assertThat(result.isCreated(), equalTo(firstOp)); + assertThat(result.getVersion(), equalTo(op.version())); + assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); + + } else { + Engine.DeleteResult result = replicaEngine.delete((Engine.Delete) op); + // Replicas don't really care to about found status of documents + // this allows to ignore the case where a document was found in the live version maps in + // a delete state and return true for the found flag in favor of code simplicity + // his check is just signal regression so a decision can be made if it's + // intentional + assertThat(result.isFound(), equalTo(firstOp == false)); + assertThat(result.getVersion(), equalTo(op.version())); + assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); + } + if (randomBoolean()) { + replicaEngine.refresh("test"); + } + if (randomBoolean()) { + replicaEngine.flush(); + replicaEngine.refresh("test"); + } + firstOp = false; + } + + assertVisibleCount(replicaEngine, lastFieldValue == null ? 0 : 1); + if (lastFieldValue != null) { + try (Engine.Searcher searcher = replicaEngine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); + assertThat(collector.getTotalHits(), equalTo(1)); + } + } + } + + protected void concurrentlyApplyOps(List ops, InternalEngine engine) throws InterruptedException { + Thread[] thread = new Thread[randomIntBetween(3, 5)]; + CountDownLatch startGun = new CountDownLatch(thread.length); + AtomicInteger offset = new AtomicInteger(-1); + for (int i = 0; i < thread.length; i++) { + thread[i] = new Thread(() -> { + startGun.countDown(); + try { + startGun.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + int docOffset; + while ((docOffset = offset.incrementAndGet()) < ops.size()) { + try { + final Engine.Operation op = ops.get(docOffset); + if (op instanceof Engine.Index) { + engine.index((Engine.Index) op); + } else if (op instanceof Engine.Delete){ + engine.delete((Engine.Delete) op); + } else { + engine.noOp((Engine.NoOp) op); + } + if ((docOffset + 1) % 4 == 0) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(); + } + } catch (IOException e) { + throw new AssertionError(e); + } + } + }); + thread[i].start(); + } + for (int i = 0; i < thread.length; i++) { + thread[i].join(); + } + } + + /** + * Gets all docId from the given engine. + */ + public static Set getDocIds(Engine engine, boolean refresh) throws IOException { + if (refresh) { + engine.refresh("test_get_doc_ids"); + } + try (Engine.Searcher searcher = engine.acquireSearcher("test_get_doc_ids")) { + Set ids = new HashSet<>(); + for (LeafReaderContext leafContext : searcher.reader().leaves()) { + LeafReader reader = leafContext.reader(); + Bits liveDocs = reader.getLiveDocs(); + for (int i = 0; i < reader.maxDoc(); i++) { + if (liveDocs == null || liveDocs.get(i)) { + Document uuid = reader.document(i, Collections.singleton(IdFieldMapper.NAME)); + BytesRef binaryID = uuid.getBinaryValue(IdFieldMapper.NAME); + ids.add(Uid.decodeId(Arrays.copyOfRange(binaryID.bytes, binaryID.offset, binaryID.offset + binaryID.length))); + } + } + } + return ids; + } + } + + /** + * Reads all engine operations that have been processed by the engine from Lucene index. + * The returned operations are sorted and de-duplicated, thus each sequence number will be have at most one operation. + */ + public static List readAllOperationsInLucene(Engine engine, MapperService mapper) throws IOException { + final List operations = new ArrayList<>(); + long maxSeqNo = Math.max(0, ((InternalEngine)engine).getLocalCheckpointTracker().getMaxSeqNo()); + try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", mapper, 0, maxSeqNo, false)) { + Translog.Operation op; + while ((op = snapshot.next()) != null){ + operations.add(op); + } + } + return operations; + } + + /** + * Asserts the provided engine has a consistent document history between translog and Lucene index. + */ + public static void assertConsistentHistoryBetweenTranslogAndLuceneIndex(Engine engine, MapperService mapper) throws IOException { + if (mapper.documentMapper() == null || engine.config().getIndexSettings().isSoftDeleteEnabled() == false) { + return; + } + final long maxSeqNo = ((InternalEngine) engine).getLocalCheckpointTracker().getMaxSeqNo(); + if (maxSeqNo < 0) { + return; // nothing to check + } + final Map translogOps = new HashMap<>(); + try (Translog.Snapshot snapshot = EngineTestCase.getTranslog(engine).newSnapshot()) { + Translog.Operation op; + while ((op = snapshot.next()) != null) { + translogOps.put(op.seqNo(), op); + } + } + final Map luceneOps = readAllOperationsInLucene(engine, mapper).stream() + .collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); + final long globalCheckpoint = EngineTestCase.getTranslog(engine).getLastSyncedGlobalCheckpoint(); + final long retainedOps = engine.config().getIndexSettings().getSoftDeleteRetentionOperations(); + final long seqNoForRecovery; + try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { + seqNoForRecovery = Long.parseLong(safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1; + } + final long minSeqNoToRetain = Math.min(seqNoForRecovery, globalCheckpoint + 1 - retainedOps); + for (Translog.Operation translogOp : translogOps.values()) { + final Translog.Operation luceneOp = luceneOps.get(translogOp.seqNo()); + if (luceneOp == null) { + if (minSeqNoToRetain <= translogOp.seqNo() && translogOp.seqNo() <= maxSeqNo) { + fail("Operation not found seq# [" + translogOp.seqNo() + "], global checkpoint [" + globalCheckpoint + "], " + + "retention policy [" + retainedOps + "], maxSeqNo [" + maxSeqNo + "], translog op [" + translogOp + "]"); + } else { + continue; + } + } + assertThat(luceneOp, notNullValue()); + assertThat(luceneOp.toString(), luceneOp.primaryTerm(), equalTo(translogOp.primaryTerm())); + assertThat(luceneOp.opType(), equalTo(translogOp.opType())); + if (luceneOp.opType() == Translog.Operation.Type.INDEX) { + assertThat(luceneOp.getSource().source, equalTo(translogOp.getSource().source)); + } + } + } + + protected MapperService createMapperService(String type) throws IOException { + IndexMetaData indexMetaData = IndexMetaData.builder("test") + .settings(Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)) + .putMapping(type, "{\"properties\": {}}") + .build(); + MapperService mapperService = MapperTestUtils.newMapperService(new NamedXContentRegistry(ClusterModule.getNamedXWriteables()), + createTempDir(), Settings.EMPTY, "test"); + mapperService.merge(indexMetaData, MapperService.MergeReason.MAPPING_UPDATE); + return mapperService; + } /** * Exposes a translog associated with the given engine for testing purpose. diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 3f1f5daf514..f2afdff9c3a 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -60,6 +60,7 @@ import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; @@ -99,10 +100,14 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase protected final Index index = new Index("test", "uuid"); private final ShardId shardId = new ShardId(index, 0); - private final Map indexMapping = Collections.singletonMap("type", "{ \"type\": {} }"); + protected final Map indexMapping = Collections.singletonMap("type", "{ \"type\": {} }"); protected ReplicationGroup createGroup(int replicas) throws IOException { - IndexMetaData metaData = buildIndexMetaData(replicas); + return createGroup(replicas, Settings.EMPTY); + } + + protected ReplicationGroup createGroup(int replicas, Settings settings) throws IOException { + IndexMetaData metaData = buildIndexMetaData(replicas, settings, indexMapping); return new ReplicationGroup(metaData); } @@ -111,9 +116,17 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } protected IndexMetaData buildIndexMetaData(int replicas, Map mappings) throws IOException { + return buildIndexMetaData(replicas, Settings.EMPTY, mappings); + } + + protected IndexMetaData buildIndexMetaData(int replicas, Settings indexSettings, Map mappings) throws IOException { Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, replicas) .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), + randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) + .put(indexSettings) .build(); IndexMetaData.Builder metaData = IndexMetaData.builder(index.getName()) .settings(settings) @@ -146,7 +159,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } }); - ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { + protected ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { final ShardRouting primaryRouting = this.createShardRouting("s0", true); primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting), () -> {}); replicas = new CopyOnWriteArrayList<>(); @@ -448,7 +461,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } } - abstract class ReplicationAction, + protected abstract class ReplicationAction, ReplicaRequest extends ReplicationRequest, Response extends ReplicationResponse> { private final Request request; @@ -456,7 +469,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase private final ReplicationGroup replicationGroup; private final String opType; - ReplicationAction(Request request, ActionListener listener, ReplicationGroup group, String opType) { + protected ReplicationAction(Request request, ActionListener listener, ReplicationGroup group, String opType) { this.request = request; this.listener = listener; this.replicationGroup = group; @@ -582,11 +595,11 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } } - class PrimaryResult implements ReplicationOperation.PrimaryResult { + protected class PrimaryResult implements ReplicationOperation.PrimaryResult { final ReplicaRequest replicaRequest; final Response finalResponse; - PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) { + public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) { this.replicaRequest = replicaRequest; this.finalResponse = finalResponse; } diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index d2a84589669..2f4a3dfd6c1 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -18,13 +18,8 @@ */ package org.elasticsearch.index.shard; -import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexNotFoundException; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.store.Directory; -import org.apache.lucene.util.Bits; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.index.IndexRequest; @@ -57,10 +52,8 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngineFactory; -import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.SourceToParse; -import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.similarity.SimilarityService; @@ -180,37 +173,63 @@ public abstract class IndexShardTestCase extends ESTestCase { } /** - * creates a new initializing shard. The shard will have its own unique data path. + * Creates a new initializing shard. The shard will have its own unique data path. * - * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica - * (ready to recover from another shard) + * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica (ready to recover from + * another shard) */ protected IndexShard newShard(boolean primary) throws IOException { - ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), primary, - ShardRoutingState.INITIALIZING, - primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); - return newShard(shardRouting); + return newShard(primary, Settings.EMPTY, new InternalEngineFactory()); } /** - * creates a new initializing shard. The shard will have its own unique data path. + * Creates a new initializing shard. The shard will have its own unique data path. * - * @param shardRouting the {@link ShardRouting} to use for this shard - * @param listeners an optional set of listeners to add to the shard + * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica (ready to recover from + * another shard) + * @param settings the settings to use for this shard + * @param engineFactory the engine factory to use for this shard + */ + protected IndexShard newShard(boolean primary, Settings settings, EngineFactory engineFactory) throws IOException { + final RecoverySource recoverySource = + primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE; + final ShardRouting shardRouting = + TestShardRouting.newShardRouting( + new ShardId("index", "_na_", 0), randomAlphaOfLength(10), primary, ShardRoutingState.INITIALIZING, recoverySource); + return newShard(shardRouting, settings, engineFactory); + } + + protected IndexShard newShard(ShardRouting shardRouting, final IndexingOperationListener... listeners) throws IOException { + return newShard(shardRouting, Settings.EMPTY, new InternalEngineFactory(), listeners); + } + + /** + * Creates a new initializing shard. The shard will have its own unique data path. + * + * @param shardRouting the {@link ShardRouting} to use for this shard + * @param settings the settings to use for this shard + * @param engineFactory the engine factory to use for this shard + * @param listeners an optional set of listeners to add to the shard */ protected IndexShard newShard( final ShardRouting shardRouting, + final Settings settings, + final EngineFactory engineFactory, final IndexingOperationListener... listeners) throws IOException { assert shardRouting.initializing() : shardRouting; - Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .build(); + Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), + randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) + .put(settings) + .build(); IndexMetaData.Builder metaData = IndexMetaData.builder(shardRouting.getIndexName()) - .settings(settings) + .settings(indexSettings) .primaryTerm(0, primaryTerm) .putMapping("_doc", "{ \"properties\": {} }"); - return newShard(shardRouting, metaData.build(), listeners); + return newShard(shardRouting, metaData.build(), engineFactory, listeners); } /** @@ -225,7 +244,7 @@ public abstract class IndexShardTestCase extends ESTestCase { ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, randomAlphaOfLength(5), primary, ShardRoutingState.INITIALIZING, primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); - return newShard(shardRouting, listeners); + return newShard(shardRouting, Settings.EMPTY, new InternalEngineFactory(), listeners); } /** @@ -265,9 +284,10 @@ public abstract class IndexShardTestCase extends ESTestCase { * @param indexMetaData indexMetaData for the shard, including any mapping * @param listeners an optional set of listeners to add to the shard */ - protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, IndexingOperationListener... listeners) + protected IndexShard newShard( + ShardRouting routing, IndexMetaData indexMetaData, EngineFactory engineFactory, IndexingOperationListener... listeners) throws IOException { - return newShard(routing, indexMetaData, null, new InternalEngineFactory(), () -> {}, listeners); + return newShard(routing, indexMetaData, null, engineFactory, () -> {}, listeners); } /** @@ -372,19 +392,39 @@ public abstract class IndexShardTestCase extends ESTestCase { } /** - * creates a new empyu shard and starts it. The shard will be either a replica or a primary. + * Creates a new empty shard and starts it. The shard will randomly be a replica or a primary. */ protected IndexShard newStartedShard() throws IOException { return newStartedShard(randomBoolean()); } /** - * creates a new empty shard and starts it. + * Creates a new empty shard and starts it + * @param settings the settings to use for this shard + */ + protected IndexShard newStartedShard(Settings settings) throws IOException { + return newStartedShard(randomBoolean(), settings, new InternalEngineFactory()); + } + + /** + * Creates a new empty shard and starts it. * * @param primary controls whether the shard will be a primary or a replica. */ - protected IndexShard newStartedShard(boolean primary) throws IOException { - IndexShard shard = newShard(primary); + protected IndexShard newStartedShard(final boolean primary) throws IOException { + return newStartedShard(primary, Settings.EMPTY, new InternalEngineFactory()); + } + + /** + * Creates a new empty shard with the specified settings and engine factory and starts it. + * + * @param primary controls whether the shard will be a primary or a replica. + * @param settings the settings to use for this shard + * @param engineFactory the engine factory to use for this shard + */ + protected IndexShard newStartedShard( + final boolean primary, final Settings settings, final EngineFactory engineFactory) throws IOException { + IndexShard shard = newShard(primary, settings, engineFactory); if (primary) { recoverShardFromStore(shard); } else { @@ -401,6 +441,7 @@ public abstract class IndexShardTestCase extends ESTestCase { for (IndexShard shard : shards) { if (shard != null) { try { + assertConsistentHistoryBetweenTranslogAndLucene(shard); shard.close("test", false); } finally { IOUtils.close(shard.store()); @@ -582,22 +623,7 @@ public abstract class IndexShardTestCase extends ESTestCase { } protected Set getShardDocUIDs(final IndexShard shard) throws IOException { - shard.refresh("get_uids"); - try (Engine.Searcher searcher = shard.acquireSearcher("test")) { - Set ids = new HashSet<>(); - for (LeafReaderContext leafContext : searcher.reader().leaves()) { - LeafReader reader = leafContext.reader(); - Bits liveDocs = reader.getLiveDocs(); - for (int i = 0; i < reader.maxDoc(); i++) { - if (liveDocs == null || liveDocs.get(i)) { - Document uuid = reader.document(i, Collections.singleton(IdFieldMapper.NAME)); - BytesRef binaryID = uuid.getBinaryValue(IdFieldMapper.NAME); - ids.add(Uid.decodeId(Arrays.copyOfRange(binaryID.bytes, binaryID.offset, binaryID.offset + binaryID.length))); - } - } - } - return ids; - } + return EngineTestCase.getDocIds(shard.getEngine(), true); } protected void assertDocCount(IndexShard shard, int docDount) throws IOException { @@ -610,6 +636,12 @@ public abstract class IndexShardTestCase extends ESTestCase { assertThat(shardDocUIDs, hasSize(ids.length)); } + public static void assertConsistentHistoryBetweenTranslogAndLucene(IndexShard shard) throws IOException { + final Engine engine = shard.getEngineOrNull(); + if (engine != null) { + EngineTestCase.assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, shard.mapperService()); + } + } protected Engine.IndexResult indexDoc(IndexShard shard, String type, String id) throws IOException { return indexDoc(shard, type, id, "{}"); @@ -653,11 +685,14 @@ public abstract class IndexShardTestCase extends ESTestCase { } protected Engine.DeleteResult deleteDoc(IndexShard shard, String type, String id) throws IOException { + final Engine.DeleteResult result; if (shard.routingEntry().primary()) { - return shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, type, id, VersionType.INTERNAL); + result = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, type, id, VersionType.INTERNAL); + shard.updateLocalCheckpointForShard(shard.routingEntry().allocationId().getId(), shard.getEngine().getLocalCheckpoint()); } else { - return shard.applyDeleteOperationOnReplica(shard.seqNoStats().getMaxSeqNo() + 1, 0L, type, id); + result = shard.applyDeleteOperationOnReplica(shard.seqNoStats().getMaxSeqNo() + 1, 0L, type, id); } + return result; } protected void flushShard(IndexShard shard) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 322e2a128c9..be9e40ab420 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -723,6 +723,10 @@ public abstract class ESIntegTestCase extends ESTestCase { } // always default delayed allocation to 0 to make sure we have tests are not delayed builder.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0); + builder.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()); + if (randomBoolean()) { + builder.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), between(0, 1000)); + } return builder.build(); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java index 9633f56dea9..19290f8cf11 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java @@ -41,6 +41,7 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.node.MockNode; @@ -87,6 +88,14 @@ public abstract class ESSingleNodeTestCase extends ESTestCase { .setOrder(0) .setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)).get(); + client().admin().indices() + .preparePutTemplate("random-soft-deletes-template") + .setPatterns(Collections.singletonList("*")) + .setOrder(0) + .setSettings(Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), + randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) + ).get(); } private static void stopNode() throws IOException { diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 306f79e5e16..4c813372fae 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -1163,6 +1163,26 @@ public final class InternalTestCluster extends TestCluster { }); } + /** + * Asserts that the document history in Lucene index is consistent with Translog's on every index shard of the cluster. + * This assertion might be expensive, thus we prefer not to execute on every test but only interesting tests. + */ + public void assertConsistentHistoryBetweenTranslogAndLuceneIndex() throws IOException { + final Collection nodesAndClients = nodes.values(); + for (NodeAndClient nodeAndClient : nodesAndClients) { + IndicesService indexServices = getInstance(IndicesService.class, nodeAndClient.name); + for (IndexService indexService : indexServices) { + for (IndexShard indexShard : indexService) { + try { + IndexShardTestCase.assertConsistentHistoryBetweenTranslogAndLucene(indexShard); + } catch (AlreadyClosedException ignored) { + // shard is closed + } + } + } + } + } + private void randomlyResetClients() throws IOException { // only reset the clients on nightly tests, it causes heavy load... if (RandomizedTest.isNightly() && rarely(random)) { From 273c82d7c9d70f3378c3e2dafcd2fab851c4308f Mon Sep 17 00:00:00 2001 From: Tim Vernum Date: Fri, 31 Aug 2018 13:25:27 +1000 Subject: [PATCH 05/16] Add support for "authorization_realms" (#33262) Authorization Realms allow an authenticating realm to delegate the task of constructing a User object (with name, roles, etc) to one or more other realms. E.g. A client could authenticate using PKI, but then delegate to an LDAP realm. The LDAP realm performs a "lookup" by principal, and then does regular role-mapping from the discovered user. This commit includes: - authorization_realm support in the pki, ldap, saml & kerberos realms - docs for authorization_realms - checks that there are no "authorization chains" (whereby "realm-a" delegates to "realm-b", but "realm-b" delegates to "realm-c") Authorization realms is a platinum feature. --- .../settings/security-settings.asciidoc | 19 ++ .../configuring-kerberos-realm.asciidoc | 5 + .../configuring-ldap-realm.asciidoc | 7 +- .../configuring-pki-realm.asciidoc | 10 +- .../configuring-saml-realm.asciidoc | 5 + .../authentication/saml-guide.asciidoc | 21 +- .../authorization/mapping-roles.asciidoc | 3 + .../authorization/run-as-privilege.asciidoc | 2 +- .../license/XPackLicenseState.java | 12 +- .../xpack/core/security/authc/Realm.java | 10 + .../authc/kerberos/KerberosRealmSettings.java | 7 +- .../authc/ldap/LdapRealmSettings.java | 2 + .../security/authc/pki/PkiRealmSettings.java | 2 + .../authc/saml/SamlRealmSettings.java | 2 + .../DelegatedAuthorizationSettings.java | 27 +++ .../security/authc/AuthenticationService.java | 41 ++-- .../xpack/security/authc/Realms.java | 1 + .../authc/kerberos/KerberosRealm.java | 56 ++++-- .../xpack/security/authc/ldap/LdapRealm.java | 95 ++++++--- .../xpack/security/authc/pki/PkiRealm.java | 53 +++-- .../xpack/security/authc/saml/SamlRealm.java | 41 +++- .../support/CachingUsernamePasswordRealm.java | 30 ++- .../DelegatedAuthorizationSupport.java | 146 ++++++++++++++ .../authc/support/RealmUserLookup.java | 63 ++++++ .../KerberosRealmAuthenticateFailedTests.java | 35 ++++ .../authc/kerberos/KerberosRealmTestCase.java | 12 ++ .../authc/kerberos/KerberosRealmTests.java | 38 ++++ .../authc/ldap/ActiveDirectoryRealmTests.java | 14 ++ .../security/authc/ldap/LdapRealmTests.java | 89 ++++++++- .../security/authc/pki/PkiRealmTests.java | 115 ++++++++--- .../security/authc/saml/SamlRealmTests.java | 131 ++++++++---- .../CachingUsernamePasswordRealmTests.java | 29 +++ .../DelegatedAuthorizationSupportTests.java | 189 ++++++++++++++++++ .../authc/support/MockLookupRealm.java | 52 +++++ .../authc/support/RealmUserLookupTests.java | 128 ++++++++++++ x-pack/qa/saml-idp-tests/build.gradle | 17 +- .../authc/saml/SamlAuthenticationIT.java | 154 ++++++++++---- 37 files changed, 1430 insertions(+), 233 deletions(-) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/DelegatedAuthorizationSettings.java create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/DelegatedAuthorizationSupport.java create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/RealmUserLookup.java create mode 100644 x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/DelegatedAuthorizationSupportTests.java create mode 100644 x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/MockLookupRealm.java create mode 100644 x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/RealmUserLookupTests.java diff --git a/docs/reference/settings/security-settings.asciidoc b/docs/reference/settings/security-settings.asciidoc index bcc00ce30c5..1fc441a0622 100644 --- a/docs/reference/settings/security-settings.asciidoc +++ b/docs/reference/settings/security-settings.asciidoc @@ -246,6 +246,13 @@ This setting is multivalued; you can specify multiple user contexts. Required to operate in user template mode. If `user_search.base_dn` is specified, this setting is not valid. For more information on the different modes, see {xpack-ref}/ldap-realm.html[LDAP realms]. + +`authorization_realms`:: +The names of the realms that should be consulted for delegate authorization. +If this setting is used, then the LDAP realm does not perform role mapping and +instead loads the user from the listed realms. The referenced realms are +consulted in the order that they are defined in this list. +See {stack-ov}/realm-chains.html#authorization_realms[Delegating authorization to another realm] + -- NOTE: If any settings starting with `user_search` are specified, the @@ -733,6 +740,12 @@ Specifies the {xpack-ref}/security-files.html[location] of the {xpack-ref}/mapping-roles.html[YAML role mapping configuration file]. Defaults to `ES_PATH_CONF/role_mapping.yml`. +`authorization_realms`:: +The names of the realms that should be consulted for delegate authorization. +If this setting is used, then the PKI realm does not perform role mapping and +instead loads the user from the listed realms. +See {stack-ov}/realm-chains.html#authorization_realms[Delegating authorization to another realm] + `cache.ttl`:: Specifies the time-to-live for cached user entries. A user and a hash of its credentials are cached for this period of time. Use the @@ -856,6 +869,12 @@ Defaults to `false`. Specifies whether to populate the {es} user's metadata with the values that are provided by the SAML attributes. Defaults to `true`. +`authorization_realms`:: +The names of the realms that should be consulted for delegate authorization. +If this setting is used, then the SAML realm does not perform role mapping and +instead loads the user from the listed realms. +See {stack-ov}/realm-chains.html#authorization_realms[Delegating authorization to another realm] + `allowed_clock_skew`:: The maximum amount of skew that can be tolerated between the IdP's clock and the {es} node's clock. diff --git a/x-pack/docs/en/security/authentication/configuring-kerberos-realm.asciidoc b/x-pack/docs/en/security/authentication/configuring-kerberos-realm.asciidoc index 30968355f3c..9e7ed476272 100644 --- a/x-pack/docs/en/security/authentication/configuring-kerberos-realm.asciidoc +++ b/x-pack/docs/en/security/authentication/configuring-kerberos-realm.asciidoc @@ -166,5 +166,10 @@ POST _xpack/security/role_mapping/kerbrolemapping // CONSOLE For more information, see {stack-ov}/mapping-roles.html[Mapping users and groups to roles]. + +NOTE: The Kerberos realm supports +{stack-ov}/realm-chains.html#authorization_realms[authorization realms] as an +alternative to role mapping. + -- diff --git a/x-pack/docs/en/security/authentication/configuring-ldap-realm.asciidoc b/x-pack/docs/en/security/authentication/configuring-ldap-realm.asciidoc index d3572ae5e1b..a5f8c3e4412 100644 --- a/x-pack/docs/en/security/authentication/configuring-ldap-realm.asciidoc +++ b/x-pack/docs/en/security/authentication/configuring-ldap-realm.asciidoc @@ -189,6 +189,11 @@ For more information, see {xpack-ref}/ldap-realm.html#mapping-roles-ldap[Mapping LDAP Groups to Roles] and {xpack-ref}/mapping-roles.html[Mapping Users and Groups to Roles]. + +NOTE: The LDAP realm supports +{stack-ov}/realm-chains.html#authorization_realms[authorization realms] as an +alternative to role mapping. + -- . (Optional) Configure the `metadata` setting on the LDAP realm to include extra @@ -211,4 +216,4 @@ xpack: type: ldap metadata: cn -------------------------------------------------- --- \ No newline at end of file +-- diff --git a/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc b/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc index acaa8429d07..9a4d5fcf18b 100644 --- a/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc +++ b/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc @@ -10,7 +10,8 @@ NOTE: You cannot use PKI certificates to authenticate users in {kib}. To use PKI in {es}, you configure a PKI realm, enable client authentication on the desired network layers (transport or http), and map the Distinguished Names -(DNs) from the user certificates to {security} roles in the role mapping file. +(DNs) from the user certificates to {security} roles in the +<> or role-mapping file. You can also use a combination of PKI and username/password authentication. For example, you can enable SSL/TLS on the transport layer and define a PKI realm to @@ -173,4 +174,9 @@ key. You can also use the authenticate API to validate your role mapping. For more information, see {xpack-ref}/mapping-roles.html[Mapping Users and Groups to Roles]. --- \ No newline at end of file + +NOTE: The PKI realm supports +{stack-ov}/realm-chains.html#authorization_realms[authorization realms] as an +alternative to role mapping. + +-- diff --git a/x-pack/docs/en/security/authentication/configuring-saml-realm.asciidoc b/x-pack/docs/en/security/authentication/configuring-saml-realm.asciidoc index cbcbeebb359..d16e1302550 100644 --- a/x-pack/docs/en/security/authentication/configuring-saml-realm.asciidoc +++ b/x-pack/docs/en/security/authentication/configuring-saml-realm.asciidoc @@ -219,6 +219,11 @@ access any data. Your SAML users cannot do anything until they are mapped to {security} roles. See {stack-ov}/saml-role-mapping.html[Configuring role mappings]. + +NOTE: The SAML realm supports +{stack-ov}/realm-chains.html#authorization_realms[authorization realms] as an +alternative to role mapping. + -- . {stack-ov}/saml-kibana.html[Configure {kib} to use SAML SSO]. diff --git a/x-pack/docs/en/security/authentication/saml-guide.asciidoc b/x-pack/docs/en/security/authentication/saml-guide.asciidoc index 4facceff81c..b0077dc1ba9 100644 --- a/x-pack/docs/en/security/authentication/saml-guide.asciidoc +++ b/x-pack/docs/en/security/authentication/saml-guide.asciidoc @@ -473,7 +473,7 @@ or separate keys used for each of those. The Elastic Stack uses X.509 certificates with RSA private keys for SAML cryptography. These keys can be generated using any standard SSL tool, including -the `elasticsearch-certutil` tool that ships with X-Pack. +the `elasticsearch-certutil` tool that ships with {xpack}. Your IdP may require that the Elastic Stack have a cryptographic key for signing SAML messages, and that you provide the corresponding signing certificate within @@ -624,9 +624,10 @@ When a user authenticates using SAML, they are identified to the Elastic Stack, but this does not automatically grant them access to perform any actions or access any data. -Your SAML users cannot do anything until they are mapped to {security} -roles. This mapping is performed through the -{ref}/security-api-put-role-mapping.html[add role mapping API]. +Your SAML users cannot do anything until they are assigned {security} +roles. This is done through either the +{ref}/security-api-put-role-mapping.html[add role mapping API], or with +<>. This is an example of a simple role mapping that grants the `kibana_user` role to any user who authenticates against the `saml1` realm: @@ -683,6 +684,18 @@ PUT /_xpack/security/role_mapping/saml-finance // CONSOLE // TEST +If your users also exist in a repository that can be directly accessed by {security} +(such as an LDAP directory) then you can use +<> instead of role mappings. + +In this case, you perform the following steps: +1. In your SAML realm, assigned a SAML attribute to act as the lookup userid, + by configuring the `attributes.principal` setting. +2. Create a new realm that can lookup users from your local repository (e.g. an + `ldap` realm) +3. In your SAML realm, set `authorization_realms` to the name of the realm you + created in step 2. + [[saml-user-metadata]] === User metadata diff --git a/x-pack/docs/en/security/authorization/mapping-roles.asciidoc b/x-pack/docs/en/security/authorization/mapping-roles.asciidoc index ecafe2bd3ec..166238c32ac 100644 --- a/x-pack/docs/en/security/authorization/mapping-roles.asciidoc +++ b/x-pack/docs/en/security/authorization/mapping-roles.asciidoc @@ -24,6 +24,9 @@ either role management method. For example, when you use the role mapping API, you are able to map users to both API-managed roles and file-managed roles (and likewise for file-based role-mappings). +NOTE: The PKI, LDAP, Kerberos and SAML realms support using +<> as an alternative to role mapping. + [[mapping-roles-api]] ==== Using the role mapping API diff --git a/x-pack/docs/en/security/authorization/run-as-privilege.asciidoc b/x-pack/docs/en/security/authorization/run-as-privilege.asciidoc index 93d11c0ab2a..8dba764cc1c 100644 --- a/x-pack/docs/en/security/authorization/run-as-privilege.asciidoc +++ b/x-pack/docs/en/security/authorization/run-as-privilege.asciidoc @@ -12,7 +12,7 @@ the realm you use to authenticate. Both the internal `native` and `file` realms support this out of the box. The LDAP realm must be configured to run in <>. The Active Directory realm must be <> to support -_run as_. The PKI realm does not support _run as_. +_run as_. The PKI, Kerberos, and SAML realms do not support _run as_. To submit requests on behalf of other users, you need to have the `run_as` permission. For example, the following role grants permission to submit request diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java index 722c9d0e711..a0dbc644922 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java @@ -410,10 +410,20 @@ public class XPackLicenseState { */ public boolean isCustomRoleProvidersAllowed() { final Status localStatus = status; - return (localStatus.mode == OperationMode.PLATINUM || localStatus.mode == OperationMode.TRIAL ) + return (localStatus.mode == OperationMode.PLATINUM || localStatus.mode == OperationMode.TRIAL) && localStatus.active; } + /** + * @return whether "authorization_realms" are allowed based on the license {@link OperationMode} + * @see org.elasticsearch.xpack.core.security.authc.support.DelegatedAuthorizationSettings + */ + public boolean isAuthorizationRealmAllowed() { + final Status localStatus = status; + return (localStatus.mode == OperationMode.PLATINUM || localStatus.mode == OperationMode.TRIAL) + && localStatus.active; + } + /** * Determine if Watcher is available based on the current license. *

diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/Realm.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/Realm.java index 2c63ca95eb9..bc8869d5d83 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/Realm.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/Realm.java @@ -8,6 +8,8 @@ package org.elasticsearch.xpack.core.security.authc; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.xpack.core.security.authc.support.DelegatedAuthorizationSettings; import org.elasticsearch.xpack.core.XPackField; import org.elasticsearch.xpack.core.security.user.User; @@ -146,6 +148,14 @@ public abstract class Realm implements Comparable { return type + "/" + config.name; } + /** + * This is no-op in the base class, but allows realms to be aware of what other realms are configured + * + * @see DelegatedAuthorizationSettings + */ + public void initialize(Iterable realms, XPackLicenseState licenseState) { + } + /** * A factory interface to construct a security realm. */ diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/kerberos/KerberosRealmSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/kerberos/KerberosRealmSettings.java index 7524ef08c1e..656632a2ec6 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/kerberos/KerberosRealmSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/kerberos/KerberosRealmSettings.java @@ -10,6 +10,7 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.xpack.core.security.authc.support.DelegatedAuthorizationSettings; import java.util.Set; @@ -44,7 +45,9 @@ public final class KerberosRealmSettings { * @return the valid set of {@link Setting}s for a {@value #TYPE} realm */ public static Set> getSettings() { - return Sets.newHashSet(HTTP_SERVICE_KEYTAB_PATH, CACHE_TTL_SETTING, CACHE_MAX_USERS_SETTING, SETTING_KRB_DEBUG_ENABLE, - SETTING_REMOVE_REALM_NAME); + final Set> settings = Sets.newHashSet(HTTP_SERVICE_KEYTAB_PATH, CACHE_TTL_SETTING, CACHE_MAX_USERS_SETTING, + SETTING_KRB_DEBUG_ENABLE, SETTING_REMOVE_REALM_NAME); + settings.addAll(DelegatedAuthorizationSettings.getSettings()); + return settings; } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/ldap/LdapRealmSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/ldap/LdapRealmSettings.java index 0bb9f195af7..3f79c722be3 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/ldap/LdapRealmSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/ldap/LdapRealmSettings.java @@ -9,6 +9,7 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.xpack.core.security.authc.ldap.support.LdapMetaDataResolverSettings; import org.elasticsearch.xpack.core.security.authc.support.CachingUsernamePasswordRealmSettings; +import org.elasticsearch.xpack.core.security.authc.support.DelegatedAuthorizationSettings; import org.elasticsearch.xpack.core.security.authc.support.mapper.CompositeRoleMapperSettings; import java.util.HashSet; @@ -37,6 +38,7 @@ public final class LdapRealmSettings { assert LDAP_TYPE.equals(type) : "type [" + type + "] is unknown. expected one of [" + AD_TYPE + ", " + LDAP_TYPE + "]"; settings.addAll(LdapSessionFactorySettings.getSettings()); settings.addAll(LdapUserSearchSessionFactorySettings.getSettings()); + settings.addAll(DelegatedAuthorizationSettings.getSettings()); } settings.addAll(LdapMetaDataResolverSettings.getSettings()); return settings; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/pki/PkiRealmSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/pki/PkiRealmSettings.java index a3539b30d3e..53af4938a8f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/pki/PkiRealmSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/pki/PkiRealmSettings.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.core.security.authc.pki; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.xpack.core.security.authc.support.DelegatedAuthorizationSettings; import org.elasticsearch.xpack.core.security.authc.support.mapper.CompositeRoleMapperSettings; import org.elasticsearch.xpack.core.ssl.SSLConfigurationSettings; @@ -43,6 +44,7 @@ public final class PkiRealmSettings { settings.add(SSL_SETTINGS.truststoreAlgorithm); settings.add(SSL_SETTINGS.caPaths); + settings.addAll(DelegatedAuthorizationSettings.getSettings()); settings.addAll(CompositeRoleMapperSettings.getSettings()); return settings; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/saml/SamlRealmSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/saml/SamlRealmSettings.java index cf28b995127..e254cee1243 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/saml/SamlRealmSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/saml/SamlRealmSettings.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.core.security.authc.saml; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.xpack.core.security.authc.support.DelegatedAuthorizationSettings; import org.elasticsearch.xpack.core.ssl.SSLConfigurationSettings; import org.elasticsearch.xpack.core.ssl.X509KeyPairSettings; @@ -89,6 +90,7 @@ public class SamlRealmSettings { set.addAll(DN_ATTRIBUTE.settings()); set.addAll(NAME_ATTRIBUTE.settings()); set.addAll(MAIL_ATTRIBUTE.settings()); + set.addAll(DelegatedAuthorizationSettings.getSettings()); return set; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/DelegatedAuthorizationSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/DelegatedAuthorizationSettings.java new file mode 100644 index 00000000000..b8384a76b41 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/DelegatedAuthorizationSettings.java @@ -0,0 +1,27 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.core.security.authc.support; + +import org.elasticsearch.common.settings.Setting; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.function.Function; + +/** + * Settings related to "Delegated Authorization" (aka Lookup Realms) + */ +public class DelegatedAuthorizationSettings { + + public static final Setting> AUTHZ_REALMS = Setting.listSetting("authorization_realms", + Collections.emptyList(), Function.identity(), Setting.Property.NodeScope); + + public static Collection> getSettings() { + return Collections.singleton(AUTHZ_REALMS); + } +} diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/AuthenticationService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/AuthenticationService.java index 85084da8464..c3888ba9453 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/AuthenticationService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/AuthenticationService.java @@ -34,10 +34,12 @@ import org.elasticsearch.xpack.core.security.user.SystemUser; import org.elasticsearch.xpack.core.security.user.User; import org.elasticsearch.xpack.security.audit.AuditTrail; import org.elasticsearch.xpack.security.audit.AuditTrailService; +import org.elasticsearch.xpack.security.authc.support.RealmUserLookup; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.function.BiConsumer; import java.util.function.Consumer; @@ -381,33 +383,18 @@ public class AuthenticationService extends AbstractComponent { * names of users that exist using a timing attack */ private void lookupRunAsUser(final User user, String runAsUsername, Consumer userConsumer) { - final List realmsList = realms.asList(); - final BiConsumer> realmLookupConsumer = (realm, lookupUserListener) -> - realm.lookupUser(runAsUsername, ActionListener.wrap((lookedupUser) -> { - if (lookedupUser != null) { - lookedupBy = new RealmRef(realm.name(), realm.type(), nodeName); - lookupUserListener.onResponse(lookedupUser); - } else { - lookupUserListener.onResponse(null); - } - }, lookupUserListener::onFailure)); - - final IteratingActionListener userLookupListener = - new IteratingActionListener<>(ActionListener.wrap((lookupUser) -> { - if (lookupUser == null) { - // the user does not exist, but we still create a User object, which will later be rejected by authz - userConsumer.accept(new User(runAsUsername, null, user)); - } else { - userConsumer.accept(new User(lookupUser, user)); - } - }, - (e) -> listener.onFailure(request.exceptionProcessingRequest(e, authenticationToken))), - realmLookupConsumer, realmsList, threadContext); - try { - userLookupListener.run(); - } catch (Exception e) { - listener.onFailure(request.exceptionProcessingRequest(e, authenticationToken)); - } + final RealmUserLookup lookup = new RealmUserLookup(realms.asList(), threadContext); + lookup.lookup(runAsUsername, ActionListener.wrap(tuple -> { + if (tuple == null) { + // the user does not exist, but we still create a User object, which will later be rejected by authz + userConsumer.accept(new User(runAsUsername, null, user)); + } else { + User foundUser = Objects.requireNonNull(tuple.v1()); + Realm realm = Objects.requireNonNull(tuple.v2()); + lookedupBy = new RealmRef(realm.name(), realm.type(), nodeName); + userConsumer.accept(new User(foundUser, user)); + } + }, exception -> listener.onFailure(request.exceptionProcessingRequest(exception, authenticationToken)))); } /** diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/Realms.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/Realms.java index 8b80c1f1d1c..d2573b9343d 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/Realms.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/Realms.java @@ -93,6 +93,7 @@ public class Realms extends AbstractComponent implements Iterable { this.standardRealmsOnly = Collections.unmodifiableList(standardRealms); this.nativeRealmsOnly = Collections.unmodifiableList(nativeRealms); + realms.forEach(r -> r.initialize(this, licenseState)); } @Override diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealm.java index d57bb3052d8..9c531d3159f 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealm.java @@ -13,6 +13,7 @@ import org.elasticsearch.common.cache.Cache; import org.elasticsearch.common.cache.CacheBuilder; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; import org.elasticsearch.xpack.core.security.authc.AuthenticationToken; @@ -21,6 +22,7 @@ import org.elasticsearch.xpack.core.security.authc.RealmConfig; import org.elasticsearch.xpack.core.security.authc.kerberos.KerberosRealmSettings; import org.elasticsearch.xpack.core.security.user.User; import org.elasticsearch.xpack.security.authc.support.CachingRealm; +import org.elasticsearch.xpack.security.authc.support.DelegatedAuthorizationSupport; import org.elasticsearch.xpack.security.authc.support.UserRoleMapper; import org.elasticsearch.xpack.security.authc.support.mapper.NativeRoleMappingStore; import org.ietf.jgss.GSSException; @@ -63,6 +65,7 @@ public final class KerberosRealm extends Realm implements CachingRealm { private final Path keytabPath; private final boolean enableKerberosDebug; private final boolean removeRealmName; + private DelegatedAuthorizationSupport delegatedRealms; public KerberosRealm(final RealmConfig config, final NativeRoleMappingStore nativeRoleMappingStore, final ThreadPool threadPool) { this(config, nativeRoleMappingStore, new KerberosTicketValidator(), threadPool, null); @@ -100,6 +103,15 @@ public final class KerberosRealm extends Realm implements CachingRealm { } this.enableKerberosDebug = KerberosRealmSettings.SETTING_KRB_DEBUG_ENABLE.get(config.settings()); this.removeRealmName = KerberosRealmSettings.SETTING_REMOVE_REALM_NAME.get(config.settings()); + this.delegatedRealms = null; + } + + @Override + public void initialize(Iterable realms, XPackLicenseState licenseState) { + if (delegatedRealms != null) { + throw new IllegalStateException("Realm has already been initialized"); + } + delegatedRealms = new DelegatedAuthorizationSupport(realms, config, licenseState); } @Override @@ -133,13 +145,14 @@ public final class KerberosRealm extends Realm implements CachingRealm { @Override public void authenticate(final AuthenticationToken token, final ActionListener listener) { + assert delegatedRealms != null : "Realm has not been initialized correctly"; assert token instanceof KerberosAuthenticationToken; final KerberosAuthenticationToken kerbAuthnToken = (KerberosAuthenticationToken) token; kerberosTicketValidator.validateTicket((byte[]) kerbAuthnToken.credentials(), keytabPath, enableKerberosDebug, ActionListener.wrap(userPrincipalNameOutToken -> { if (userPrincipalNameOutToken.v1() != null) { final String username = maybeRemoveRealmName(userPrincipalNameOutToken.v1()); - buildUser(username, userPrincipalNameOutToken.v2(), listener); + resolveUser(username, userPrincipalNameOutToken.v2(), listener); } else { /** * This is when security context could not be established may be due to ongoing @@ -192,35 +205,36 @@ public final class KerberosRealm extends Realm implements CachingRealm { } } - private void buildUser(final String username, final String outToken, final ActionListener listener) { + private void resolveUser(final String username, final String outToken, final ActionListener listener) { // if outToken is present then it needs to be communicated with peer, add it to // response header in thread context. if (Strings.hasText(outToken)) { threadPool.getThreadContext().addResponseHeader(WWW_AUTHENTICATE, NEGOTIATE_AUTH_HEADER_PREFIX + outToken); } - final User user = (userPrincipalNameToUserCache != null) ? userPrincipalNameToUserCache.get(username) : null; - if (user != null) { - /** - * TODO: bizybot If authorizing realms configured, resolve user from those - * realms and then return. - */ - listener.onResponse(AuthenticationResult.success(user)); + + if (delegatedRealms.hasDelegation()) { + delegatedRealms.resolve(username, listener); } else { - /** - * TODO: bizybot If authorizing realms configured, resolve user from those - * realms, cache it and then return. - */ - final UserRoleMapper.UserData userData = new UserRoleMapper.UserData(username, null, Collections.emptySet(), null, this.config); - userRoleMapper.resolveRoles(userData, ActionListener.wrap(roles -> { - final User computedUser = new User(username, roles.toArray(new String[roles.size()]), null, null, null, true); - if (userPrincipalNameToUserCache != null) { - userPrincipalNameToUserCache.put(username, computedUser); - } - listener.onResponse(AuthenticationResult.success(computedUser)); - }, listener::onFailure)); + final User user = (userPrincipalNameToUserCache != null) ? userPrincipalNameToUserCache.get(username) : null; + if (user != null) { + listener.onResponse(AuthenticationResult.success(user)); + } else { + buildUser(username, listener); + } } } + private void buildUser(final String username, final ActionListener listener) { + final UserRoleMapper.UserData userData = new UserRoleMapper.UserData(username, null, Collections.emptySet(), null, this.config); + userRoleMapper.resolveRoles(userData, ActionListener.wrap(roles -> { + final User computedUser = new User(username, roles.toArray(new String[roles.size()]), null, null, null, true); + if (userPrincipalNameToUserCache != null) { + userPrincipalNameToUserCache.put(username, computedUser); + } + listener.onResponse(AuthenticationResult.success(computedUser)); + }, listener::onFailure)); + } + @Override public void lookupUser(final String username, final ActionListener listener) { listener.onResponse(null); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealm.java index 87749850141..193b33b7d8f 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealm.java @@ -8,7 +8,6 @@ package org.elasticsearch.xpack.security.authc.ldap; import com.unboundid.ldap.sdk.LDAPException; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ContextPreservingActionListener; @@ -16,10 +15,13 @@ import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool.Names; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; +import org.elasticsearch.xpack.core.security.authc.Realm; import org.elasticsearch.xpack.core.security.authc.RealmConfig; import org.elasticsearch.xpack.core.security.authc.RealmSettings; import org.elasticsearch.xpack.core.security.authc.ldap.LdapRealmSettings; @@ -31,6 +33,7 @@ import org.elasticsearch.xpack.security.authc.ldap.support.LdapLoadBalancing; import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession; import org.elasticsearch.xpack.security.authc.ldap.support.SessionFactory; import org.elasticsearch.xpack.security.authc.support.CachingUsernamePasswordRealm; +import org.elasticsearch.xpack.security.authc.support.DelegatedAuthorizationSupport; import org.elasticsearch.xpack.security.authc.support.UserRoleMapper; import org.elasticsearch.xpack.security.authc.support.UserRoleMapper.UserData; import org.elasticsearch.xpack.security.authc.support.mapper.CompositeRoleMapper; @@ -53,7 +56,7 @@ public final class LdapRealm extends CachingUsernamePasswordRealm { private final UserRoleMapper roleMapper; private final ThreadPool threadPool; private final TimeValue executionTimeout; - + private DelegatedAuthorizationSupport delegatedRealms; public LdapRealm(String type, RealmConfig config, SSLService sslService, ResourceWatcherService watcherService, @@ -118,6 +121,7 @@ public final class LdapRealm extends CachingUsernamePasswordRealm { */ @Override protected void doAuthenticate(UsernamePasswordToken token, ActionListener listener) { + assert delegatedRealms != null : "Realm has not been initialized correctly"; // we submit to the threadpool because authentication using LDAP will execute blocking I/O for a bind request and we don't want // network threads stuck waiting for a socket to connect. After the bind, then all interaction with LDAP should be async final CancellableLdapRunnable cancellableLdapRunnable = new CancellableLdapRunnable<>(listener, @@ -159,6 +163,14 @@ public final class LdapRealm extends CachingUsernamePasswordRealm { sessionListener); } + @Override + public void initialize(Iterable realms, XPackLicenseState licenseState) { + if (delegatedRealms != null) { + throw new IllegalStateException("Realm has already been initialized"); + } + delegatedRealms = new DelegatedAuthorizationSupport(realms, config, licenseState); + } + @Override public void usageStats(ActionListener> listener) { super.usageStats(ActionListener.wrap(usage -> { @@ -171,39 +183,56 @@ public final class LdapRealm extends CachingUsernamePasswordRealm { } private static void buildUser(LdapSession session, String username, ActionListener listener, - UserRoleMapper roleMapper) { + UserRoleMapper roleMapper, DelegatedAuthorizationSupport delegatedAuthz) { + assert delegatedAuthz != null : "DelegatedAuthorizationSupport is null"; if (session == null) { listener.onResponse(AuthenticationResult.notHandled()); + } else if (delegatedAuthz.hasDelegation()) { + delegatedAuthz.resolve(username, listener); } else { - boolean loadingGroups = false; - try { - final Consumer onFailure = e -> { - IOUtils.closeWhileHandlingException(session); - listener.onFailure(e); - }; - session.resolve(ActionListener.wrap((ldapData) -> { - final Map metadata = MapBuilder.newMapBuilder() - .put("ldap_dn", session.userDn()) - .put("ldap_groups", ldapData.groups) - .putAll(ldapData.metaData) - .map(); - final UserData user = new UserData(username, session.userDn(), ldapData.groups, - metadata, session.realm()); - roleMapper.resolveRoles(user, ActionListener.wrap( - roles -> { - IOUtils.close(session); - String[] rolesArray = roles.toArray(new String[roles.size()]); - listener.onResponse(AuthenticationResult.success( - new User(username, rolesArray, null, null, metadata, true)) - ); - }, onFailure - )); - }, onFailure)); - loadingGroups = true; - } finally { - if (loadingGroups == false) { - session.close(); - } + lookupUserFromSession(username, session, roleMapper, listener); + } + } + + @Override + protected void handleCachedAuthentication(User user, ActionListener listener) { + if (delegatedRealms.hasDelegation()) { + delegatedRealms.resolve(user.principal(), listener); + } else { + super.handleCachedAuthentication(user, listener); + } + } + + private static void lookupUserFromSession(String username, LdapSession session, UserRoleMapper roleMapper, + ActionListener listener) { + boolean loadingGroups = false; + try { + final Consumer onFailure = e -> { + IOUtils.closeWhileHandlingException(session); + listener.onFailure(e); + }; + session.resolve(ActionListener.wrap((ldapData) -> { + final Map metadata = MapBuilder.newMapBuilder() + .put("ldap_dn", session.userDn()) + .put("ldap_groups", ldapData.groups) + .putAll(ldapData.metaData) + .map(); + final UserData user = new UserData(username, session.userDn(), ldapData.groups, + metadata, session.realm()); + roleMapper.resolveRoles(user, ActionListener.wrap( + roles -> { + IOUtils.close(session); + String[] rolesArray = roles.toArray(new String[roles.size()]); + listener.onResponse(AuthenticationResult.success( + new User(username, rolesArray, null, null, metadata, true)) + ); + }, onFailure + )); + }, onFailure)); + loadingGroups = true; + } finally { + if (loadingGroups == false) { + session.close(); } } } @@ -233,7 +262,7 @@ public final class LdapRealm extends CachingUsernamePasswordRealm { resultListener.onResponse(AuthenticationResult.notHandled()); } else { ldapSessionAtomicReference.set(session); - buildUser(session, username, resultListener, roleMapper); + buildUser(session, username, resultListener, roleMapper, delegatedRealms); } } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/pki/PkiRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/pki/PkiRealm.java index 7b9eabfd706..4d13f332ffe 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/pki/PkiRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/pki/PkiRealm.java @@ -19,6 +19,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.Environment; +import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; import org.elasticsearch.xpack.core.security.authc.AuthenticationToken; @@ -31,12 +32,12 @@ import org.elasticsearch.xpack.core.ssl.CertParsingUtils; import org.elasticsearch.xpack.core.ssl.SSLConfigurationSettings; import org.elasticsearch.xpack.security.authc.BytesKey; import org.elasticsearch.xpack.security.authc.support.CachingRealm; +import org.elasticsearch.xpack.security.authc.support.DelegatedAuthorizationSupport; import org.elasticsearch.xpack.security.authc.support.UserRoleMapper; import org.elasticsearch.xpack.security.authc.support.mapper.CompositeRoleMapper; import org.elasticsearch.xpack.security.authc.support.mapper.NativeRoleMappingStore; import javax.net.ssl.X509TrustManager; - import java.security.MessageDigest; import java.security.cert.Certificate; import java.security.cert.CertificateEncodingException; @@ -75,6 +76,7 @@ public class PkiRealm extends Realm implements CachingRealm { private final Pattern principalPattern; private final UserRoleMapper roleMapper; private final Cache cache; + private DelegatedAuthorizationSupport delegatedRealms; public PkiRealm(RealmConfig config, ResourceWatcherService watcherService, NativeRoleMappingStore nativeRoleMappingStore) { this(config, new CompositeRoleMapper(PkiRealmSettings.TYPE, config, watcherService, nativeRoleMappingStore)); @@ -91,6 +93,15 @@ public class PkiRealm extends Realm implements CachingRealm { .setExpireAfterWrite(PkiRealmSettings.CACHE_TTL_SETTING.get(config.settings())) .setMaximumWeight(PkiRealmSettings.CACHE_MAX_USERS_SETTING.get(config.settings())) .build(); + this.delegatedRealms = null; + } + + @Override + public void initialize(Iterable realms, XPackLicenseState licenseState) { + if (delegatedRealms != null) { + throw new IllegalStateException("Realm has already been initialized"); + } + delegatedRealms = new DelegatedAuthorizationSupport(realms, config, licenseState); } @Override @@ -105,32 +116,50 @@ public class PkiRealm extends Realm implements CachingRealm { @Override public void authenticate(AuthenticationToken authToken, ActionListener listener) { + assert delegatedRealms != null : "Realm has not been initialized correctly"; X509AuthenticationToken token = (X509AuthenticationToken)authToken; try { final BytesKey fingerprint = computeFingerprint(token.credentials()[0]); User user = cache.get(fingerprint); if (user != null) { - listener.onResponse(AuthenticationResult.success(user)); + if (delegatedRealms.hasDelegation()) { + delegatedRealms.resolve(token.principal(), listener); + } else { + listener.onResponse(AuthenticationResult.success(user)); + } } else if (isCertificateChainTrusted(trustManager, token, logger) == false) { listener.onResponse(AuthenticationResult.unsuccessful("Certificate for " + token.dn() + " is not trusted", null)); } else { - final Map metadata = Collections.singletonMap("pki_dn", token.dn()); - final UserRoleMapper.UserData userData = new UserRoleMapper.UserData(token.principal(), - token.dn(), Collections.emptySet(), metadata, this.config); - roleMapper.resolveRoles(userData, ActionListener.wrap(roles -> { - final User computedUser = - new User(token.principal(), roles.toArray(new String[roles.size()]), null, null, metadata, true); - try (ReleasableLock ignored = readLock.acquire()) { - cache.put(fingerprint, computedUser); + final ActionListener cachingListener = ActionListener.wrap(result -> { + if (result.isAuthenticated()) { + try (ReleasableLock ignored = readLock.acquire()) { + cache.put(fingerprint, result.getUser()); + } } - listener.onResponse(AuthenticationResult.success(computedUser)); - }, listener::onFailure)); + listener.onResponse(result); + }, listener::onFailure); + if (delegatedRealms.hasDelegation()) { + delegatedRealms.resolve(token.principal(), cachingListener); + } else { + this.buildUser(token, cachingListener); + } } } catch (CertificateEncodingException e) { listener.onResponse(AuthenticationResult.unsuccessful("Certificate for " + token.dn() + " has encoding issues", e)); } } + private void buildUser(X509AuthenticationToken token, ActionListener listener) { + final Map metadata = Collections.singletonMap("pki_dn", token.dn()); + final UserRoleMapper.UserData userData = new UserRoleMapper.UserData(token.principal(), + token.dn(), Collections.emptySet(), metadata, this.config); + roleMapper.resolveRoles(userData, ActionListener.wrap(roles -> { + final User computedUser = + new User(token.principal(), roles.toArray(new String[roles.size()]), null, null, metadata, true); + listener.onResponse(AuthenticationResult.success(computedUser)); + }, listener::onFailure)); + } + @Override public void lookupUser(String username, ActionListener listener) { listener.onResponse(null); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/saml/SamlRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/saml/SamlRealm.java index cc160c8f78b..4a9db7c5d61 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/saml/SamlRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/saml/SamlRealm.java @@ -33,6 +33,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.watcher.FileChangesListener; import org.elasticsearch.watcher.FileWatcher; import org.elasticsearch.watcher.ResourceWatcherService; @@ -46,10 +47,12 @@ import org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings; import org.elasticsearch.xpack.core.security.user.User; import org.elasticsearch.xpack.core.ssl.SSLConfiguration; import org.elasticsearch.xpack.core.ssl.CertParsingUtils; +import org.elasticsearch.xpack.core.ssl.SSLConfiguration; import org.elasticsearch.xpack.core.ssl.SSLService; import org.elasticsearch.xpack.core.ssl.X509KeyPairSettings; import org.elasticsearch.xpack.security.authc.Realms; import org.elasticsearch.xpack.security.authc.TokenService; +import org.elasticsearch.xpack.security.authc.support.DelegatedAuthorizationSupport; import org.elasticsearch.xpack.security.authc.support.UserRoleMapper; import org.opensaml.core.criterion.EntityIdCriterion; import org.opensaml.saml.common.xml.SAMLConstants; @@ -117,6 +120,7 @@ import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings.NAME_ATTRIBUTE; import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings.POPULATE_USER_METADATA; import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings.PRINCIPAL_ATTRIBUTE; +import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings.REQUESTED_AUTHN_CONTEXT_CLASS_REF; import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings.SIGNING_KEY_ALIAS; import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings.SIGNING_MESSAGE_TYPES; import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings.SIGNING_SETTINGS; @@ -124,7 +128,6 @@ import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings.SP_ENTITY_ID; import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings.SP_LOGOUT; import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings.TYPE; -import static org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings.REQUESTED_AUTHN_CONTEXT_CLASS_REF; /** * This class is {@link Releasable} because it uses a library that thinks timers and timer tasks @@ -166,6 +169,7 @@ public final class SamlRealm extends Realm implements Releasable { private final AttributeParser nameAttribute; private final AttributeParser mailAttribute; + private DelegatedAuthorizationSupport delegatedRealms; /** * Factory for SAML realm. @@ -231,6 +235,14 @@ public final class SamlRealm extends Realm implements Releasable { this.releasables = new ArrayList<>(); } + @Override + public void initialize(Iterable realms, XPackLicenseState licenseState) { + if (delegatedRealms != null) { + throw new IllegalStateException("Realm has already been initialized"); + } + delegatedRealms = new DelegatedAuthorizationSupport(realms, config, licenseState); + } + static String require(RealmConfig config, Setting setting) { final String value = setting.get(config.settings()); if (value.isEmpty()) { @@ -402,14 +414,27 @@ public final class SamlRealm extends Realm implements Releasable { } } - private void buildUser(SamlAttributes attributes, ActionListener listener) { + private void buildUser(SamlAttributes attributes, ActionListener baseListener) { final String principal = resolveSingleValueAttribute(attributes, principalAttribute, PRINCIPAL_ATTRIBUTE.name()); if (Strings.isNullOrEmpty(principal)) { - listener.onResponse(AuthenticationResult.unsuccessful( + baseListener.onResponse(AuthenticationResult.unsuccessful( principalAttribute + " not found in " + attributes.attributes(), null)); return; } + final Map tokenMetadata = createTokenMetadata(attributes.name(), attributes.session()); + ActionListener wrappedListener = ActionListener.wrap(auth -> { + if (auth.isAuthenticated()) { + config.threadContext().putTransient(CONTEXT_TOKEN_DATA, tokenMetadata); + } + baseListener.onResponse(auth); + }, baseListener::onFailure); + + if (delegatedRealms.hasDelegation()) { + delegatedRealms.resolve(principal, wrappedListener); + return; + } + final Map userMeta = new HashMap<>(); if (populateUserMetadata) { for (SamlAttributes.SamlAttribute a : attributes.attributes()) { @@ -424,7 +449,6 @@ public final class SamlRealm extends Realm implements Releasable { userMeta.put(USER_METADATA_NAMEID_FORMAT, attributes.name().format); } - final Map tokenMetadata = createTokenMetadata(attributes.name(), attributes.session()); final List groups = groupsAttribute.getAttribute(attributes); final String dn = resolveSingleValueAttribute(attributes, dnAttribute, DN_ATTRIBUTE.name()); @@ -433,9 +457,8 @@ public final class SamlRealm extends Realm implements Releasable { UserRoleMapper.UserData userData = new UserRoleMapper.UserData(principal, dn, groups, userMeta, config); roleMapper.resolveRoles(userData, ActionListener.wrap(roles -> { final User user = new User(principal, roles.toArray(new String[roles.size()]), name, mail, userMeta, true); - config.threadContext().putTransient(CONTEXT_TOKEN_DATA, tokenMetadata); - listener.onResponse(AuthenticationResult.success(user)); - }, listener::onFailure)); + wrappedListener.onResponse(AuthenticationResult.success(user)); + }, wrappedListener::onFailure)); } public Map createTokenMetadata(SamlNameId nameId, String session) { @@ -745,10 +768,10 @@ public final class SamlRealm extends Realm implements Releasable { attributes -> attributes.getAttributeValues(attributeName)); } } else if (required) { - throw new SettingsException("Setting" + RealmSettings.getFullSettingKey(realmConfig, setting.getAttribute()) + throw new SettingsException("Setting " + RealmSettings.getFullSettingKey(realmConfig, setting.getAttribute()) + " is required"); } else if (setting.getPattern().exists(settings)) { - throw new SettingsException("Setting" + RealmSettings.getFullSettingKey(realmConfig, setting.getPattern()) + throw new SettingsException("Setting " + RealmSettings.getFullSettingKey(realmConfig, setting.getPattern()) + " cannot be set unless " + RealmSettings.getFullSettingKey(realmConfig, setting.getAttribute()) + " is also set"); } else { return new AttributeParser("No SAML attribute for [" + setting.name() + "]", attributes -> Collections.emptyList()); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java index 6e321f9f7dd..af93a180072 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java @@ -39,9 +39,9 @@ public abstract class CachingUsernamePasswordRealm extends UsernamePasswordRealm final TimeValue ttl = CachingUsernamePasswordRealmSettings.CACHE_TTL_SETTING.get(config.settings()); if (ttl.getNanos() > 0) { cache = CacheBuilder.>builder() - .setExpireAfterWrite(ttl) - .setMaximumWeight(CachingUsernamePasswordRealmSettings.CACHE_MAX_USERS_SETTING.get(config.settings())) - .build(); + .setExpireAfterWrite(ttl) + .setMaximumWeight(CachingUsernamePasswordRealmSettings.CACHE_MAX_USERS_SETTING.get(config.settings())) + .build(); } else { cache = null; } @@ -108,10 +108,16 @@ public abstract class CachingUsernamePasswordRealm extends UsernamePasswordRealm listenableCacheEntry.addListener(ActionListener.wrap(authenticatedUserWithHash -> { if (authenticatedUserWithHash != null && authenticatedUserWithHash.verify(token.credentials())) { // cached credential hash matches the credential hash for this forestalled request - final User user = authenticatedUserWithHash.user; - logger.debug("realm [{}] authenticated user [{}], with roles [{}], from cache", name(), token.principal(), - user.roles()); - listener.onResponse(AuthenticationResult.success(user)); + handleCachedAuthentication(authenticatedUserWithHash.user, ActionListener.wrap(cacheResult -> { + if (cacheResult.isAuthenticated()) { + logger.debug("realm [{}] authenticated user [{}], with roles [{}]", + name(), token.principal(), cacheResult.getUser().roles()); + } else { + logger.debug("realm [{}] authenticated user [{}] from cache, but then failed [{}]", + name(), token.principal(), cacheResult.getMessage()); + } + listener.onResponse(cacheResult); + }, listener::onFailure)); } else { // The inflight request has failed or its credential hash does not match the // hash of the credential for this forestalled request. @@ -153,6 +159,16 @@ public abstract class CachingUsernamePasswordRealm extends UsernamePasswordRealm } } + /** + * {@code handleCachedAuthentication} is called when a {@link User} is retrieved from the cache. + * The first {@code user} parameter is the user object that was found in the cache. + * The default implementation returns a {@link AuthenticationResult#success(User) success result} with the + * provided user, but sub-classes can return a different {@code User} object, or an unsuccessful result. + */ + protected void handleCachedAuthentication(User user, ActionListener listener) { + listener.onResponse(AuthenticationResult.success(user)); + } + @Override public void usageStats(ActionListener> listener) { super.usageStats(ActionListener.wrap(stats -> { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/DelegatedAuthorizationSupport.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/DelegatedAuthorizationSupport.java new file mode 100644 index 00000000000..ff6fc6042e7 --- /dev/null +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/DelegatedAuthorizationSupport.java @@ -0,0 +1,146 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.security.authc.support; + +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.license.LicenseUtils; +import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; +import org.elasticsearch.xpack.core.security.authc.Realm; +import org.elasticsearch.xpack.core.security.authc.RealmConfig; +import org.elasticsearch.xpack.core.security.authc.RealmSettings; +import org.elasticsearch.xpack.core.security.authc.support.DelegatedAuthorizationSettings; +import org.elasticsearch.xpack.core.security.user.User; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.common.Strings.collectionToDelimitedString; + +/** + * Utility class for supporting "delegated authorization" (aka "authorization_realms", aka "lookup realms"). + * A {@link Realm} may support delegating authorization to another realm. It does this by registering a + * setting for {@link DelegatedAuthorizationSettings#AUTHZ_REALMS}, and constructing an instance of this + * class. Then, after the realm has performed any authentication steps, if {@link #hasDelegation()} is + * {@code true}, it delegates the construction of the {@link User} object and {@link AuthenticationResult} + * to {@link #resolve(String, ActionListener)}. + */ +public class DelegatedAuthorizationSupport { + + private final RealmUserLookup lookup; + private final Logger logger; + private final XPackLicenseState licenseState; + + /** + * Resolves the {@link DelegatedAuthorizationSettings#AUTHZ_REALMS} setting from {@code config} and calls + * {@link #DelegatedAuthorizationSupport(Iterable, List, Settings, ThreadContext, XPackLicenseState)} + */ + public DelegatedAuthorizationSupport(Iterable allRealms, RealmConfig config, XPackLicenseState licenseState) { + this(allRealms, DelegatedAuthorizationSettings.AUTHZ_REALMS.get(config.settings()), config.globalSettings(), config.threadContext(), + licenseState); + } + + /** + * Constructs a new object that delegates to the named realms ({@code lookupRealms}), which must exist within + * {@code allRealms}. + * @throws IllegalArgumentException if one of the specified realms does not exist + */ + protected DelegatedAuthorizationSupport(Iterable allRealms, List lookupRealms, Settings settings, + ThreadContext threadContext, XPackLicenseState licenseState) { + final List resolvedLookupRealms = resolveRealms(allRealms, lookupRealms); + checkForRealmChains(resolvedLookupRealms, settings); + this.lookup = new RealmUserLookup(resolvedLookupRealms, threadContext); + this.logger = Loggers.getLogger(getClass()); + this.licenseState = licenseState; + } + + /** + * Are there any realms configured for delegated lookup + */ + public boolean hasDelegation() { + return this.lookup.hasRealms(); + } + + /** + * Attempts to find the user specified by {@code username} in one of the delegated realms. + * The realms are searched in the order specified during construction. + * Returns a {@link AuthenticationResult#success(User) successful result} if a {@link User} + * was found, otherwise returns an + * {@link AuthenticationResult#unsuccessful(String, Exception) unsuccessful result} + * with a meaningful diagnostic message. + */ + public void resolve(String username, ActionListener resultListener) { + if (licenseState.isAuthorizationRealmAllowed() == false) { + resultListener.onResponse(AuthenticationResult.unsuccessful( + DelegatedAuthorizationSettings.AUTHZ_REALMS.getKey() + " are not permitted", + LicenseUtils.newComplianceException(DelegatedAuthorizationSettings.AUTHZ_REALMS.getKey()) + )); + return; + } + if (hasDelegation() == false) { + resultListener.onResponse(AuthenticationResult.unsuccessful( + "No [" + DelegatedAuthorizationSettings.AUTHZ_REALMS.getKey() + "] have been configured", null)); + return; + } + ActionListener> userListener = ActionListener.wrap(tuple -> { + if (tuple != null) { + logger.trace("Found user " + tuple.v1() + " in realm " + tuple.v2()); + resultListener.onResponse(AuthenticationResult.success(tuple.v1())); + } else { + resultListener.onResponse(AuthenticationResult.unsuccessful("the principal [" + username + + "] was authenticated, but no user could be found in realms [" + collectionToDelimitedString(lookup.getRealms(), ",") + + "]", null)); + } + }, resultListener::onFailure); + lookup.lookup(username, userListener); + } + + private List resolveRealms(Iterable allRealms, List lookupRealms) { + final List result = new ArrayList<>(lookupRealms.size()); + for (String name : lookupRealms) { + result.add(findRealm(name, allRealms)); + } + assert result.size() == lookupRealms.size(); + return result; + } + + /** + * Checks for (and rejects) chains of delegation in the provided realms. + * A chain occurs when "realmA" delegates authorization to "realmB", and realmB also delegates authorization (to any realm). + * Since "realmB" does not handle its own authorization, it is not a valid target for delegated authorization. + * @param delegatedRealms The list of realms that are going to be used for authorization. If is an error if any of these realms are + * also configured to delegate their authorization. + * @throws IllegalArgumentException if a chain is detected + */ + private void checkForRealmChains(Iterable delegatedRealms, Settings globalSettings) { + final Map settingsByRealm = RealmSettings.getRealmSettings(globalSettings); + for (Realm realm : delegatedRealms) { + final Settings realmSettings = settingsByRealm.get(realm.name()); + if (realmSettings != null && DelegatedAuthorizationSettings.AUTHZ_REALMS.exists(realmSettings)) { + throw new IllegalArgumentException("cannot use realm [" + realm + + "] as an authorization realm - it is already delegating authorization to [" + + DelegatedAuthorizationSettings.AUTHZ_REALMS.get(realmSettings) + "]"); + } + } + } + + private Realm findRealm(String name, Iterable allRealms) { + for (Realm realm : allRealms) { + if (name.equals(realm.name())) { + return realm; + } + } + throw new IllegalArgumentException("configured authorization realm [" + name + "] does not exist (or is not enabled)"); + } + +} diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/RealmUserLookup.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/RealmUserLookup.java new file mode 100644 index 00000000000..428b7c1e4a1 --- /dev/null +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/RealmUserLookup.java @@ -0,0 +1,63 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.security.authc.support; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.xpack.core.common.IteratingActionListener; +import org.elasticsearch.xpack.core.security.authc.Realm; +import org.elasticsearch.xpack.core.security.user.User; + +import java.util.Collections; +import java.util.List; + +public class RealmUserLookup { + + private final List realms; + private final ThreadContext threadContext; + + public RealmUserLookup(List realms, ThreadContext threadContext) { + this.realms = realms; + this.threadContext = threadContext; + } + + public List getRealms() { + return Collections.unmodifiableList(realms); + } + + public boolean hasRealms() { + return realms.isEmpty() == false; + } + + /** + * Lookup the {@code principal} in the list of {@link #realms}. + * The realms are consulted in order. When one realm responds with a non-null {@link User}, this + * is returned with the matching realm, through the {@code listener}. + * If no user if found (including the case where the {@link #realms} list is empty), then + * {@link ActionListener#onResponse(Object)} is called with a {@code null} {@link Tuple}. + */ + public void lookup(String principal, ActionListener> listener) { + final IteratingActionListener, ? extends Realm> userLookupListener = + new IteratingActionListener<>(listener, + (realm, lookupUserListener) -> realm.lookupUser(principal, + ActionListener.wrap(foundUser -> { + if (foundUser != null) { + lookupUserListener.onResponse(new Tuple<>(foundUser, realm)); + } else { + lookupUserListener.onResponse(null); + } + }, + lookupUserListener::onFailure)), + realms, threadContext); + try { + userLookupListener.run(); + } catch (Exception e) { + listener.onFailure(e); + } + } +} diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmAuthenticateFailedTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmAuthenticateFailedTests.java index 5bc239241cf..7c5904d048a 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmAuthenticateFailedTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmAuthenticateFailedTests.java @@ -11,13 +11,20 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.SecureString; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.env.TestEnvironment; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; +import org.elasticsearch.xpack.core.security.authc.RealmConfig; import org.elasticsearch.xpack.core.security.authc.kerberos.KerberosRealmSettings; import org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken; +import org.elasticsearch.xpack.security.authc.support.MockLookupRealm; import org.elasticsearch.xpack.core.security.user.User; import org.ietf.jgss.GSSException; +import java.nio.charset.StandardCharsets; import java.nio.file.Path; +import java.util.Collections; import java.util.List; import javax.security.auth.login.LoginException; @@ -29,7 +36,9 @@ import static org.hamcrest.Matchers.notNullValue; import static org.mockito.AdditionalMatchers.aryEq; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; public class KerberosRealmAuthenticateFailedTests extends KerberosRealmTestCase { @@ -105,4 +114,30 @@ public class KerberosRealmAuthenticateFailedTests extends KerberosRealmTestCase any(ActionListener.class)); } } + + public void testDelegatedAuthorizationFailedToResolve() throws Exception { + final String username = randomPrincipalName(); + final MockLookupRealm otherRealm = new MockLookupRealm(new RealmConfig("other_realm", Settings.EMPTY, globalSettings, + TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings))); + final User lookupUser = new User(randomAlphaOfLength(5)); + otherRealm.registerUser(lookupUser); + + settings = Settings.builder().put(settings).putList("authorization_realms", "other_realm").build(); + final KerberosRealm kerberosRealm = createKerberosRealm(Collections.singletonList(otherRealm), username); + final byte[] decodedTicket = "base64encodedticket".getBytes(StandardCharsets.UTF_8); + final Path keytabPath = config.env().configFile().resolve(KerberosRealmSettings.HTTP_SERVICE_KEYTAB_PATH.get(config.settings())); + final boolean krbDebug = KerberosRealmSettings.SETTING_KRB_DEBUG_ENABLE.get(config.settings()); + mockKerberosTicketValidator(decodedTicket, keytabPath, krbDebug, new Tuple<>(username, "out-token"), null); + final KerberosAuthenticationToken kerberosAuthenticationToken = new KerberosAuthenticationToken(decodedTicket); + + final PlainActionFuture future = new PlainActionFuture<>(); + kerberosRealm.authenticate(kerberosAuthenticationToken, future); + + AuthenticationResult result = future.actionGet(); + assertThat(result.getStatus(), is(equalTo(AuthenticationResult.Status.CONTINUE))); + verify(mockKerberosTicketValidator, times(1)).validateTicket(aryEq(decodedTicket), eq(keytabPath), eq(krbDebug), + any(ActionListener.class)); + verify(mockNativeRoleMappingStore).refreshRealmOnChange(kerberosRealm); + verifyNoMoreInteractions(mockKerberosTicketValidator, mockNativeRoleMappingStore); + } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmTestCase.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmTestCase.java index 9c2c6484c82..dd83da49a0b 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmTestCase.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmTestCase.java @@ -13,11 +13,13 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; +import org.elasticsearch.xpack.core.security.authc.Realm; import org.elasticsearch.xpack.core.security.authc.RealmConfig; import org.elasticsearch.xpack.core.security.authc.kerberos.KerberosRealmSettings; import org.elasticsearch.xpack.core.security.support.Exceptions; @@ -30,6 +32,7 @@ import org.junit.Before; import java.nio.file.Path; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Locale; import java.util.Map; @@ -58,6 +61,7 @@ public abstract class KerberosRealmTestCase extends ESTestCase { protected KerberosTicketValidator mockKerberosTicketValidator; protected NativeRoleMappingStore mockNativeRoleMappingStore; + protected XPackLicenseState licenseState; protected static final Set roles = Sets.newHashSet("admin", "kibana_user"); @@ -69,6 +73,8 @@ public abstract class KerberosRealmTestCase extends ESTestCase { globalSettings = Settings.builder().put("path.home", dir).build(); settings = KerberosTestCase.buildKerberosRealmSettings(KerberosTestCase.writeKeyTab(dir.resolve("key.keytab"), "asa").toString(), 100, "10m", true, randomBoolean()); + licenseState = mock(XPackLicenseState.class); + when(licenseState.isAuthorizationRealmAllowed()).thenReturn(true); } @After @@ -102,12 +108,18 @@ public abstract class KerberosRealmTestCase extends ESTestCase { } protected KerberosRealm createKerberosRealm(final String... userForRoleMapping) { + return createKerberosRealm(Collections.emptyList(), userForRoleMapping); + } + + protected KerberosRealm createKerberosRealm(final List delegatedRealms, final String... userForRoleMapping) { config = new RealmConfig("test-kerb-realm", settings, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings)); mockNativeRoleMappingStore = roleMappingStore(Arrays.asList(userForRoleMapping)); mockKerberosTicketValidator = mock(KerberosTicketValidator.class); final KerberosRealm kerberosRealm = new KerberosRealm(config, mockNativeRoleMappingStore, mockKerberosTicketValidator, threadPool, null); + Collections.shuffle(delegatedRealms, random()); + kerberosRealm.initialize(delegatedRealms, licenseState); return kerberosRealm; } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmTests.java index fee8df535f2..d35068fd07a 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosRealmTests.java @@ -12,6 +12,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.SecureString; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.TestEnvironment; import org.elasticsearch.xpack.core.security.user.User; @@ -20,6 +21,7 @@ import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; import org.elasticsearch.xpack.core.security.authc.RealmConfig; import org.elasticsearch.xpack.core.security.authc.kerberos.KerberosRealmSettings; import org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken; +import org.elasticsearch.xpack.security.authc.support.MockLookupRealm; import org.elasticsearch.xpack.security.authc.support.UserRoleMapper.UserData; import org.ietf.jgss.GSSException; @@ -34,6 +36,7 @@ import java.nio.file.attribute.FileAttribute; import java.nio.file.attribute.PosixFilePermission; import java.nio.file.attribute.PosixFilePermissions; import java.util.Arrays; +import java.util.Collections; import java.util.EnumSet; import java.util.Locale; import java.util.Set; @@ -47,6 +50,7 @@ import static org.mockito.AdditionalMatchers.aryEq; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; @@ -160,4 +164,38 @@ public class KerberosRealmTests extends KerberosRealmTestCase { () -> new KerberosRealm(config, mockNativeRoleMappingStore, mockKerberosTicketValidator, threadPool, null)); assertThat(iae.getMessage(), is(equalTo(expectedErrorMessage))); } + + public void testDelegatedAuthorization() throws Exception { + final String username = randomPrincipalName(); + final String expectedUsername = maybeRemoveRealmName(username); + final MockLookupRealm otherRealm = spy(new MockLookupRealm(new RealmConfig("other_realm", Settings.EMPTY, globalSettings, + TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings)))); + final User lookupUser = new User(expectedUsername, new String[] { "admin-role" }, expectedUsername, + expectedUsername + "@example.com", Collections.singletonMap("k1", "v1"), true); + otherRealm.registerUser(lookupUser); + + settings = Settings.builder().put(settings).putList("authorization_realms", "other_realm").build(); + final KerberosRealm kerberosRealm = createKerberosRealm(Collections.singletonList(otherRealm), username); + final User expectedUser = lookupUser; + final byte[] decodedTicket = "base64encodedticket".getBytes(StandardCharsets.UTF_8); + final Path keytabPath = config.env().configFile().resolve(KerberosRealmSettings.HTTP_SERVICE_KEYTAB_PATH.get(config.settings())); + final boolean krbDebug = KerberosRealmSettings.SETTING_KRB_DEBUG_ENABLE.get(config.settings()); + mockKerberosTicketValidator(decodedTicket, keytabPath, krbDebug, new Tuple<>(username, "out-token"), null); + final KerberosAuthenticationToken kerberosAuthenticationToken = new KerberosAuthenticationToken(decodedTicket); + + PlainActionFuture future = new PlainActionFuture<>(); + kerberosRealm.authenticate(kerberosAuthenticationToken, future); + assertSuccessAuthenticationResult(expectedUser, "out-token", future.actionGet()); + + future = new PlainActionFuture<>(); + kerberosRealm.authenticate(kerberosAuthenticationToken, future); + assertSuccessAuthenticationResult(expectedUser, "out-token", future.actionGet()); + + verify(mockKerberosTicketValidator, times(2)).validateTicket(aryEq(decodedTicket), eq(keytabPath), eq(krbDebug), + any(ActionListener.class)); + verify(mockNativeRoleMappingStore).refreshRealmOnChange(kerberosRealm); + verifyNoMoreInteractions(mockKerberosTicketValidator, mockNativeRoleMappingStore); + verify(otherRealm, times(2)).lookupUser(eq(expectedUsername), any(ActionListener.class)); + } } + diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ActiveDirectoryRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ActiveDirectoryRealmTests.java index 2c6756aada7..2f5147ca2b1 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ActiveDirectoryRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ActiveDirectoryRealmTests.java @@ -22,6 +22,8 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.license.TestUtils; +import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -48,6 +50,7 @@ import org.junit.BeforeClass; import java.security.AccessController; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -91,6 +94,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase { private ThreadPool threadPool; private Settings globalSettings; private SSLService sslService; + private XPackLicenseState licenseState; @BeforeClass public static void setNumberOfLdapServers() { @@ -125,6 +129,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase { resourceWatcherService = new ResourceWatcherService(Settings.EMPTY, threadPool); globalSettings = Settings.builder().put("path.home", createTempDir()).build(); sslService = new SSLService(globalSettings, TestEnvironment.newEnvironment(globalSettings)); + licenseState = new TestUtils.UpdatableLicenseState(); } @After @@ -163,6 +168,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase { ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService, threadPool); DnRoleMapper roleMapper = new DnRoleMapper(config, resourceWatcherService); LdapRealm realm = new LdapRealm(LdapRealmSettings.AD_TYPE, config, sessionFactory, roleMapper, threadPool); + realm.initialize(Collections.singleton(realm), licenseState); PlainActionFuture future = new PlainActionFuture<>(); realm.authenticate(new UsernamePasswordToken("CN=ironman", new SecureString(PASSWORD)), future); @@ -179,6 +185,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase { ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService, threadPool); DnRoleMapper roleMapper = new DnRoleMapper(config, resourceWatcherService); LdapRealm realm = new LdapRealm(LdapRealmSettings.AD_TYPE, config, sessionFactory, roleMapper, threadPool); + realm.initialize(Collections.singleton(realm), licenseState); // Thor does not have a UPN of form CN=Thor@ad.test.elasticsearch.com PlainActionFuture future = new PlainActionFuture<>(); @@ -203,6 +210,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase { ActiveDirectorySessionFactory sessionFactory = spy(new ActiveDirectorySessionFactory(config, sslService, threadPool)); DnRoleMapper roleMapper = new DnRoleMapper(config, resourceWatcherService); LdapRealm realm = new LdapRealm(LdapRealmSettings.AD_TYPE, config, sessionFactory, roleMapper, threadPool); + realm.initialize(Collections.singleton(realm), licenseState); int count = randomIntBetween(2, 10); for (int i = 0; i < count; i++) { @@ -221,6 +229,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase { ActiveDirectorySessionFactory sessionFactory = spy(new ActiveDirectorySessionFactory(config, sslService, threadPool)); DnRoleMapper roleMapper = new DnRoleMapper(config, resourceWatcherService); LdapRealm realm = new LdapRealm(LdapRealmSettings.AD_TYPE, config, sessionFactory, roleMapper, threadPool); + realm.initialize(Collections.singleton(realm), licenseState); int count = randomIntBetween(2, 10); for (int i = 0; i < count; i++) { @@ -239,6 +248,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase { ActiveDirectorySessionFactory sessionFactory = spy(new ActiveDirectorySessionFactory(config, sslService, threadPool)); DnRoleMapper roleMapper = new DnRoleMapper(config, resourceWatcherService); LdapRealm realm = new LdapRealm(LdapRealmSettings.AD_TYPE, config, sessionFactory, roleMapper, threadPool); + realm.initialize(Collections.singleton(realm), licenseState); int count = randomIntBetween(2, 10); for (int i = 0; i < count; i++) { @@ -287,6 +297,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase { try (ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService, threadPool)) { DnRoleMapper roleMapper = new DnRoleMapper(config, resourceWatcherService); LdapRealm realm = new LdapRealm(LdapRealmSettings.AD_TYPE, config, sessionFactory, roleMapper, threadPool); + realm.initialize(Collections.singleton(realm), licenseState); PlainActionFuture future = new PlainActionFuture<>(); realm.lookupUser("CN=Thor", future); @@ -304,6 +315,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase { ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService, threadPool); DnRoleMapper roleMapper = new DnRoleMapper(config, resourceWatcherService); LdapRealm realm = new LdapRealm(LdapRealmSettings.AD_TYPE, config, sessionFactory, roleMapper, threadPool); + realm.initialize(Collections.singleton(realm), licenseState); PlainActionFuture future = new PlainActionFuture<>(); realm.authenticate(new UsernamePasswordToken("CN=ironman", new SecureString(PASSWORD)), future); @@ -320,6 +332,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase { ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService, threadPool); DnRoleMapper roleMapper = new DnRoleMapper(config, resourceWatcherService); LdapRealm realm = new LdapRealm(LdapRealmSettings.AD_TYPE, config, sessionFactory, roleMapper, threadPool); + realm.initialize(Collections.singleton(realm), licenseState); PlainActionFuture future = new PlainActionFuture<>(); realm.authenticate(new UsernamePasswordToken("CN=Thor", new SecureString(PASSWORD)), future); @@ -338,6 +351,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase { ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService, threadPool); DnRoleMapper roleMapper = new DnRoleMapper(config, resourceWatcherService); LdapRealm realm = new LdapRealm(LdapRealmSettings.AD_TYPE, config, sessionFactory, roleMapper, threadPool); + realm.initialize(Collections.singleton(realm), licenseState); PlainActionFuture> future = new PlainActionFuture<>(); realm.usageStats(future); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealmTests.java index 4aff821217d..fb20527575d 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealmTests.java @@ -14,6 +14,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; @@ -25,6 +26,7 @@ import org.elasticsearch.xpack.core.security.authc.ldap.LdapRealmSettings; import org.elasticsearch.xpack.core.security.authc.ldap.LdapSessionFactorySettings; import org.elasticsearch.xpack.core.security.authc.ldap.support.LdapSearchScope; import org.elasticsearch.xpack.core.security.authc.support.CachingUsernamePasswordRealmSettings; +import org.elasticsearch.xpack.core.security.authc.support.DelegatedAuthorizationSettings; import org.elasticsearch.xpack.core.security.authc.support.DnRoleMapperSettings; import org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken; import org.elasticsearch.xpack.core.security.user.User; @@ -33,10 +35,12 @@ import org.elasticsearch.xpack.core.ssl.VerificationMode; import org.elasticsearch.xpack.security.authc.ldap.support.LdapTestCase; import org.elasticsearch.xpack.security.authc.ldap.support.SessionFactory; import org.elasticsearch.xpack.security.authc.support.DnRoleMapper; +import org.elasticsearch.xpack.security.authc.support.MockLookupRealm; import org.junit.After; import org.junit.Before; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -50,11 +54,14 @@ import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.sameInstance; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class LdapRealmTests extends LdapTestCase { @@ -68,6 +75,7 @@ public class LdapRealmTests extends LdapTestCase { private ResourceWatcherService resourceWatcherService; private Settings defaultGlobalSettings; private SSLService sslService; + private XPackLicenseState licenseState; @Before public void init() throws Exception { @@ -75,6 +83,8 @@ public class LdapRealmTests extends LdapTestCase { resourceWatcherService = new ResourceWatcherService(Settings.EMPTY, threadPool); defaultGlobalSettings = Settings.builder().put("path.home", createTempDir()).build(); sslService = new SSLService(defaultGlobalSettings, TestEnvironment.newEnvironment(defaultGlobalSettings)); + licenseState = mock(XPackLicenseState.class); + when(licenseState.isAuthorizationRealmAllowed()).thenReturn(true); } @After @@ -87,10 +97,12 @@ public class LdapRealmTests extends LdapTestCase { String groupSearchBase = "o=sevenSeas"; String userTemplate = VALID_USER_TEMPLATE; Settings settings = buildLdapSettings(ldapUrls(), userTemplate, groupSearchBase, LdapSearchScope.SUB_TREE); - RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); + RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, + TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); LdapSessionFactory ldapFactory = new LdapSessionFactory(config, sslService, threadPool); LdapRealm ldap = new LdapRealm(LdapRealmSettings.LDAP_TYPE, config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService), threadPool); + ldap.initialize(Collections.singleton(ldap), licenseState); PlainActionFuture future = new PlainActionFuture<>(); ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, new SecureString(PASSWORD)), future); @@ -111,11 +123,13 @@ public class LdapRealmTests extends LdapTestCase { Settings settings = Settings.builder() .put(buildLdapSettings(ldapUrls(), userTemplate, groupSearchBase, LdapSearchScope.ONE_LEVEL)) .build(); - RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); + RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, + TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); LdapSessionFactory ldapFactory = new LdapSessionFactory(config, sslService, threadPool); LdapRealm ldap = new LdapRealm(LdapRealmSettings.LDAP_TYPE, config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService), threadPool); + ldap.initialize(Collections.singleton(ldap), licenseState); PlainActionFuture future = new PlainActionFuture<>(); ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, new SecureString(PASSWORD)), future); @@ -136,12 +150,14 @@ public class LdapRealmTests extends LdapTestCase { Settings settings = Settings.builder() .put(buildLdapSettings(ldapUrls(), userTemplate, groupSearchBase, LdapSearchScope.SUB_TREE)) .build(); - RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); + RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, + TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); LdapSessionFactory ldapFactory = new LdapSessionFactory(config, sslService, threadPool); ldapFactory = spy(ldapFactory); LdapRealm ldap = new LdapRealm(LdapRealmSettings.LDAP_TYPE, config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService), threadPool); + ldap.initialize(Collections.singleton(ldap), licenseState); PlainActionFuture future = new PlainActionFuture<>(); ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, new SecureString(PASSWORD)), future); @@ -161,12 +177,15 @@ public class LdapRealmTests extends LdapTestCase { Settings settings = Settings.builder() .put(buildLdapSettings(ldapUrls(), userTemplate, groupSearchBase, LdapSearchScope.SUB_TREE)) .build(); - RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); + RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, + TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); LdapSessionFactory ldapFactory = new LdapSessionFactory(config, sslService, threadPool); DnRoleMapper roleMapper = buildGroupAsRoleMapper(resourceWatcherService); ldapFactory = spy(ldapFactory); LdapRealm ldap = new LdapRealm(LdapRealmSettings.LDAP_TYPE, config, ldapFactory, roleMapper, threadPool); + ldap.initialize(Collections.singleton(ldap), licenseState); + PlainActionFuture future = new PlainActionFuture<>(); ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, new SecureString(PASSWORD)), future); future.actionGet(); @@ -194,12 +213,15 @@ public class LdapRealmTests extends LdapTestCase { .put(buildLdapSettings(ldapUrls(), userTemplate, groupSearchBase, LdapSearchScope.SUB_TREE)) .put(CachingUsernamePasswordRealmSettings.CACHE_TTL_SETTING.getKey(), -1) .build(); - RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); + RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, + TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); LdapSessionFactory ldapFactory = new LdapSessionFactory(config, sslService, threadPool); ldapFactory = spy(ldapFactory); LdapRealm ldap = new LdapRealm(LdapRealmSettings.LDAP_TYPE, config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService), threadPool); + ldap.initialize(Collections.singleton(ldap), licenseState); + PlainActionFuture future = new PlainActionFuture<>(); ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, new SecureString(PASSWORD)), future); future.actionGet(); @@ -211,6 +233,48 @@ public class LdapRealmTests extends LdapTestCase { verify(ldapFactory, times(2)).session(anyString(), any(SecureString.class), any(ActionListener.class)); } + public void testDelegatedAuthorization() throws Exception { + String groupSearchBase = "o=sevenSeas"; + String userTemplate = VALID_USER_TEMPLATE; + final Settings.Builder builder = Settings.builder() + .put(buildLdapSettings(ldapUrls(), userTemplate, groupSearchBase, LdapSearchScope.SUB_TREE)) + .putList(DelegatedAuthorizationSettings.AUTHZ_REALMS.getKey(), "mock_lookup"); + + if (randomBoolean()) { + // maybe disable caching + builder.put(CachingUsernamePasswordRealmSettings.CACHE_TTL_SETTING.getKey(), -1); + } + + final Settings realmSettings = builder.build(); + final Environment env = TestEnvironment.newEnvironment(defaultGlobalSettings); + RealmConfig config = new RealmConfig("test-ldap-realm", realmSettings, defaultGlobalSettings, env, threadPool.getThreadContext()); + + final LdapSessionFactory ldapFactory = new LdapSessionFactory(config, sslService, threadPool); + final DnRoleMapper roleMapper = buildGroupAsRoleMapper(resourceWatcherService); + final LdapRealm ldap = new LdapRealm(LdapRealmSettings.LDAP_TYPE, config, ldapFactory, roleMapper, threadPool); + + final MockLookupRealm mockLookup = new MockLookupRealm(new RealmConfig("mock_lookup", Settings.EMPTY, defaultGlobalSettings, env, + threadPool.getThreadContext())); + + ldap.initialize(Arrays.asList(ldap, mockLookup), licenseState); + mockLookup.initialize(Arrays.asList(ldap, mockLookup), licenseState); + + PlainActionFuture future = new PlainActionFuture<>(); + ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, new SecureString(PASSWORD)), future); + final AuthenticationResult result1 = future.actionGet(); + assertThat(result1.getStatus(), equalTo(AuthenticationResult.Status.CONTINUE)); + assertThat(result1.getMessage(), + equalTo("the principal [" + VALID_USERNAME + "] was authenticated, but no user could be found in realms [mock/mock_lookup]")); + + future = new PlainActionFuture<>(); + final User fakeUser = new User(VALID_USERNAME, "fake_role"); + mockLookup.registerUser(fakeUser); + ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, new SecureString(PASSWORD)), future); + final AuthenticationResult result2 = future.actionGet(); + assertThat(result2.getStatus(), equalTo(AuthenticationResult.Status.SUCCESS)); + assertThat(result2.getUser(), sameInstance(fakeUser)); + } + public void testLdapRealmSelectsLdapSessionFactory() throws Exception { String groupSearchBase = "o=sevenSeas"; String userTemplate = VALID_USER_TEMPLATE; @@ -279,7 +343,8 @@ public class LdapRealmTests extends LdapTestCase { .put("group_search.scope", LdapSearchScope.SUB_TREE) .put("ssl.verification_mode", VerificationMode.CERTIFICATE) .build(); - RealmConfig config = new RealmConfig("test-ldap-realm-user-search", settings, defaultGlobalSettings, TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); + RealmConfig config = new RealmConfig("test-ldap-realm-user-search", settings, defaultGlobalSettings, + TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> LdapRealm.sessionFactory(config, null, threadPool, LdapRealmSettings.LDAP_TYPE)); assertThat(e.getMessage(), @@ -295,7 +360,8 @@ public class LdapRealmTests extends LdapTestCase { .put("group_search.scope", LdapSearchScope.SUB_TREE) .put("ssl.verification_mode", VerificationMode.CERTIFICATE) .build(); - RealmConfig config = new RealmConfig("test-ldap-realm-user-search", settings, defaultGlobalSettings, TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); + RealmConfig config = new RealmConfig("test-ldap-realm-user-search", settings, defaultGlobalSettings, + TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> LdapRealm.sessionFactory(config, null, threadPool, LdapRealmSettings.LDAP_TYPE)); assertThat(e.getMessage(), @@ -312,11 +378,13 @@ public class LdapRealmTests extends LdapTestCase { .put(DnRoleMapperSettings.ROLE_MAPPING_FILE_SETTING.getKey(), getDataPath("/org/elasticsearch/xpack/security/authc/support/role_mapping.yml")) .build(); - RealmConfig config = new RealmConfig("test-ldap-realm-userdn", settings, defaultGlobalSettings, TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); + RealmConfig config = new RealmConfig("test-ldap-realm-userdn", settings, defaultGlobalSettings, + TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); LdapSessionFactory ldapFactory = new LdapSessionFactory(config, sslService, threadPool); LdapRealm ldap = new LdapRealm(LdapRealmSettings.LDAP_TYPE, config, ldapFactory, new DnRoleMapper(config, resourceWatcherService), threadPool); + ldap.initialize(Collections.singleton(ldap), licenseState); PlainActionFuture future = new PlainActionFuture<>(); ldap.authenticate(new UsernamePasswordToken("Horatio Hornblower", new SecureString(PASSWORD)), future); @@ -339,10 +407,12 @@ public class LdapRealmTests extends LdapTestCase { String groupSearchBase = "o=sevenSeas"; String userTemplate = VALID_USER_TEMPLATE; Settings settings = buildLdapSettings(new String[] { url.toString() }, userTemplate, groupSearchBase, LdapSearchScope.SUB_TREE); - RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); + RealmConfig config = new RealmConfig("test-ldap-realm", settings, defaultGlobalSettings, + TestEnvironment.newEnvironment(defaultGlobalSettings), new ThreadContext(defaultGlobalSettings)); LdapSessionFactory ldapFactory = new LdapSessionFactory(config, sslService, threadPool); LdapRealm ldap = new LdapRealm(LdapRealmSettings.LDAP_TYPE, config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService), threadPool); + ldap.initialize(Collections.singleton(ldap), licenseState); PlainActionFuture future = new PlainActionFuture<>(); ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, new SecureString(PASSWORD)), future); @@ -386,6 +456,7 @@ public class LdapRealmTests extends LdapTestCase { LdapSessionFactory ldapFactory = new LdapSessionFactory(config, new SSLService(globalSettings, env), threadPool); LdapRealm realm = new LdapRealm(LdapRealmSettings.LDAP_TYPE, config, ldapFactory, new DnRoleMapper(config, resourceWatcherService), threadPool); + realm.initialize(Collections.singleton(realm), licenseState); PlainActionFuture> future = new PlainActionFuture<>(); realm.usageStats(future); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiRealmTests.java index 44d5859d12b..8d4c5d75c73 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiRealmTests.java @@ -12,10 +12,13 @@ import org.elasticsearch.common.settings.MockSecureSettings; import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; +import org.elasticsearch.xpack.core.security.authc.Realm; import org.elasticsearch.xpack.core.security.authc.RealmConfig; import org.elasticsearch.xpack.core.security.authc.RealmSettings; import org.elasticsearch.xpack.core.security.authc.pki.PkiRealmSettings; @@ -23,6 +26,7 @@ import org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken import org.elasticsearch.xpack.core.security.support.NoOpLogger; import org.elasticsearch.xpack.core.security.user.User; import org.elasticsearch.xpack.core.ssl.SSLConfigurationSettings; +import org.elasticsearch.xpack.security.authc.support.MockLookupRealm; import org.elasticsearch.xpack.security.authc.support.UserRoleMapper; import org.junit.Before; import org.mockito.Mockito; @@ -43,9 +47,11 @@ import java.util.regex.Pattern; import static org.hamcrest.Matchers.arrayContainingInAnyOrder; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.sameInstance; import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -56,12 +62,15 @@ import static org.mockito.Mockito.when; public class PkiRealmTests extends ESTestCase { private Settings globalSettings; + private XPackLicenseState licenseState; @Before public void setup() throws Exception { globalSettings = Settings.builder() .put("path.home", createTempDir()) .build(); + licenseState = mock(XPackLicenseState.class); + when(licenseState.isAuthorizationRealmAllowed()).thenReturn(true); } public void testTokenSupport() { @@ -98,28 +107,14 @@ public class PkiRealmTests extends ESTestCase { } private void assertSuccessfulAuthentication(Set roles) throws Exception { - String dn = "CN=Elasticsearch Test Node,"; - final String expectedUsername = "Elasticsearch Test Node"; - X509Certificate certificate = readCert(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")); - X509AuthenticationToken token = new X509AuthenticationToken(new X509Certificate[] { certificate }, "Elasticsearch Test Node", dn); - UserRoleMapper roleMapper = mock(UserRoleMapper.class); - PkiRealm realm = new PkiRealm(new RealmConfig("", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), - new ThreadContext(globalSettings)), roleMapper); + X509AuthenticationToken token = buildToken(); + UserRoleMapper roleMapper = buildRoleMapper(roles, token.dn()); + PkiRealm realm = buildRealm(roleMapper, Settings.EMPTY); verify(roleMapper).refreshRealmOnChange(realm); - Mockito.doAnswer(invocation -> { - final UserRoleMapper.UserData userData = (UserRoleMapper.UserData) invocation.getArguments()[0]; - final ActionListener> listener = (ActionListener>) invocation.getArguments()[1]; - if (userData.getDn().equals(dn)) { - listener.onResponse(roles); - } else { - listener.onFailure(new IllegalArgumentException("Expected DN '" + dn + "' but was '" + userData + "'")); - } - return null; - }).when(roleMapper).resolveRoles(any(UserRoleMapper.UserData.class), any(ActionListener.class)); - PlainActionFuture future = new PlainActionFuture<>(); - realm.authenticate(token, future); - final AuthenticationResult result = future.actionGet(); + final String expectedUsername = token.principal(); + final AuthenticationResult result = authenticate(token, realm); + final PlainActionFuture future; assertThat(result.getStatus(), is(AuthenticationResult.Status.SUCCESS)); User user = result.getUser(); assertThat(user, is(notNullValue())); @@ -149,17 +144,54 @@ public class PkiRealmTests extends ESTestCase { verifyNoMoreInteractions(roleMapper); } + private UserRoleMapper buildRoleMapper(Set roles, String dn) { + UserRoleMapper roleMapper = mock(UserRoleMapper.class); + Mockito.doAnswer(invocation -> { + final UserRoleMapper.UserData userData = (UserRoleMapper.UserData) invocation.getArguments()[0]; + final ActionListener> listener = (ActionListener>) invocation.getArguments()[1]; + if (userData.getDn().equals(dn)) { + listener.onResponse(roles); + } else { + listener.onFailure(new IllegalArgumentException("Expected DN '" + dn + "' but was '" + userData + "'")); + } + return null; + }).when(roleMapper).resolveRoles(any(UserRoleMapper.UserData.class), any(ActionListener.class)); + return roleMapper; + } + + private PkiRealm buildRealm(UserRoleMapper roleMapper, Settings realmSettings, Realm... otherRealms) { + PkiRealm realm = new PkiRealm(new RealmConfig("", realmSettings, globalSettings, TestEnvironment.newEnvironment(globalSettings), + new ThreadContext(globalSettings)), roleMapper); + List allRealms = CollectionUtils.arrayAsArrayList(otherRealms); + allRealms.add(realm); + Collections.shuffle(allRealms, random()); + realm.initialize(allRealms, licenseState); + return realm; + } + + private X509AuthenticationToken buildToken() throws Exception { + X509Certificate certificate = readCert(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")); + return new X509AuthenticationToken(new X509Certificate[]{certificate}, "Elasticsearch Test Node", "CN=Elasticsearch Test Node,"); + } + + private AuthenticationResult authenticate(X509AuthenticationToken token, PkiRealm realm) { + PlainActionFuture future = new PlainActionFuture<>(); + realm.authenticate(token, future); + return future.actionGet(); + } + public void testCustomUsernamePattern() throws Exception { + ThreadContext threadContext = new ThreadContext(globalSettings); X509Certificate certificate = readCert(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")); UserRoleMapper roleMapper = mock(UserRoleMapper.class); - PkiRealm realm = new PkiRealm(new RealmConfig("", Settings.builder().put("username_pattern", "OU=(.*?),").build(), globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings)), - roleMapper); + PkiRealm realm = new PkiRealm(new RealmConfig("", Settings.builder().put("username_pattern", "OU=(.*?),").build(), globalSettings, + TestEnvironment.newEnvironment(globalSettings), threadContext), roleMapper); + realm.initialize(Collections.emptyList(), licenseState); Mockito.doAnswer(invocation -> { ActionListener> listener = (ActionListener>) invocation.getArguments()[1]; listener.onResponse(Collections.emptySet()); return null; }).when(roleMapper).resolveRoles(any(UserRoleMapper.UserData.class), any(ActionListener.class)); - ThreadContext threadContext = new ThreadContext(Settings.EMPTY); threadContext.putTransient(PkiRealm.PKI_CERT_HEADER_NAME, new X509Certificate[] { certificate }); X509AuthenticationToken token = realm.token(threadContext); @@ -182,15 +214,16 @@ public class PkiRealmTests extends ESTestCase { .put("truststore.path", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks")) .setSecureSettings(secureSettings) .build(); + ThreadContext threadContext = new ThreadContext(globalSettings); PkiRealm realm = new PkiRealm(new RealmConfig("", settings, globalSettings, TestEnvironment.newEnvironment(globalSettings), - new ThreadContext(globalSettings)), roleMapper); + threadContext), roleMapper); + realm.initialize(Collections.emptyList(), licenseState); Mockito.doAnswer(invocation -> { ActionListener> listener = (ActionListener>) invocation.getArguments()[1]; listener.onResponse(Collections.emptySet()); return null; }).when(roleMapper).resolveRoles(any(UserRoleMapper.UserData.class), any(ActionListener.class)); - ThreadContext threadContext = new ThreadContext(Settings.EMPTY); threadContext.putTransient(PkiRealm.PKI_CERT_HEADER_NAME, new X509Certificate[] { certificate }); X509AuthenticationToken token = realm.token(threadContext); @@ -213,15 +246,16 @@ public class PkiRealmTests extends ESTestCase { getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode-client-profile.jks")) .setSecureSettings(secureSettings) .build(); + final ThreadContext threadContext = new ThreadContext(globalSettings); PkiRealm realm = new PkiRealm(new RealmConfig("", settings, globalSettings, TestEnvironment.newEnvironment(globalSettings), - new ThreadContext(globalSettings)), roleMapper); + threadContext), roleMapper); + realm.initialize(Collections.emptyList(), licenseState); Mockito.doAnswer(invocation -> { ActionListener> listener = (ActionListener>) invocation.getArguments()[1]; listener.onResponse(Collections.emptySet()); return null; }).when(roleMapper).resolveRoles(any(UserRoleMapper.UserData.class), any(ActionListener.class)); - ThreadContext threadContext = new ThreadContext(Settings.EMPTY); threadContext.putTransient(PkiRealm.PKI_CERT_HEADER_NAME, new X509Certificate[] { certificate }); X509AuthenticationToken token = realm.token(threadContext); @@ -307,6 +341,33 @@ public class PkiRealmTests extends ESTestCase { assertSettingDeprecationsAndWarnings(new Setting[] { SSLConfigurationSettings.withoutPrefix().legacyTruststorePassword }); } + public void testDelegatedAuthorization() throws Exception { + final X509AuthenticationToken token = buildToken(); + + final MockLookupRealm otherRealm = new MockLookupRealm(new RealmConfig("other_realm", Settings.EMPTY, globalSettings, + TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings))); + final User lookupUser = new User(token.principal()); + otherRealm.registerUser(lookupUser); + + final Settings realmSettings = Settings.builder() + .putList("authorization_realms", "other_realm") + .build(); + final UserRoleMapper roleMapper = buildRoleMapper(Collections.emptySet(), token.dn()); + final PkiRealm pkiRealm = buildRealm(roleMapper, realmSettings, otherRealm); + + AuthenticationResult result = authenticate(token, pkiRealm); + assertThat(result.getStatus(), equalTo(AuthenticationResult.Status.SUCCESS)); + assertThat(result.getUser(), sameInstance(lookupUser)); + + // check that the authorizing realm is consulted even for cached principals + final User lookupUser2 = new User(token.principal()); + otherRealm.registerUser(lookupUser2); + + result = authenticate(token, pkiRealm); + assertThat(result.getStatus(), equalTo(AuthenticationResult.Status.SUCCESS)); + assertThat(result.getUser(), sameInstance(lookupUser2)); + } + static X509Certificate readCert(Path path) throws Exception { try (InputStream in = Files.newInputStream(path)) { CertificateFactory factory = CertificateFactory.getInstance("X.509"); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlRealmTests.java index 980abc46831..2ecfdb50230 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlRealmTests.java @@ -14,18 +14,24 @@ import org.elasticsearch.common.settings.SettingsException; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.test.http.MockResponse; import org.elasticsearch.test.http.MockWebServer; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; +import org.elasticsearch.xpack.core.security.authc.Realm; import org.elasticsearch.xpack.core.security.authc.RealmConfig; import org.elasticsearch.xpack.core.security.authc.RealmSettings; import org.elasticsearch.xpack.core.security.authc.saml.SamlRealmSettings; +import org.elasticsearch.xpack.core.security.authc.support.DelegatedAuthorizationSettings; +import org.elasticsearch.xpack.core.security.user.User; import org.elasticsearch.xpack.core.ssl.CertParsingUtils; import org.elasticsearch.xpack.core.ssl.PemUtils; import org.elasticsearch.xpack.core.ssl.SSLService; import org.elasticsearch.xpack.core.ssl.TestsSSLService; +import org.elasticsearch.xpack.security.authc.support.MockLookupRealm; import org.elasticsearch.xpack.security.authc.support.UserRoleMapper; +import org.hamcrest.Matchers; import org.junit.Before; import org.mockito.Mockito; import org.opensaml.saml.common.xml.SAMLConstants; @@ -71,6 +77,7 @@ import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Basic unit tests for the SAMLRealm @@ -83,9 +90,16 @@ public class SamlRealmTests extends SamlTestCase { private static final String REALM_NAME = "my-saml"; private static final String REALM_SETTINGS_PREFIX = "xpack.security.authc.realms." + REALM_NAME; + private Settings globalSettings; + private Environment env; + private ThreadContext threadContext; + @Before - public void initRealm() throws PrivilegedActionException { + public void setupEnv() throws PrivilegedActionException { SamlUtils.initialize(logger); + globalSettings = Settings.builder().put("path.home", createTempDir()).build(); + env = TestEnvironment.newEnvironment(globalSettings); + threadContext = new ThreadContext(globalSettings); } public void testReadIdpMetadataFromFile() throws Exception { @@ -140,15 +154,70 @@ public class SamlRealmTests extends SamlTestCase { } public void testAuthenticateWithRoleMapping() throws Exception { + final UserRoleMapper roleMapper = mock(UserRoleMapper.class); + AtomicReference userData = new AtomicReference<>(); + Mockito.doAnswer(invocation -> { + assert invocation.getArguments().length == 2; + userData.set((UserRoleMapper.UserData) invocation.getArguments()[0]); + ActionListener> listener = (ActionListener>) invocation.getArguments()[1]; + listener.onResponse(Collections.singleton("superuser")); + return null; + }).when(roleMapper).resolveRoles(any(UserRoleMapper.UserData.class), any(ActionListener.class)); + final boolean useNameId = randomBoolean(); final boolean principalIsEmailAddress = randomBoolean(); final Boolean populateUserMetadata = randomFrom(Boolean.TRUE, Boolean.FALSE, null); + + AuthenticationResult result = performAuthentication(roleMapper, useNameId, principalIsEmailAddress, populateUserMetadata, false); + assertThat(result.getUser().roles(), arrayContainingInAnyOrder("superuser")); + if (populateUserMetadata == Boolean.FALSE) { + // TODO : "saml_nameid" should be null too, but the logout code requires it for now. + assertThat(result.getUser().metadata().get("saml_uid"), nullValue()); + } else { + final String nameIdValue = principalIsEmailAddress ? "clint.barton@shield.gov" : "clint.barton"; + final String uidValue = principalIsEmailAddress ? "cbarton@shield.gov" : "cbarton"; + assertThat(result.getUser().metadata().get("saml_nameid"), equalTo(nameIdValue)); + assertThat(result.getUser().metadata().get("saml_uid"), instanceOf(Iterable.class)); + assertThat((Iterable) result.getUser().metadata().get("saml_uid"), contains(uidValue)); + } + + assertThat(userData.get().getUsername(), equalTo(useNameId ? "clint.barton" : "cbarton")); + assertThat(userData.get().getGroups(), containsInAnyOrder("avengers", "shield")); + } + + public void testAuthenticateWithAuthorizingRealm() throws Exception { final UserRoleMapper roleMapper = mock(UserRoleMapper.class); + Mockito.doAnswer(invocation -> { + assert invocation.getArguments().length == 2; + ActionListener> listener = (ActionListener>) invocation.getArguments()[1]; + listener.onFailure(new RuntimeException("Role mapping should not be called")); + return null; + }).when(roleMapper).resolveRoles(any(UserRoleMapper.UserData.class), any(ActionListener.class)); + + final boolean useNameId = randomBoolean(); + final boolean principalIsEmailAddress = randomBoolean(); + + AuthenticationResult result = performAuthentication(roleMapper, useNameId, principalIsEmailAddress, null, true); + assertThat(result.getUser().roles(), arrayContainingInAnyOrder("lookup_user_role")); + assertThat(result.getUser().fullName(), equalTo("Clinton Barton")); + assertThat(result.getUser().metadata().entrySet(), Matchers.iterableWithSize(1)); + assertThat(result.getUser().metadata().get("is_lookup"), Matchers.equalTo(true)); + } + + private AuthenticationResult performAuthentication(UserRoleMapper roleMapper, boolean useNameId, boolean principalIsEmailAddress, + Boolean populateUserMetadata, boolean useAuthorizingRealm) throws Exception { final EntityDescriptor idp = mockIdp(); final SpConfiguration sp = new SpConfiguration("", "https://saml/", null, null, null, Collections.emptyList()); final SamlAuthenticator authenticator = mock(SamlAuthenticator.class); final SamlLogoutRequestHandler logoutHandler = mock(SamlLogoutRequestHandler.class); + final String userPrincipal = useNameId ? "clint.barton" : "cbarton"; + final String nameIdValue = principalIsEmailAddress ? "clint.barton@shield.gov" : "clint.barton"; + final String uidValue = principalIsEmailAddress ? "cbarton@shield.gov" : "cbarton"; + + final MockLookupRealm lookupRealm = new MockLookupRealm( + new RealmConfig("mock_lookup", Settings.EMPTY,globalSettings, env, threadContext)); + final Settings.Builder settingsBuilder = Settings.builder() .put(SamlRealmSettings.PRINCIPAL_ATTRIBUTE.name(), useNameId ? "nameid" : "uid") .put(SamlRealmSettings.GROUPS_ATTRIBUTE.name(), "groups") @@ -161,15 +230,20 @@ public class SamlRealmTests extends SamlTestCase { if (populateUserMetadata != null) { settingsBuilder.put(SamlRealmSettings.POPULATE_USER_METADATA.getKey(), populateUserMetadata.booleanValue()); } + if (useAuthorizingRealm) { + settingsBuilder.putList(DelegatedAuthorizationSettings.AUTHZ_REALMS.getKey(), lookupRealm.name()); + lookupRealm.registerUser(new User(userPrincipal, new String[]{ "lookup_user_role" }, "Clinton Barton", "cbarton@shield.gov", + Collections.singletonMap("is_lookup", true), true)); + } + final Settings realmSettings = settingsBuilder.build(); - final RealmConfig config = realmConfigFromRealmSettings(realmSettings); - final SamlRealm realm = new SamlRealm(config, roleMapper, authenticator, logoutHandler, () -> idp, sp); + + initializeRealms(realm, lookupRealm); + final SamlToken token = new SamlToken(new byte[0], Collections.singletonList("")); - final String nameIdValue = principalIsEmailAddress ? "clint.barton@shield.gov" : "clint.barton"; - final String uidValue = principalIsEmailAddress ? "cbarton@shield.gov" : "cbarton"; final SamlAttributes attributes = new SamlAttributes( new SamlNameId(NameIDType.PERSISTENT, nameIdValue, idp.getEntityID(), sp.getEntityId(), null), randomAlphaOfLength(16), @@ -178,36 +252,27 @@ public class SamlRealmTests extends SamlTestCase { new SamlAttributes.SamlAttribute("urn:oid:1.3.6.1.4.1.5923.1.5.1.1", "groups", Arrays.asList("avengers", "shield")), new SamlAttributes.SamlAttribute("urn:oid:0.9.2342.19200300.100.1.3", "mail", Arrays.asList("cbarton@shield.gov")) )); - Mockito.when(authenticator.authenticate(token)).thenReturn(attributes); - - AtomicReference userData = new AtomicReference<>(); - Mockito.doAnswer(invocation -> { - assert invocation.getArguments().length == 2; - userData.set((UserRoleMapper.UserData) invocation.getArguments()[0]); - ActionListener> listener = (ActionListener>) invocation.getArguments()[1]; - listener.onResponse(Collections.singleton("superuser")); - return null; - }).when(roleMapper).resolveRoles(any(UserRoleMapper.UserData.class), any(ActionListener.class)); + when(authenticator.authenticate(token)).thenReturn(attributes); final PlainActionFuture future = new PlainActionFuture<>(); realm.authenticate(token, future); final AuthenticationResult result = future.get(); assertThat(result, notNullValue()); assertThat(result.getStatus(), equalTo(AuthenticationResult.Status.SUCCESS)); - assertThat(result.getUser().principal(), equalTo(useNameId ? "clint.barton" : "cbarton")); + assertThat(result.getUser().principal(), equalTo(userPrincipal)); assertThat(result.getUser().email(), equalTo("cbarton@shield.gov")); - assertThat(result.getUser().roles(), arrayContainingInAnyOrder("superuser")); - if (populateUserMetadata == Boolean.FALSE) { - // TODO : "saml_nameid" should be null too, but the logout code requires it for now. - assertThat(result.getUser().metadata().get("saml_uid"), nullValue()); - } else { - assertThat(result.getUser().metadata().get("saml_nameid"), equalTo(nameIdValue)); - assertThat(result.getUser().metadata().get("saml_uid"), instanceOf(Iterable.class)); - assertThat((Iterable) result.getUser().metadata().get("saml_uid"), contains(uidValue)); - } - assertThat(userData.get().getUsername(), equalTo(useNameId ? "clint.barton" : "cbarton")); - assertThat(userData.get().getGroups(), containsInAnyOrder("avengers", "shield")); + return result; + } + + private void initializeRealms(Realm... realms) { + XPackLicenseState licenseState = mock(XPackLicenseState.class); + when(licenseState.isAuthorizationRealmAllowed()).thenReturn(true); + + final List realmList = Arrays.asList(realms); + for (Realm realm : realms) { + realm.initialize(realmList, licenseState); + } } public void testAttributeSelectionWithRegex() throws Exception { @@ -291,7 +356,7 @@ public class SamlRealmTests extends SamlTestCase { Collections.singletonList( new SamlAttributes.SamlAttribute("urn:oid:0.9.2342.19200300.100.1.3", "mail", Collections.singletonList(mail)) )); - Mockito.when(authenticator.authenticate(token)).thenReturn(attributes); + when(authenticator.authenticate(token)).thenReturn(attributes); final PlainActionFuture future = new PlainActionFuture<>(); realm.authenticate(token, future); @@ -515,8 +580,8 @@ public class SamlRealmTests extends SamlTestCase { final EntityDescriptor idp = mockIdp(); final IDPSSODescriptor role = mock(IDPSSODescriptor.class); final SingleLogoutService slo = SamlUtils.buildObject(SingleLogoutService.class, SingleLogoutService.DEFAULT_ELEMENT_NAME); - Mockito.when(idp.getRoleDescriptors(IDPSSODescriptor.DEFAULT_ELEMENT_NAME)).thenReturn(Collections.singletonList(role)); - Mockito.when(role.getSingleLogoutServices()).thenReturn(Collections.singletonList(slo)); + when(idp.getRoleDescriptors(IDPSSODescriptor.DEFAULT_ELEMENT_NAME)).thenReturn(Collections.singletonList(role)); + when(role.getSingleLogoutServices()).thenReturn(Collections.singletonList(slo)); slo.setBinding(SAMLConstants.SAML2_REDIRECT_BINDING_URI); slo.setLocation("https://logout.saml/"); @@ -553,7 +618,7 @@ public class SamlRealmTests extends SamlTestCase { private EntityDescriptor mockIdp() { final EntityDescriptor descriptor = mock(EntityDescriptor.class); - Mockito.when(descriptor.getEntityID()).thenReturn("https://idp.saml/"); + when(descriptor.getEntityID()).thenReturn("https://idp.saml/"); return descriptor; } @@ -585,9 +650,7 @@ public class SamlRealmTests extends SamlTestCase { } private RealmConfig realmConfigFromRealmSettings(Settings realmSettings) { - final Settings globalSettings = Settings.builder().put("path.home", createTempDir()).build(); - final Environment env = TestEnvironment.newEnvironment(globalSettings); - return new RealmConfig(REALM_NAME, realmSettings, globalSettings, env, new ThreadContext(globalSettings)); + return new RealmConfig(REALM_NAME, realmSettings, globalSettings, env, threadContext); } private RealmConfig realmConfigFromGlobalSettings(Settings globalSettings) { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java index 052758d8371..e9e8908c584 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java @@ -31,6 +31,7 @@ import java.util.List; import java.util.Locale; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import static java.util.Collections.emptyMap; import static org.hamcrest.Matchers.arrayContaining; @@ -39,6 +40,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.sameInstance; @@ -341,6 +343,33 @@ public class CachingUsernamePasswordRealmTests extends ESTestCase { assertThat(e.getMessage(), containsString("lookup exception")); } + public void testReturnDifferentObjectFromCache() throws Exception { + final AtomicReference userArg = new AtomicReference<>(); + final AtomicReference result = new AtomicReference<>(); + Realm realm = new AlwaysAuthenticateCachingRealm(globalSettings, threadPool) { + @Override + protected void handleCachedAuthentication(User user, ActionListener listener) { + userArg.set(user); + listener.onResponse(result.get()); + } + }; + PlainActionFuture future = new PlainActionFuture<>(); + realm.authenticate(new UsernamePasswordToken("user", new SecureString("pass")), future); + final AuthenticationResult result1 = future.actionGet(); + assertThat(result1, notNullValue()); + assertThat(result1.getUser(), notNullValue()); + assertThat(result1.getUser().principal(), equalTo("user")); + + final AuthenticationResult result2 = AuthenticationResult.success(new User("user")); + result.set(result2); + + future = new PlainActionFuture<>(); + realm.authenticate(new UsernamePasswordToken("user", new SecureString("pass")), future); + final AuthenticationResult result3 = future.actionGet(); + assertThat(result3, sameInstance(result2)); + assertThat(userArg.get(), sameInstance(result1.getUser())); + } + public void testSingleAuthPerUserLimit() throws Exception { final String username = "username"; final SecureString password = SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING; diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/DelegatedAuthorizationSupportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/DelegatedAuthorizationSupportTests.java new file mode 100644 index 00000000000..8f0d360b759 --- /dev/null +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/DelegatedAuthorizationSupportTests.java @@ -0,0 +1,189 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.security.authc.support; + +import org.elasticsearch.ElasticsearchSecurityException; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.env.Environment; +import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; +import org.elasticsearch.xpack.core.security.authc.Realm; +import org.elasticsearch.xpack.core.security.authc.RealmConfig; +import org.elasticsearch.xpack.core.security.user.User; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +import static org.elasticsearch.common.Strings.collectionToDelimitedString; +import static org.hamcrest.Matchers.arrayContaining; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class DelegatedAuthorizationSupportTests extends ESTestCase { + + private List realms; + private Settings globalSettings; + private ThreadContext threadContext; + private Environment env; + + @Before + public void setupRealms() { + globalSettings = Settings.builder() + .put("path.home", createTempDir()) + .build(); + env = TestEnvironment.newEnvironment(globalSettings); + threadContext = new ThreadContext(globalSettings); + + final int realmCount = randomIntBetween(5, 9); + realms = new ArrayList<>(realmCount); + for (int i = 1; i <= realmCount; i++) { + realms.add(new MockLookupRealm(buildRealmConfig("lookup-" + i, Settings.EMPTY))); + } + shuffle(realms); + } + + private List shuffle(List list) { + Collections.shuffle(list, random()); + return list; + } + + private RealmConfig buildRealmConfig(String name, Settings settings) { + return new RealmConfig(name, settings, globalSettings, env, threadContext); + } + + public void testEmptyDelegationList() throws ExecutionException, InterruptedException { + final XPackLicenseState license = getLicenseState(true); + final DelegatedAuthorizationSupport das = new DelegatedAuthorizationSupport(realms, buildRealmConfig("r", Settings.EMPTY), license); + assertThat(das.hasDelegation(), equalTo(false)); + final PlainActionFuture future = new PlainActionFuture<>(); + das.resolve("any", future); + final AuthenticationResult result = future.get(); + assertThat(result.getStatus(), equalTo(AuthenticationResult.Status.CONTINUE)); + assertThat(result.getUser(), nullValue()); + assertThat(result.getMessage(), equalTo("No [authorization_realms] have been configured")); + } + + public void testMissingRealmInDelegationList() { + final XPackLicenseState license = getLicenseState(true); + final Settings settings = Settings.builder() + .putList("authorization_realms", "no-such-realm") + .build(); + final IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> + new DelegatedAuthorizationSupport(realms, buildRealmConfig("r", settings), license) + ); + assertThat(ex.getMessage(), equalTo("configured authorization realm [no-such-realm] does not exist (or is not enabled)")); + } + + public void testDelegationChainsAreRejected() { + final XPackLicenseState license = getLicenseState(true); + final Settings settings = Settings.builder() + .putList("authorization_realms", "lookup-1", "lookup-2", "lookup-3") + .build(); + globalSettings = Settings.builder() + .put(globalSettings) + .putList("xpack.security.authc.realms.lookup-2.authorization_realms", "lookup-1") + .build(); + final IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> + new DelegatedAuthorizationSupport(realms, buildRealmConfig("realm1", settings), license) + ); + assertThat(ex.getMessage(), + equalTo("cannot use realm [mock/lookup-2] as an authorization realm - it is already delegating authorization to [[lookup-1]]")); + } + + public void testMatchInDelegationList() throws Exception { + final XPackLicenseState license = getLicenseState(true); + final List useRealms = shuffle(randomSubsetOf(randomIntBetween(1, realms.size()), realms)); + final Settings settings = Settings.builder() + .putList("authorization_realms", useRealms.stream().map(Realm::name).collect(Collectors.toList())) + .build(); + final User user = new User("my_user"); + randomFrom(useRealms).registerUser(user); + final DelegatedAuthorizationSupport das = new DelegatedAuthorizationSupport(realms, buildRealmConfig("r", settings), license); + assertThat(das.hasDelegation(), equalTo(true)); + final PlainActionFuture future = new PlainActionFuture<>(); + das.resolve("my_user", future); + final AuthenticationResult result = future.get(); + assertThat(result.getStatus(), equalTo(AuthenticationResult.Status.SUCCESS)); + assertThat(result.getUser(), sameInstance(user)); + } + + public void testRealmsAreOrdered() throws Exception { + final XPackLicenseState license = getLicenseState(true); + final List useRealms = shuffle(randomSubsetOf(randomIntBetween(3, realms.size()), realms)); + final List names = useRealms.stream().map(Realm::name).collect(Collectors.toList()); + final Settings settings = Settings.builder() + .putList("authorization_realms", names) + .build(); + final List users = new ArrayList<>(names.size()); + final String username = randomAlphaOfLength(8); + for (MockLookupRealm r : useRealms) { + final User user = new User(username, "role_" + r.name()); + users.add(user); + r.registerUser(user); + } + + final DelegatedAuthorizationSupport das = new DelegatedAuthorizationSupport(realms, buildRealmConfig("r", settings), license); + assertThat(das.hasDelegation(), equalTo(true)); + final PlainActionFuture future = new PlainActionFuture<>(); + das.resolve(username, future); + final AuthenticationResult result = future.get(); + assertThat(result.getStatus(), equalTo(AuthenticationResult.Status.SUCCESS)); + assertThat(result.getUser(), sameInstance(users.get(0))); + assertThat(result.getUser().roles(), arrayContaining("role_" + useRealms.get(0).name())); + } + + public void testNoMatchInDelegationList() throws Exception { + final XPackLicenseState license = getLicenseState(true); + final List useRealms = shuffle(randomSubsetOf(randomIntBetween(1, realms.size()), realms)); + final Settings settings = Settings.builder() + .putList("authorization_realms", useRealms.stream().map(Realm::name).collect(Collectors.toList())) + .build(); + final DelegatedAuthorizationSupport das = new DelegatedAuthorizationSupport(realms, buildRealmConfig("r", settings), license); + assertThat(das.hasDelegation(), equalTo(true)); + final PlainActionFuture future = new PlainActionFuture<>(); + das.resolve("my_user", future); + final AuthenticationResult result = future.get(); + assertThat(result.getStatus(), equalTo(AuthenticationResult.Status.CONTINUE)); + assertThat(result.getUser(), nullValue()); + assertThat(result.getMessage(), equalTo("the principal [my_user] was authenticated, but no user could be found in realms [" + + collectionToDelimitedString(useRealms.stream().map(Realm::toString).collect(Collectors.toList()), ",") + "]")); + } + + public void testLicenseRejection() throws Exception { + final XPackLicenseState license = getLicenseState(false); + final Settings settings = Settings.builder() + .putList("authorization_realms", realms.get(0).name()) + .build(); + final DelegatedAuthorizationSupport das = new DelegatedAuthorizationSupport(realms, buildRealmConfig("r", settings), license); + assertThat(das.hasDelegation(), equalTo(true)); + final PlainActionFuture future = new PlainActionFuture<>(); + das.resolve("my_user", future); + final AuthenticationResult result = future.get(); + assertThat(result.getStatus(), equalTo(AuthenticationResult.Status.CONTINUE)); + assertThat(result.getUser(), nullValue()); + assertThat(result.getMessage(), equalTo("authorization_realms are not permitted")); + assertThat(result.getException(), instanceOf(ElasticsearchSecurityException.class)); + assertThat(result.getException().getMessage(), equalTo("current license is non-compliant for [authorization_realms]")); + } + + private XPackLicenseState getLicenseState(boolean authzRealmsAllowed) { + final XPackLicenseState license = mock(XPackLicenseState.class); + when(license.isAuthorizationRealmAllowed()).thenReturn(authzRealmsAllowed); + return license; + } +} diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/MockLookupRealm.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/MockLookupRealm.java new file mode 100644 index 00000000000..01700347f50 --- /dev/null +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/MockLookupRealm.java @@ -0,0 +1,52 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.security.authc.support; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; +import org.elasticsearch.xpack.core.security.authc.AuthenticationToken; +import org.elasticsearch.xpack.core.security.authc.Realm; +import org.elasticsearch.xpack.core.security.authc.RealmConfig; +import org.elasticsearch.xpack.core.security.user.User; + +import java.util.HashMap; +import java.util.Map; + +public class MockLookupRealm extends Realm { + + private final Map lookup; + + public MockLookupRealm(RealmConfig config) { + super("mock", config); + lookup = new HashMap<>(); + } + + public void registerUser(User user) { + this.lookup.put(user.principal(), user); + } + + @Override + public boolean supports(AuthenticationToken token) { + return false; + } + + @Override + public AuthenticationToken token(ThreadContext context) { + return null; + } + + @Override + public void authenticate(AuthenticationToken token, ActionListener listener) { + listener.onResponse(AuthenticationResult.notHandled()); + } + + @Override + public void lookupUser(String username, ActionListener listener) { + listener.onResponse(lookup.get(username)); + } +} diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/RealmUserLookupTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/RealmUserLookupTests.java new file mode 100644 index 00000000000..78be4b3ddf4 --- /dev/null +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/RealmUserLookupTests.java @@ -0,0 +1,128 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.security.authc.support; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.env.Environment; +import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; +import org.elasticsearch.xpack.core.security.authc.AuthenticationToken; +import org.elasticsearch.xpack.core.security.authc.Realm; +import org.elasticsearch.xpack.core.security.authc.RealmConfig; +import org.elasticsearch.xpack.core.security.user.User; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.sameInstance; + +public class RealmUserLookupTests extends ESTestCase { + + private Settings globalSettings; + private ThreadContext threadContext; + private Environment env; + + @Before + public void setup() { + globalSettings = Settings.builder() + .put("path.home", createTempDir()) + .build(); + env = TestEnvironment.newEnvironment(globalSettings); + threadContext = new ThreadContext(globalSettings); + } + + public void testNoRealms() throws Exception { + final RealmUserLookup lookup = new RealmUserLookup(Collections.emptyList(), threadContext); + final PlainActionFuture> listener = new PlainActionFuture<>(); + lookup.lookup(randomAlphaOfLengthBetween(3, 12), listener); + final Tuple tuple = listener.get(); + assertThat(tuple, nullValue()); + } + + public void testUserFound() throws Exception { + final List realms = buildRealms(randomIntBetween(5, 9)); + final RealmUserLookup lookup = new RealmUserLookup(realms, threadContext); + + final MockLookupRealm matchRealm = randomFrom(realms); + final User user = new User(randomAlphaOfLength(5)); + matchRealm.registerUser(user); + + final PlainActionFuture> listener = new PlainActionFuture<>(); + lookup.lookup(user.principal(), listener); + final Tuple tuple = listener.get(); + assertThat(tuple, notNullValue()); + assertThat(tuple.v1(), notNullValue()); + assertThat(tuple.v1(), sameInstance(user)); + assertThat(tuple.v2(), notNullValue()); + assertThat(tuple.v2(), sameInstance(matchRealm)); + } + + public void testUserNotFound() throws Exception { + final List realms = buildRealms(randomIntBetween(5, 9)); + final RealmUserLookup lookup = new RealmUserLookup(realms, threadContext); + + final String username = randomAlphaOfLength(5); + + final PlainActionFuture> listener = new PlainActionFuture<>(); + lookup.lookup(username, listener); + final Tuple tuple = listener.get(); + assertThat(tuple, nullValue()); + } + + public void testRealmException() { + final Realm realm = new Realm("test", new RealmConfig("test", Settings.EMPTY, globalSettings, env, threadContext)) { + @Override + public boolean supports(AuthenticationToken token) { + return false; + } + + @Override + public AuthenticationToken token(ThreadContext context) { + return null; + } + + @Override + public void authenticate(AuthenticationToken token, ActionListener listener) { + listener.onResponse(AuthenticationResult.notHandled()); + } + + @Override + public void lookupUser(String username, ActionListener listener) { + listener.onFailure(new RuntimeException("FAILURE")); + } + }; + final RealmUserLookup lookup = new RealmUserLookup(Collections.singletonList(realm), threadContext); + final PlainActionFuture> listener = new PlainActionFuture<>(); + lookup.lookup("anyone", listener); + final RuntimeException e = expectThrows(RuntimeException.class, listener::actionGet); + assertThat(e.getMessage(), equalTo("FAILURE")); + } + + private List buildRealms(int realmCount) { + final List realms = new ArrayList<>(realmCount); + for (int i = 1; i <= realmCount; i++) { + final RealmConfig config = new RealmConfig("lookup-" + i, Settings.EMPTY, globalSettings, env, threadContext); + final MockLookupRealm realm = new MockLookupRealm(config); + for (int j = 0; j < 5; j++) { + realm.registerUser(new User(randomAlphaOfLengthBetween(6, 12))); + } + realms.add(realm); + } + Collections.shuffle(realms, random()); + return realms; + } +} diff --git a/x-pack/qa/saml-idp-tests/build.gradle b/x-pack/qa/saml-idp-tests/build.gradle index 9dd5d6d848f..11e89d93c8e 100644 --- a/x-pack/qa/saml-idp-tests/build.gradle +++ b/x-pack/qa/saml-idp-tests/build.gradle @@ -37,17 +37,30 @@ integTestCluster { setting 'xpack.security.authc.token.enabled', 'true' setting 'xpack.security.authc.realms.file.type', 'file' setting 'xpack.security.authc.realms.file.order', '0' + // SAML realm 1 (no authorization_realms) setting 'xpack.security.authc.realms.shibboleth.type', 'saml' setting 'xpack.security.authc.realms.shibboleth.order', '1' setting 'xpack.security.authc.realms.shibboleth.idp.entity_id', 'https://test.shibboleth.elastic.local/' setting 'xpack.security.authc.realms.shibboleth.idp.metadata.path', 'idp-metadata.xml' - setting 'xpack.security.authc.realms.shibboleth.sp.entity_id', 'http://mock.http.elastic.local/' + setting 'xpack.security.authc.realms.shibboleth.sp.entity_id', 'http://mock1.http.elastic.local/' // The port in the ACS URL is fake - the test will bind the mock webserver // to a random port and then whenever it needs to connect to a URL on the // mock webserver it will replace 54321 with the real port - setting 'xpack.security.authc.realms.shibboleth.sp.acs', 'http://localhost:54321/saml/acs' + setting 'xpack.security.authc.realms.shibboleth.sp.acs', 'http://localhost:54321/saml/acs1' setting 'xpack.security.authc.realms.shibboleth.attributes.principal', 'uid' setting 'xpack.security.authc.realms.shibboleth.attributes.name', 'urn:oid:2.5.4.3' + // SAML realm 2 (uses authorization_realms) + setting 'xpack.security.authc.realms.shibboleth_native.type', 'saml' + setting 'xpack.security.authc.realms.shibboleth_native.order', '2' + setting 'xpack.security.authc.realms.shibboleth_native.idp.entity_id', 'https://test.shibboleth.elastic.local/' + setting 'xpack.security.authc.realms.shibboleth_native.idp.metadata.path', 'idp-metadata.xml' + setting 'xpack.security.authc.realms.shibboleth_native.sp.entity_id', 'http://mock2.http.elastic.local/' + setting 'xpack.security.authc.realms.shibboleth_native.sp.acs', 'http://localhost:54321/saml/acs2' + setting 'xpack.security.authc.realms.shibboleth_native.attributes.principal', 'uid' + setting 'xpack.security.authc.realms.shibboleth_native.authorization_realms', 'native' + setting 'xpack.security.authc.realms.native.type', 'native' + setting 'xpack.security.authc.realms.native.order', '3' + setting 'xpack.ml.enabled', 'false' extraConfigFile 'idp-metadata.xml', idpFixtureProject.file("src/main/resources/provision/generated/idp-metadata.xml") diff --git a/x-pack/qa/saml-idp-tests/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlAuthenticationIT.java b/x-pack/qa/saml-idp-tests/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlAuthenticationIT.java index 031ee20ba0c..b3fc7dd0c2f 100644 --- a/x-pack/qa/saml-idp-tests/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlAuthenticationIT.java +++ b/x-pack/qa/saml-idp-tests/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlAuthenticationIT.java @@ -42,6 +42,8 @@ import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.Response; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.settings.SecureString; @@ -49,6 +51,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.mocksocket.MockHttpServer; import org.elasticsearch.test.rest.ESRestTestCase; @@ -65,7 +68,6 @@ import javax.net.ssl.KeyManager; import javax.net.ssl.SSLContext; import javax.net.ssl.TrustManager; import javax.net.ssl.X509ExtendedTrustManager; - import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -102,13 +104,16 @@ import static org.hamcrest.Matchers.startsWith; public class SamlAuthenticationIT extends ESRestTestCase { private static final String SP_LOGIN_PATH = "/saml/login"; - private static final String SP_ACS_PATH = "/saml/acs"; + private static final String SP_ACS_PATH_1 = "/saml/acs1"; + private static final String SP_ACS_PATH_2 = "/saml/acs2"; private static final String SAML_RESPONSE_FIELD = "SAMLResponse"; private static final String REQUEST_ID_COOKIE = "saml-request-id"; private static final String KIBANA_PASSWORD = "K1b@na K1b@na K1b@na"; private static HttpServer httpServer; + private URI acs; + @BeforeClass public static void setupHttpServer() throws IOException { InetSocketAddress address = new InetSocketAddress(InetAddress.getLoopbackAddress().getHostAddress(), 0); @@ -133,7 +138,8 @@ public class SamlAuthenticationIT extends ESRestTestCase { @Before public void setupHttpContext() { httpServer.createContext(SP_LOGIN_PATH, wrapFailures(this::httpLogin)); - httpServer.createContext(SP_ACS_PATH, wrapFailures(this::httpAcs)); + httpServer.createContext(SP_ACS_PATH_1, wrapFailures(this::httpAcs)); + httpServer.createContext(SP_ACS_PATH_2, wrapFailures(this::httpAcs)); } /** @@ -157,7 +163,8 @@ public class SamlAuthenticationIT extends ESRestTestCase { @After public void clearHttpContext() { httpServer.removeContext(SP_LOGIN_PATH); - httpServer.removeContext(SP_ACS_PATH); + httpServer.removeContext(SP_ACS_PATH_1); + httpServer.removeContext(SP_ACS_PATH_2); } @Override @@ -202,6 +209,21 @@ public class SamlAuthenticationIT extends ESRestTestCase { adminClient().performRequest(request); } + /** + * Create a native user for "thor" that is used for user-lookup (authorizing realms) + */ + @Before + public void setupNativeUser() throws IOException { + final Map body = MapBuilder.newMapBuilder() + .put("roles", Collections.singletonList("kibana_dashboard_only_user")) + .put("full_name", "Thor Son of Odin") + .put("password", randomAlphaOfLengthBetween(8, 16)) + .put("metadata", Collections.singletonMap("is_native", true)) + .map(); + final Response response = adminClient().performRequest(buildRequest("PUT", "/_xpack/security/user/thor", body)); + assertOK(response); + } + /** * Tests that a user can login via a SAML idp: * It uses: @@ -218,7 +240,24 @@ public class SamlAuthenticationIT extends ESRestTestCase { *

  • Uses that token to verify the user details
  • * */ - public void testLoginUser() throws Exception { + public void testLoginUserWithSamlRoleMapping() throws Exception { + // this ACS comes from the config in build.gradle + final Tuple authTokens = loginViaSaml("http://localhost:54321" + SP_ACS_PATH_1); + verifyElasticsearchAccessTokenForRoleMapping(authTokens.v1()); + final String accessToken = verifyElasticsearchRefreshToken(authTokens.v2()); + verifyElasticsearchAccessTokenForRoleMapping(accessToken); + } + + public void testLoginUserWithAuthorizingRealm() throws Exception { + // this ACS comes from the config in build.gradle + final Tuple authTokens = loginViaSaml("http://localhost:54321" + SP_ACS_PATH_2); + verifyElasticsearchAccessTokenForAuthorizingRealms(authTokens.v1()); + final String accessToken = verifyElasticsearchRefreshToken(authTokens.v2()); + verifyElasticsearchAccessTokenForAuthorizingRealms(accessToken); + } + + private Tuple loginViaSaml(String acs) throws Exception { + this.acs = new URI(acs); final BasicHttpContext context = new BasicHttpContext(); try (CloseableHttpClient client = getHttpClient()) { final URI loginUri = goToLoginPage(client, context); @@ -234,25 +273,21 @@ public class SamlAuthenticationIT extends ESRestTestCase { final Object accessToken = result.get("access_token"); assertThat(accessToken, notNullValue()); assertThat(accessToken, instanceOf(String.class)); - verifyElasticsearchAccessToken((String) accessToken); final Object refreshToken = result.get("refresh_token"); assertThat(refreshToken, notNullValue()); assertThat(refreshToken, instanceOf(String.class)); - verifyElasticsearchRefreshToken((String) refreshToken); + + return new Tuple<>((String) accessToken, (String) refreshToken); } } /** * Verifies that the provided "Access Token" (see {@link org.elasticsearch.xpack.security.authc.TokenService}) - * is for the expected user with the expected name and roles. + * is for the expected user with the expected name and roles if the user was created from Role-Mapping */ - private void verifyElasticsearchAccessToken(String accessToken) throws IOException { - Request request = new Request("GET", "/_xpack/security/_authenticate"); - RequestOptions.Builder options = request.getOptions().toBuilder(); - options.addHeader("Authorization", "Bearer " + accessToken); - request.setOptions(options); - final Map map = entityAsMap(client().performRequest(request)); + private void verifyElasticsearchAccessTokenForRoleMapping(String accessToken) throws IOException { + final Map map = callAuthenticateApiUsingAccessToken(accessToken); assertThat(map.get("username"), equalTo("thor")); assertThat(map.get("full_name"), equalTo("Thor Odinson")); assertSingletonList(map.get("roles"), "kibana_user"); @@ -266,15 +301,37 @@ public class SamlAuthenticationIT extends ESRestTestCase { } /** - * Verifies that the provided "Refresh Token" (see {@link org.elasticsearch.xpack.security.authc.TokenService}) - * can be used to get a new valid access token and refresh token. + * Verifies that the provided "Access Token" (see {@link org.elasticsearch.xpack.security.authc.TokenService}) + * is for the expected user with the expected name and roles if the user was retrieved from the native realm */ - private void verifyElasticsearchRefreshToken(String refreshToken) throws IOException { - Request request = new Request("POST", "/_xpack/security/oauth2/token"); - request.setJsonEntity("{ \"grant_type\":\"refresh_token\", \"refresh_token\":\"" + refreshToken + "\" }"); - kibanaAuth(request); + private void verifyElasticsearchAccessTokenForAuthorizingRealms(String accessToken) throws IOException { + final Map map = callAuthenticateApiUsingAccessToken(accessToken); + assertThat(map.get("username"), equalTo("thor")); + assertThat(map.get("full_name"), equalTo("Thor Son of Odin")); + assertSingletonList(map.get("roles"), "kibana_dashboard_only_user"); - final Map result = entityAsMap(client().performRequest(request)); + assertThat(map.get("metadata"), instanceOf(Map.class)); + final Map metadata = (Map) map.get("metadata"); + assertThat(metadata.get("is_native"), equalTo(true)); + } + + private Map callAuthenticateApiUsingAccessToken(String accessToken) throws IOException { + Request request = new Request("GET", "/_xpack/security/_authenticate"); + RequestOptions.Builder options = request.getOptions().toBuilder(); + options.addHeader("Authorization", "Bearer " + accessToken); + request.setOptions(options); + return entityAsMap(client().performRequest(request)); + } + + private String verifyElasticsearchRefreshToken(String refreshToken) throws IOException { + final Map body = MapBuilder.newMapBuilder() + .put("grant_type", "refresh_token") + .put("refresh_token", refreshToken) + .map(); + final Response response = client().performRequest(buildRequest("POST", "/_xpack/security/oauth2/token", body, kibanaAuth())); + assertOK(response); + + final Map result = entityAsMap(response); final Object newRefreshToken = result.get("refresh_token"); assertThat(newRefreshToken, notNullValue()); assertThat(newRefreshToken, instanceOf(String.class)); @@ -282,7 +339,7 @@ public class SamlAuthenticationIT extends ESRestTestCase { final Object accessToken = result.get("access_token"); assertThat(accessToken, notNullValue()); assertThat(accessToken, instanceOf(String.class)); - verifyElasticsearchAccessToken((String) accessToken); + return (String) accessToken; } /** @@ -348,7 +405,7 @@ public class SamlAuthenticationIT extends ESRestTestCase { form.setEntity(new UrlEncodedFormEntity(params)); return execute(client, form, context, - response -> parseSamlSubmissionForm(response.getEntity().getContent())); + response -> parseSamlSubmissionForm(response.getEntity().getContent())); } /** @@ -358,14 +415,14 @@ public class SamlAuthenticationIT extends ESRestTestCase { * @param saml The (deflated + base64 encoded) {@code SAMLResponse} parameter to post the ACS */ private Map submitSamlResponse(BasicHttpContext context, CloseableHttpClient client, URI acs, String saml) - throws IOException { + throws IOException { assertThat("SAML submission target", acs, notNullValue()); - assertThat(acs.getPath(), equalTo(SP_ACS_PATH)); + assertThat(acs, equalTo(this.acs)); assertThat("SAML submission content", saml, notNullValue()); // The ACS url provided from the SP is going to be wrong because the gradle // build doesn't know what the web server's port is, so it uses a fake one. - final HttpPost form = new HttpPost(getUrl(SP_ACS_PATH)); + final HttpPost form = new HttpPost(getUrl(this.acs.getPath())); List params = new ArrayList<>(); params.add(new BasicNameValuePair(SAML_RESPONSE_FIELD, saml)); form.setEntity(new UrlEncodedFormEntity(params)); @@ -460,13 +517,14 @@ public class SamlAuthenticationIT extends ESRestTestCase { * sends a redirect to that page. */ private void httpLogin(HttpExchange http) throws IOException { - Request request = new Request("POST", "/_xpack/security/saml/prepare"); - request.setJsonEntity("{}"); - kibanaAuth(request); - final Map body = entityAsMap(client().performRequest(request)); - logger.info("Created SAML authentication request {}", body); - http.getResponseHeaders().add("Set-Cookie", REQUEST_ID_COOKIE + "=" + body.get("id")); - http.getResponseHeaders().add("Location", (String) body.get("redirect")); + final Map body = Collections.singletonMap("acs", this.acs.toString()); + Request request = buildRequest("POST", "/_xpack/security/saml/prepare", body, kibanaAuth()); + final Response prepare = client().performRequest(request); + assertOK(prepare); + final Map responseBody = parseResponseAsMap(prepare.getEntity()); + logger.info("Created SAML authentication request {}", responseBody); + http.getResponseHeaders().add("Set-Cookie", REQUEST_ID_COOKIE + "=" + responseBody.get("id")); + http.getResponseHeaders().add("Location", (String) responseBody.get("redirect")); http.sendResponseHeaders(302, 0); http.close(); } @@ -501,10 +559,11 @@ public class SamlAuthenticationIT extends ESRestTestCase { final String id = getCookie(REQUEST_ID_COOKIE, http); assertThat(id, notNullValue()); - Request request = new Request("POST", "/_xpack/security/saml/authenticate"); - request.setJsonEntity("{ \"content\" : \"" + saml + "\", \"ids\": [\"" + id + "\"] }"); - kibanaAuth(request); - return client().performRequest(request); + final Map body = MapBuilder.newMapBuilder() + .put("content", saml) + .put("ids", Collections.singletonList(id)) + .map(); + return client().performRequest(buildRequest("POST", "/_xpack/security/saml/authenticate", body, kibanaAuth())); } private List parseRequestForm(HttpExchange http) throws IOException { @@ -518,6 +577,7 @@ public class SamlAuthenticationIT extends ESRestTestCase { try { final String cookies = http.getRequestHeaders().getFirst("Cookie"); if (cookies == null) { + logger.warn("No cookies in: {}", http.getResponseHeaders()); return null; } Header header = new BasicHeader("Cookie", cookies); @@ -540,11 +600,23 @@ public class SamlAuthenticationIT extends ESRestTestCase { assertThat(((List) value), contains(expectedElement)); } - private static void kibanaAuth(Request request) { - RequestOptions.Builder options = request.getOptions().toBuilder(); - options.addHeader("Authorization", - UsernamePasswordToken.basicAuthHeaderValue("kibana", new SecureString(KIBANA_PASSWORD.toCharArray()))); + private Request buildRequest(String method, String endpoint, Map body, Header... headers) throws IOException { + Request request = new Request(method, endpoint); + XContentBuilder builder = XContentFactory.jsonBuilder().map(body); + if (body != null) { + request.setJsonEntity(BytesReference.bytes(builder).utf8ToString()); + } + final RequestOptions.Builder options = request.getOptions().toBuilder(); + for (Header header : headers) { + options.addHeader(header.getName(), header.getValue()); + } request.setOptions(options); + return request; + } + + private static BasicHeader kibanaAuth() { + final String auth = UsernamePasswordToken.basicAuthHeaderValue("kibana", new SecureString(KIBANA_PASSWORD.toCharArray())); + return new BasicHeader(UsernamePasswordToken.BASIC_AUTH_HEADER, auth); } private CloseableHttpClient getHttpClient() throws Exception { From 547de71d5991ecee65fcc2ae05cadf95fad717f9 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 30 Aug 2018 23:44:57 -0400 Subject: [PATCH 06/16] Revert "Integrates soft-deletes into Elasticsearch (#33222)" Revert to correct co-author tags. This reverts commit 6dd0aa54f6d0ad44219b3b416438b1be57cb37e5. --- .../percolator/CandidateQueryTests.java | 8 +- .../PercolatorFieldMapperTests.java | 30 +- .../elasticsearch/common/lucene/Lucene.java | 86 +-- .../uid/PerThreadIDVersionAndSeqNoLookup.java | 21 +- .../common/settings/IndexScopedSettings.java | 2 - .../elasticsearch/index/IndexSettings.java | 38 -- .../index/engine/CombinedDeletionPolicy.java | 12 +- .../elasticsearch/index/engine/Engine.java | 28 +- .../index/engine/EngineConfig.java | 27 +- .../index/engine/InternalEngine.java | 388 ++--------- .../index/engine/LuceneChangesSnapshot.java | 368 ----------- .../RecoverySourcePruneMergePolicy.java | 292 --------- .../index/engine/SoftDeletesPolicy.java | 120 ---- .../index/fieldvisitor/FieldsVisitor.java | 10 +- .../index/mapper/DocumentMapper.java | 34 +- .../index/mapper/DocumentParser.java | 33 +- .../index/mapper/FieldNamesFieldMapper.java | 5 +- .../index/mapper/ParseContext.java | 20 +- .../index/mapper/ParsedDocument.java | 11 - .../index/mapper/SeqNoFieldMapper.java | 7 +- .../index/mapper/SourceFieldMapper.java | 16 +- .../elasticsearch/index/shard/IndexShard.java | 47 +- .../index/shard/PrimaryReplicaSyncer.java | 2 +- .../index/shard/StoreRecovery.java | 1 - .../org/elasticsearch/index/store/Store.java | 2 +- .../index/translog/Translog.java | 3 - .../index/translog/TranslogWriter.java | 20 +- .../translog/TruncateTranslogCommand.java | 2 - .../recovery/RecoverySourceHandler.java | 59 +- .../blobstore/BlobStoreRepository.java | 1 - .../snapshots/RestoreService.java | 4 +- .../cluster/routing/PrimaryAllocationIT.java | 1 - .../common/lucene/LuceneTests.java | 91 --- .../discovery/AbstractDisruptionTestCase.java | 1 - .../gateway/RecoveryFromGatewayIT.java | 13 +- .../index/IndexServiceTests.java | 3 +- .../index/IndexSettingsTests.java | 8 - .../engine/CombinedDeletionPolicyTests.java | 69 +- .../index/engine/InternalEngineTests.java | 620 ++++++------------ .../engine/LuceneChangesSnapshotTests.java | 289 -------- .../RecoverySourcePruneMergePolicyTests.java | 161 ----- .../index/engine/SoftDeletesPolicyTests.java | 75 --- .../index/mapper/DocumentParserTests.java | 10 +- .../index/mapper/DynamicMappingTests.java | 6 +- .../IndexLevelReplicationTests.java | 29 +- .../RecoveryDuringReplicationTests.java | 11 +- .../index/shard/IndexShardTests.java | 58 +- .../shard/PrimaryReplicaSyncerTests.java | 21 +- .../index/shard/RefreshListenersTests.java | 4 +- .../indices/recovery/IndexRecoveryIT.java | 6 - .../PeerRecoveryTargetServiceTests.java | 2 - .../recovery/RecoverySourceHandlerTests.java | 6 + .../indices/recovery/RecoveryTests.java | 80 +-- .../indices/stats/IndexStatsIT.java | 37 +- .../AbstractSnapshotIntegTestCase.java | 6 - .../SharedClusterSnapshotRestoreIT.java | 13 +- .../versioning/SimpleVersioningIT.java | 23 - .../index/engine/EngineTestCase.java | 400 +---------- .../ESIndexLevelReplicationTestCase.java | 27 +- .../index/shard/IndexShardTestCase.java | 131 ++-- .../elasticsearch/test/ESIntegTestCase.java | 4 - .../test/ESSingleNodeTestCase.java | 9 - .../test/InternalTestCluster.java | 20 - 63 files changed, 499 insertions(+), 3432 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java delete mode 100644 server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java delete mode 100644 server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java delete mode 100644 server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java delete mode 100644 server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java delete mode 100644 server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java index 9c8979601e8..e6d637aabb1 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java @@ -77,7 +77,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; @@ -88,7 +87,6 @@ import org.elasticsearch.common.geo.ShapeRelation; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; @@ -1111,11 +1109,7 @@ public class CandidateQueryTests extends ESSingleNodeTestCase { } private void addQuery(Query query, List docs) { - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(query, parseContext); ParseContext.Document queryDocument = parseContext.doc(); diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java index 80524a2f862..ecff48b344c 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java @@ -42,7 +42,6 @@ import org.apache.lucene.search.join.ScoreMode; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.support.PlainActionFuture; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -59,7 +58,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperParser; import org.elasticsearch.index.mapper.MapperParsingException; @@ -184,11 +182,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); ParseContext.Document document = parseContext.doc(); @@ -210,7 +204,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { bq.add(termQuery1, Occur.MUST); bq.add(termQuery2, Occur.MUST); - parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), + parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); document = parseContext.doc(); @@ -238,12 +232,8 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { bq.add(rangeQuery2, Occur.MUST); DocumentMapper documentMapper = mapperService.documentMapper("doc"); - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); ParseContext.Document document = parseContext.doc(); @@ -269,7 +259,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { .rangeQuery(15, 20, true, true, null, null, null, null); bq.add(rangeQuery2, Occur.MUST); - parseContext = new ParseContext.InternalParseContext(settings, + parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); document = parseContext.doc(); @@ -293,11 +283,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { TermRangeQuery query = new TermRangeQuery("field1", new BytesRef("a"), new BytesRef("z"), true, true); DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(query, parseContext); ParseContext.Document document = parseContext.doc(); @@ -312,11 +298,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { PhraseQuery phraseQuery = new PhraseQuery("field", "term"); DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(phraseQuery, parseContext); ParseContext.Document document = parseContext.doc(); diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index 1c1e5687893..a24a6aea07f 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -27,10 +27,8 @@ import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.document.LatLonDocValuesField; -import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.FilterDirectoryReader; import org.apache.lucene.index.FilterLeafReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexFileNames; @@ -98,8 +96,6 @@ public class Lucene { assert annotation == null : "DocValuesFormat " + LATEST_DOC_VALUES_FORMAT + " is deprecated" ; } - public static final String SOFT_DELETES_FIELD = "__soft_deletes"; - public static final NamedAnalyzer STANDARD_ANALYZER = new NamedAnalyzer("_standard", AnalyzerScope.GLOBAL, new StandardAnalyzer()); public static final NamedAnalyzer KEYWORD_ANALYZER = new NamedAnalyzer("_keyword", AnalyzerScope.GLOBAL, new KeywordAnalyzer()); @@ -144,7 +140,7 @@ public class Lucene { public static int getNumDocs(SegmentInfos info) { int numDocs = 0; for (SegmentCommitInfo si : info) { - numDocs += si.info.maxDoc() - si.getDelCount() - si.getSoftDelCount(); + numDocs += si.info.maxDoc() - si.getDelCount(); } return numDocs; } @@ -201,7 +197,6 @@ public class Lucene { } final CommitPoint cp = new CommitPoint(si, directory); try (IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(Lucene.STANDARD_ANALYZER) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setIndexCommit(cp) .setCommitOnClose(false) .setMergePolicy(NoMergePolicy.INSTANCE) @@ -225,7 +220,6 @@ public class Lucene { } } try (IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(Lucene.STANDARD_ANALYZER) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setMergePolicy(NoMergePolicy.INSTANCE) // no merges .setCommitOnClose(false) // no commits .setOpenMode(IndexWriterConfig.OpenMode.CREATE))) // force creation - don't append... @@ -835,82 +829,4 @@ public class Lucene { } }; } - - /** - * Wraps a directory reader to make all documents live except those were rolled back - * or hard-deleted due to non-aborting exceptions during indexing. - * The wrapped reader can be used to query all documents. - * - * @param in the input directory reader - * @return the wrapped reader - */ - public static DirectoryReader wrapAllDocsLive(DirectoryReader in) throws IOException { - return new DirectoryReaderWithAllLiveDocs(in); - } - - private static final class DirectoryReaderWithAllLiveDocs extends FilterDirectoryReader { - static final class LeafReaderWithLiveDocs extends FilterLeafReader { - final Bits liveDocs; - final int numDocs; - LeafReaderWithLiveDocs(LeafReader in, Bits liveDocs, int numDocs) { - super(in); - this.liveDocs = liveDocs; - this.numDocs = numDocs; - } - @Override - public Bits getLiveDocs() { - return liveDocs; - } - @Override - public int numDocs() { - return numDocs; - } - @Override - public CacheHelper getCoreCacheHelper() { - return in.getCoreCacheHelper(); - } - @Override - public CacheHelper getReaderCacheHelper() { - return null; // Modifying liveDocs - } - } - - DirectoryReaderWithAllLiveDocs(DirectoryReader in) throws IOException { - super(in, new SubReaderWrapper() { - @Override - public LeafReader wrap(LeafReader leaf) { - SegmentReader segmentReader = segmentReader(leaf); - Bits hardLiveDocs = segmentReader.getHardLiveDocs(); - if (hardLiveDocs == null) { - return new LeafReaderWithLiveDocs(leaf, null, leaf.maxDoc()); - } - // TODO: Can we avoid calculate numDocs by using SegmentReader#getSegmentInfo with LUCENE-8458? - int numDocs = 0; - for (int i = 0; i < hardLiveDocs.length(); i++) { - if (hardLiveDocs.get(i)) { - numDocs++; - } - } - return new LeafReaderWithLiveDocs(segmentReader, hardLiveDocs, numDocs); - } - }); - } - - @Override - protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { - return wrapAllDocsLive(in); - } - - @Override - public CacheHelper getReaderCacheHelper() { - return null; // Modifying liveDocs - } - } - - /** - * Returns a numeric docvalues which can be used to soft-delete documents. - */ - public static NumericDocValuesField newSoftDeletesField() { - return new NumericDocValuesField(SOFT_DELETES_FIELD, 1); - } } diff --git a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java index 3a037bed62b..38fcdfe5f1b 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java @@ -28,7 +28,6 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -67,22 +66,15 @@ final class PerThreadIDVersionAndSeqNoLookup { */ PerThreadIDVersionAndSeqNoLookup(LeafReader reader, String uidField) throws IOException { this.uidField = uidField; - final Terms terms = reader.terms(uidField); + Terms terms = reader.terms(uidField); if (terms == null) { - // If a segment contains only no-ops, it does not have _uid but has both _soft_deletes and _tombstone fields. - final NumericDocValues softDeletesDV = reader.getNumericDocValues(Lucene.SOFT_DELETES_FIELD); - final NumericDocValues tombstoneDV = reader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); - if (softDeletesDV == null || tombstoneDV == null) { - throw new IllegalArgumentException("reader does not have _uid terms but not a no-op segment; " + - "_soft_deletes [" + softDeletesDV + "], _tombstone [" + tombstoneDV + "]"); - } - termsEnum = null; - } else { - termsEnum = terms.iterator(); + throw new IllegalArgumentException("reader misses the [" + uidField + "] field"); } + termsEnum = terms.iterator(); if (reader.getNumericDocValues(VersionFieldMapper.NAME) == null) { - throw new IllegalArgumentException("reader misses the [" + VersionFieldMapper.NAME + "] field; _uid terms [" + terms + "]"); + throw new IllegalArgumentException("reader misses the [" + VersionFieldMapper.NAME + "] field"); } + Object readerKey = null; assert (readerKey = reader.getCoreCacheHelper().getKey()) != null; this.readerKey = readerKey; @@ -119,8 +111,7 @@ final class PerThreadIDVersionAndSeqNoLookup { * {@link DocIdSetIterator#NO_MORE_DOCS} is returned if not found * */ private int getDocID(BytesRef id, Bits liveDocs) throws IOException { - // termsEnum can possibly be null here if this leaf contains only no-ops. - if (termsEnum != null && termsEnum.seekExact(id)) { + if (termsEnum.seekExact(id)) { int docID = DocIdSetIterator.NO_MORE_DOCS; // there may be more than one matching docID, in the case of nested docs, so we want the last one: docsEnum = termsEnum.postings(docsEnum, 0); 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 f3de294046c..46e3867f7ae 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -129,8 +129,6 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.MAX_REGEX_LENGTH_SETTING, ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING, IndexSettings.INDEX_GC_DELETES_SETTING, - IndexSettings.INDEX_SOFT_DELETES_SETTING, - IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING, UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING, diff --git a/server/src/main/java/org/elasticsearch/index/IndexSettings.java b/server/src/main/java/org/elasticsearch/index/IndexSettings.java index 3ea022bbebd..44cd743bbd4 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/server/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -237,21 +237,6 @@ public final class IndexSettings { public static final Setting INDEX_GC_DELETES_SETTING = Setting.timeSetting("index.gc_deletes", DEFAULT_GC_DELETES, new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic, Property.IndexScope); - - /** - * Specifies if the index should use soft-delete instead of hard-delete for update/delete operations. - */ - public static final Setting INDEX_SOFT_DELETES_SETTING = - Setting.boolSetting("index.soft_deletes.enabled", false, Property.IndexScope, Property.Final); - - /** - * Controls how many soft-deleted documents will be kept around before being merged away. Keeping more deleted - * documents increases the chance of operation-based recoveries and allows querying a longer history of documents. - * If soft-deletes is enabled, an engine by default will retain all operations up to the global checkpoint. - **/ - public static final Setting INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING = - Setting.longSetting("index.soft_deletes.retention.operations", 0, 0, Property.IndexScope, Property.Dynamic); - /** * The maximum number of refresh listeners allows on this shard. */ @@ -304,8 +289,6 @@ public final class IndexSettings { private final IndexSortConfig indexSortConfig; private final IndexScopedSettings scopedSettings; private long gcDeletesInMillis = DEFAULT_GC_DELETES.millis(); - private final boolean softDeleteEnabled; - private volatile long softDeleteRetentionOperations; private volatile boolean warmerEnabled; private volatile int maxResultWindow; private volatile int maxInnerResultWindow; @@ -417,8 +400,6 @@ public final class IndexSettings { generationThresholdSize = scopedSettings.get(INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING); mergeSchedulerConfig = new MergeSchedulerConfig(this); gcDeletesInMillis = scopedSettings.get(INDEX_GC_DELETES_SETTING).getMillis(); - softDeleteEnabled = version.onOrAfter(Version.V_7_0_0_alpha1) && scopedSettings.get(INDEX_SOFT_DELETES_SETTING); - softDeleteRetentionOperations = scopedSettings.get(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING); warmerEnabled = scopedSettings.get(INDEX_WARMER_ENABLED_SETTING); maxResultWindow = scopedSettings.get(MAX_RESULT_WINDOW_SETTING); maxInnerResultWindow = scopedSettings.get(MAX_INNER_RESULT_WINDOW_SETTING); @@ -477,7 +458,6 @@ public final class IndexSettings { scopedSettings.addSettingsUpdateConsumer(INDEX_SEARCH_IDLE_AFTER, this::setSearchIdleAfter); scopedSettings.addSettingsUpdateConsumer(MAX_REGEX_LENGTH_SETTING, this::setMaxRegexLength); scopedSettings.addSettingsUpdateConsumer(DEFAULT_PIPELINE, this::setDefaultPipeline); - scopedSettings.addSettingsUpdateConsumer(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, this::setSoftDeleteRetentionOperations); } private void setSearchIdleAfter(TimeValue searchIdleAfter) { this.searchIdleAfter = searchIdleAfter; } @@ -861,22 +841,4 @@ public final class IndexSettings { public void setDefaultPipeline(String defaultPipeline) { this.defaultPipeline = defaultPipeline; } - - /** - * Returns true if soft-delete is enabled. - */ - public boolean isSoftDeleteEnabled() { - return softDeleteEnabled; - } - - private void setSoftDeleteRetentionOperations(long ops) { - this.softDeleteRetentionOperations = ops; - } - - /** - * Returns the number of extra operations (i.e. soft-deleted documents) to be kept for recoveries and history purpose. - */ - public long getSoftDeleteRetentionOperations() { - return this.softDeleteRetentionOperations; - } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index d10690379ed..d0575c8a8c9 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -46,17 +46,14 @@ import java.util.function.LongSupplier; public final class CombinedDeletionPolicy extends IndexDeletionPolicy { private final Logger logger; private final TranslogDeletionPolicy translogDeletionPolicy; - private final SoftDeletesPolicy softDeletesPolicy; private final LongSupplier globalCheckpointSupplier; private final ObjectIntHashMap snapshottedCommits; // Number of snapshots held against each commit point. private volatile IndexCommit safeCommit; // the most recent safe commit point - its max_seqno at most the persisted global checkpoint. private volatile IndexCommit lastCommit; // the most recent commit point - CombinedDeletionPolicy(Logger logger, TranslogDeletionPolicy translogDeletionPolicy, - SoftDeletesPolicy softDeletesPolicy, LongSupplier globalCheckpointSupplier) { + CombinedDeletionPolicy(Logger logger, TranslogDeletionPolicy translogDeletionPolicy, LongSupplier globalCheckpointSupplier) { this.logger = logger; this.translogDeletionPolicy = translogDeletionPolicy; - this.softDeletesPolicy = softDeletesPolicy; this.globalCheckpointSupplier = globalCheckpointSupplier; this.snapshottedCommits = new ObjectIntHashMap<>(); } @@ -83,7 +80,7 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { deleteCommit(commits.get(i)); } } - updateRetentionPolicy(); + updateTranslogDeletionPolicy(); } private void deleteCommit(IndexCommit commit) throws IOException { @@ -93,7 +90,7 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { assert commit.isDeleted() : "Deletion commit [" + commitDescription(commit) + "] was suppressed"; } - private void updateRetentionPolicy() throws IOException { + private void updateTranslogDeletionPolicy() throws IOException { assert Thread.holdsLock(this); logger.debug("Safe commit [{}], last commit [{}]", commitDescription(safeCommit), commitDescription(lastCommit)); assert safeCommit.isDeleted() == false : "The safe commit must not be deleted"; @@ -104,9 +101,6 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { assert minRequiredGen <= lastGen : "minRequiredGen must not be greater than lastGen"; translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastGen); translogDeletionPolicy.setMinTranslogGenerationForRecovery(minRequiredGen); - - softDeletesPolicy.setLocalCheckpointOfSafeCommit( - Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY))); } /** 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 08724d6e794..4d95cf89ef0 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -58,7 +58,6 @@ import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; @@ -98,7 +97,6 @@ public abstract class Engine implements Closeable { public static final String SYNC_COMMIT_ID = "sync_id"; public static final String HISTORY_UUID_KEY = "history_uuid"; - public static final String MIN_RETAINED_SEQNO = "min_retained_seq_no"; protected final ShardId shardId; protected final String allocationId; @@ -587,32 +585,18 @@ public abstract class Engine implements Closeable { public abstract void syncTranslog() throws IOException; - /** - * Acquires a lock on the translog files and Lucene soft-deleted documents to prevent them from being trimmed - */ - public abstract Closeable acquireRetentionLockForPeerRecovery(); + public abstract Closeable acquireTranslogRetentionLock(); /** - * Creates a new history snapshot from Lucene for reading operations whose seqno in the requesting seqno range (both inclusive) + * Creates a new translog snapshot from this engine for reading translog operations whose seq# at least the provided seq#. + * The caller has to close the returned snapshot after finishing the reading. */ - public abstract Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, - long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException; + public abstract Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException; /** - * Creates a new history snapshot for reading operations since {@code startingSeqNo} (inclusive). - * The returned snapshot can be retrieved from either Lucene index or translog files. + * Returns the estimated number of translog operations in this engine whose seq# at least the provided seq#. */ - public abstract Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException; - - /** - * Returns the estimated number of history operations whose seq# at least {@code startingSeqNo}(inclusive) in this engine. - */ - public abstract int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException; - - /** - * Checks if this engine has every operations since {@code startingSeqNo}(inclusive) in its history (either Lucene or translog) - */ - public abstract boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException; + public abstract int estimateTranslogOperationsFromMinSeq(long minSeqNo); public abstract TranslogStats getTranslogStats(); diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index 23a90553f60..2deae61bd52 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.unit.MemorySizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.codec.CodecService; -import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; @@ -81,7 +80,6 @@ public final class EngineConfig { private final CircuitBreakerService circuitBreakerService; private final LongSupplier globalCheckpointSupplier; private final LongSupplier primaryTermSupplier; - private final TombstoneDocSupplier tombstoneDocSupplier; /** * Index setting to change the low level lucene codec used for writing new segments. @@ -128,8 +126,7 @@ public final class EngineConfig { List externalRefreshListener, List internalRefreshListener, Sort indexSort, TranslogRecoveryRunner translogRecoveryRunner, CircuitBreakerService circuitBreakerService, - LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier, - TombstoneDocSupplier tombstoneDocSupplier) { + LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier) { this.shardId = shardId; this.allocationId = allocationId; this.indexSettings = indexSettings; @@ -167,7 +164,6 @@ public final class EngineConfig { this.circuitBreakerService = circuitBreakerService; this.globalCheckpointSupplier = globalCheckpointSupplier; this.primaryTermSupplier = primaryTermSupplier; - this.tombstoneDocSupplier = tombstoneDocSupplier; } /** @@ -377,25 +373,4 @@ public final class EngineConfig { public LongSupplier getPrimaryTermSupplier() { return primaryTermSupplier; } - - /** - * A supplier supplies tombstone documents which will be used in soft-update methods. - * The returned document consists only _uid, _seqno, _term and _version fields; other metadata fields are excluded. - */ - public interface TombstoneDocSupplier { - /** - * Creates a tombstone document for a delete operation. - */ - ParsedDocument newDeleteTombstoneDoc(String type, String id); - - /** - * Creates a tombstone document for a noop operation. - * @param reason the reason of an a noop - */ - ParsedDocument newNoopTombstoneDoc(String reason); - } - - public TombstoneDocSupplier getTombstoneDocSupplier() { - return tombstoneDocSupplier; - } } 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 da4decc93b1..023e659ffab 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -21,20 +21,16 @@ package org.elasticsearch.index.engine; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.document.Field; -import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.ReferenceManager; @@ -46,7 +42,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.InfoStream; -import org.elasticsearch.Assertions; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.Nullable; @@ -66,11 +61,7 @@ import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; import org.elasticsearch.index.seqno.LocalCheckpointTracker; @@ -149,10 +140,6 @@ public class InternalEngine extends Engine { private final CounterMetric numDocDeletes = new CounterMetric(); private final CounterMetric numDocAppends = new CounterMetric(); private final CounterMetric numDocUpdates = new CounterMetric(); - private final NumericDocValuesField softDeletesField = Lucene.newSoftDeletesField(); - private final boolean softDeleteEnabled; - private final SoftDeletesPolicy softDeletesPolicy; - private final LastRefreshedCheckpointListener lastRefreshedCheckpointListener; /** * How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh. IndexingMemoryController polls this @@ -197,10 +184,8 @@ public class InternalEngine extends Engine { assert translog.getGeneration() != null; this.translog = translog; this.localCheckpointTracker = createLocalCheckpointTracker(localCheckpointTrackerSupplier); - this.softDeleteEnabled = engineConfig.getIndexSettings().isSoftDeleteEnabled(); - this.softDeletesPolicy = newSoftDeletesPolicy(); this.combinedDeletionPolicy = - new CombinedDeletionPolicy(logger, translogDeletionPolicy, softDeletesPolicy, translog::getLastSyncedGlobalCheckpoint); + new CombinedDeletionPolicy(logger, translogDeletionPolicy, translog::getLastSyncedGlobalCheckpoint); writer = createWriter(); bootstrapAppendOnlyInfoFromWriter(writer); historyUUID = loadHistoryUUID(writer); @@ -230,8 +215,6 @@ public class InternalEngine extends Engine { for (ReferenceManager.RefreshListener listener: engineConfig.getInternalRefreshListener()) { this.internalSearcherManager.addListener(listener); } - this.lastRefreshedCheckpointListener = new LastRefreshedCheckpointListener(localCheckpointTracker.getCheckpoint()); - this.internalSearcherManager.addListener(lastRefreshedCheckpointListener); success = true; } finally { if (success == false) { @@ -257,18 +240,6 @@ public class InternalEngine extends Engine { return localCheckpointTrackerSupplier.apply(maxSeqNo, localCheckpoint); } - private SoftDeletesPolicy newSoftDeletesPolicy() throws IOException { - final Map commitUserData = store.readLastCommittedSegmentsInfo().userData; - final long lastMinRetainedSeqNo; - if (commitUserData.containsKey(Engine.MIN_RETAINED_SEQNO)) { - lastMinRetainedSeqNo = Long.parseLong(commitUserData.get(Engine.MIN_RETAINED_SEQNO)); - } else { - lastMinRetainedSeqNo = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)) + 1; - } - return new SoftDeletesPolicy(translog::getLastSyncedGlobalCheckpoint, lastMinRetainedSeqNo, - engineConfig.getIndexSettings().getSoftDeleteRetentionOperations()); - } - /** * This reference manager delegates all it's refresh calls to another (internal) SearcherManager * The main purpose for this is that if we have external refreshes happening we don't issue extra @@ -480,31 +451,19 @@ public class InternalEngine extends Engine { revisitIndexDeletionPolicyOnTranslogSynced(); } - /** - * Creates a new history snapshot for reading operations since the provided seqno. - * The returned snapshot can be retrieved from either Lucene index or translog files. - */ @Override - public Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { - if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { - return newChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false); - } else { - return getTranslog().newSnapshotFromMinSeqNo(startingSeqNo); - } + public Closeable acquireTranslogRetentionLock() { + return getTranslog().acquireRetentionLock(); } - /** - * Returns the estimated number of history operations whose seq# at least the provided seq# in this engine. - */ @Override - public int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { - if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { - try (Translog.Snapshot snapshot = newChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false)) { - return snapshot.totalOperations(); - } - } else { - return getTranslog().estimateTotalOperationsFromMinSeq(startingSeqNo); - } + public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { + return getTranslog().newSnapshotFromMinSeqNo(minSeqNo); + } + + @Override + public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { + return getTranslog().estimateTotalOperationsFromMinSeq(minSeqNo); } @Override @@ -831,7 +790,7 @@ public class InternalEngine extends Engine { if (plan.earlyResultOnPreFlightError.isPresent()) { indexResult = plan.earlyResultOnPreFlightError.get(); assert indexResult.getResultType() == Result.Type.FAILURE : indexResult.getResultType(); - } else if (plan.indexIntoLucene || plan.addStaleOpToLucene) { + } else if (plan.indexIntoLucene) { indexResult = indexIntoLucene(index, plan); } else { indexResult = new IndexResult( @@ -842,10 +801,8 @@ public class InternalEngine extends Engine { if (indexResult.getResultType() == Result.Type.SUCCESS) { location = translog.add(new Translog.Index(index, indexResult)); } else if (indexResult.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - // if we have document failure, record it as a no-op in the translog and Lucene with the generated seq_no - final NoOp noOp = new NoOp(indexResult.getSeqNo(), index.primaryTerm(), index.origin(), - index.startTime(), indexResult.getFailure().toString()); - location = innerNoOp(noOp).getTranslogLocation(); + // if we have document failure, record it as a no-op in the translog with the generated seq_no + location = translog.add(new Translog.NoOp(indexResult.getSeqNo(), index.primaryTerm(), indexResult.getFailure().toString())); } else { location = null; } @@ -897,6 +854,7 @@ public class InternalEngine extends Engine { // unlike the primary, replicas don't really care to about creation status of documents // this allows to ignore the case where a document was found in the live version maps in // a delete state and return false for the created flag in favor of code simplicity + final OpVsLuceneDocStatus opVsLucene; if (index.seqNo() <= localCheckpointTracker.getCheckpoint()){ // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already @@ -905,15 +863,16 @@ public class InternalEngine extends Engine { // question may have been deleted in an out of order op that is not replayed. // See testRecoverFromStoreWithOutOfOrderDelete for an example of local recovery // See testRecoveryWithOutOfOrderDelete for an example of peer recovery + opVsLucene = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; + } else { + opVsLucene = compareOpToLuceneDocBasedOnSeqNo(index); + } + if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { plan = IndexingStrategy.processButSkipLucene(false, index.seqNo(), index.version()); } else { - final OpVsLuceneDocStatus opVsLucene = compareOpToLuceneDocBasedOnSeqNo(index); - if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { - plan = IndexingStrategy.processAsStaleOp(softDeleteEnabled, index.seqNo(), index.version()); - } else { - plan = IndexingStrategy.processNormally(opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, - index.seqNo(), index.version()); - } + plan = IndexingStrategy.processNormally( + opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, index.seqNo(), index.version() + ); } } return plan; @@ -962,7 +921,7 @@ public class InternalEngine extends Engine { throws IOException { assert plan.seqNoForIndexing >= 0 : "ops should have an assigned seq no.; origin: " + index.origin(); assert plan.versionForIndexing >= 0 : "version must be set. got " + plan.versionForIndexing; - assert plan.indexIntoLucene || plan.addStaleOpToLucene; + assert plan.indexIntoLucene; /* Update the document's sequence number and primary term; the sequence number here is derived here from either the sequence * number service if this is on the primary, or the existing document's sequence number if this is on the replica. The * primary term here has already been set, see IndexShard#prepareIndex where the Engine$Index operation is created. @@ -970,9 +929,7 @@ public class InternalEngine extends Engine { index.parsedDoc().updateSeqID(plan.seqNoForIndexing, index.primaryTerm()); index.parsedDoc().version().setLongValue(plan.versionForIndexing); try { - if (plan.addStaleOpToLucene) { - addStaleDocs(index.docs(), indexWriter); - } else if (plan.useLuceneUpdateDocument) { + if (plan.useLuceneUpdateDocument) { updateDocs(index.uid(), index.docs(), indexWriter); } else { // document does not exists, we can optimize for create, but double check if assertions are running @@ -1036,29 +993,16 @@ public class InternalEngine extends Engine { numDocAppends.inc(docs.size()); } - private void addStaleDocs(final List docs, final IndexWriter indexWriter) throws IOException { - assert softDeleteEnabled : "Add history documents but soft-deletes is disabled"; - for (ParseContext.Document doc : docs) { - doc.add(softDeletesField); // soft-deleted every document before adding to Lucene - } - if (docs.size() > 1) { - indexWriter.addDocuments(docs); - } else { - indexWriter.addDocument(docs.get(0)); - } - } - - protected static final class IndexingStrategy { + private static final class IndexingStrategy { final boolean currentNotFoundOrDeleted; final boolean useLuceneUpdateDocument; final long seqNoForIndexing; final long versionForIndexing; final boolean indexIntoLucene; - final boolean addStaleOpToLucene; final Optional earlyResultOnPreFlightError; private IndexingStrategy(boolean currentNotFoundOrDeleted, boolean useLuceneUpdateDocument, - boolean indexIntoLucene, boolean addStaleOpToLucene, long seqNoForIndexing, + boolean indexIntoLucene, long seqNoForIndexing, long versionForIndexing, IndexResult earlyResultOnPreFlightError) { assert useLuceneUpdateDocument == false || indexIntoLucene : "use lucene update is set to true, but we're not indexing into lucene"; @@ -1071,40 +1015,37 @@ public class InternalEngine extends Engine { this.seqNoForIndexing = seqNoForIndexing; this.versionForIndexing = versionForIndexing; this.indexIntoLucene = indexIntoLucene; - this.addStaleOpToLucene = addStaleOpToLucene; this.earlyResultOnPreFlightError = earlyResultOnPreFlightError == null ? Optional.empty() : Optional.of(earlyResultOnPreFlightError); } static IndexingStrategy optimizedAppendOnly(long seqNoForIndexing) { - return new IndexingStrategy(true, false, true, false, seqNoForIndexing, 1, null); + return new IndexingStrategy(true, false, true, seqNoForIndexing, 1, null); } static IndexingStrategy skipDueToVersionConflict( VersionConflictEngineException e, boolean currentNotFoundOrDeleted, long currentVersion, long term) { final IndexResult result = new IndexResult(e, currentVersion, term); return new IndexingStrategy( - currentNotFoundOrDeleted, false, false, false, SequenceNumbers.UNASSIGNED_SEQ_NO, Versions.NOT_FOUND, result); + currentNotFoundOrDeleted, false, false, SequenceNumbers.UNASSIGNED_SEQ_NO, Versions.NOT_FOUND, result); } static IndexingStrategy processNormally(boolean currentNotFoundOrDeleted, long seqNoForIndexing, long versionForIndexing) { return new IndexingStrategy(currentNotFoundOrDeleted, currentNotFoundOrDeleted == false, - true, false, seqNoForIndexing, versionForIndexing, null); + true, seqNoForIndexing, versionForIndexing, null); } static IndexingStrategy overrideExistingAsIfNotThere( long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(true, true, true, false, seqNoForIndexing, versionForIndexing, null); + return new IndexingStrategy(true, true, true, seqNoForIndexing, versionForIndexing, null); } - static IndexingStrategy processButSkipLucene(boolean currentNotFoundOrDeleted, long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(currentNotFoundOrDeleted, false, false, false, seqNoForIndexing, versionForIndexing, null); - } - - static IndexingStrategy processAsStaleOp(boolean addStaleOpToLucene, long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(false, false, false, addStaleOpToLucene, seqNoForIndexing, versionForIndexing, null); + static IndexingStrategy processButSkipLucene(boolean currentNotFoundOrDeleted, + long seqNoForIndexing, long versionForIndexing) { + return new IndexingStrategy(currentNotFoundOrDeleted, false, + false, seqNoForIndexing, versionForIndexing, null); } } @@ -1131,18 +1072,10 @@ public class InternalEngine extends Engine { } private void updateDocs(final Term uid, final List docs, final IndexWriter indexWriter) throws IOException { - if (softDeleteEnabled) { - if (docs.size() > 1) { - indexWriter.softUpdateDocuments(uid, docs, softDeletesField); - } else { - indexWriter.softUpdateDocument(uid, docs.get(0), softDeletesField); - } + if (docs.size() > 1) { + indexWriter.updateDocuments(uid, docs); } else { - if (docs.size() > 1) { - indexWriter.updateDocuments(uid, docs); - } else { - indexWriter.updateDocument(uid, docs.get(0)); - } + indexWriter.updateDocument(uid, docs.get(0)); } numDocUpdates.inc(docs.size()); } @@ -1166,7 +1099,7 @@ public class InternalEngine extends Engine { if (plan.earlyResultOnPreflightError.isPresent()) { deleteResult = plan.earlyResultOnPreflightError.get(); - } else if (plan.deleteFromLucene || plan.addStaleOpToLucene) { + } else if (plan.deleteFromLucene) { deleteResult = deleteInLucene(delete, plan); } else { deleteResult = new DeleteResult( @@ -1177,10 +1110,8 @@ public class InternalEngine extends Engine { if (deleteResult.getResultType() == Result.Type.SUCCESS) { location = translog.add(new Translog.Delete(delete, deleteResult)); } else if (deleteResult.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - // if we have document failure, record it as a no-op in the translog and Lucene with the generated seq_no - final NoOp noOp = new NoOp(deleteResult.getSeqNo(), delete.primaryTerm(), delete.origin(), - delete.startTime(), deleteResult.getFailure().toString()); - location = innerNoOp(noOp).getTranslogLocation(); + location = translog.add(new Translog.NoOp(deleteResult.getSeqNo(), + delete.primaryTerm(), deleteResult.getFailure().toString())); } else { location = null; } @@ -1211,7 +1142,7 @@ public class InternalEngine extends Engine { // unlike the primary, replicas don't really care to about found status of documents // this allows to ignore the case where a document was found in the live version maps in // a delete state and return true for the found flag in favor of code simplicity - final DeletionStrategy plan; + final OpVsLuceneDocStatus opVsLucene; if (delete.seqNo() <= localCheckpointTracker.getCheckpoint()) { // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already @@ -1220,15 +1151,18 @@ public class InternalEngine extends Engine { // question may have been deleted in an out of order op that is not replayed. // See testRecoverFromStoreWithOutOfOrderDelete for an example of local recovery // See testRecoveryWithOutOfOrderDelete for an example of peer recovery + opVsLucene = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; + } else { + opVsLucene = compareOpToLuceneDocBasedOnSeqNo(delete); + } + + final DeletionStrategy plan; + if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { plan = DeletionStrategy.processButSkipLucene(false, delete.seqNo(), delete.version()); } else { - final OpVsLuceneDocStatus opVsLucene = compareOpToLuceneDocBasedOnSeqNo(delete); - if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { - plan = DeletionStrategy.processAsStaleOp(softDeleteEnabled, false, delete.seqNo(), delete.version()); - } else { - plan = DeletionStrategy.processNormally(opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, - delete.seqNo(), delete.version()); - } + plan = DeletionStrategy.processNormally( + opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, + delete.seqNo(), delete.version()); } return plan; } @@ -1263,31 +1197,15 @@ public class InternalEngine extends Engine { private DeleteResult deleteInLucene(Delete delete, DeletionStrategy plan) throws IOException { try { - if (softDeleteEnabled) { - final ParsedDocument tombstone = engineConfig.getTombstoneDocSupplier().newDeleteTombstoneDoc(delete.type(), delete.id()); - assert tombstone.docs().size() == 1 : "Tombstone doc should have single doc [" + tombstone + "]"; - tombstone.updateSeqID(plan.seqNoOfDeletion, delete.primaryTerm()); - tombstone.version().setLongValue(plan.versionOfDeletion); - final ParseContext.Document doc = tombstone.docs().get(0); - assert doc.getField(SeqNoFieldMapper.TOMBSTONE_NAME) != null : - "Delete tombstone document but _tombstone field is not set [" + doc + " ]"; - doc.add(softDeletesField); - if (plan.addStaleOpToLucene || plan.currentlyDeleted) { - indexWriter.addDocument(doc); - } else { - indexWriter.softUpdateDocument(delete.uid(), doc, softDeletesField); - } - } else if (plan.currentlyDeleted == false) { + if (plan.currentlyDeleted == false) { // any exception that comes from this is a either an ACE or a fatal exception there // can't be any document failures coming from this indexWriter.deleteDocuments(delete.uid()); - } - if (plan.deleteFromLucene) { numDocDeletes.inc(); - versionMap.putDeleteUnderLock(delete.uid().bytes(), - new DeleteVersionValue(plan.versionOfDeletion, plan.seqNoOfDeletion, delete.primaryTerm(), - engineConfig.getThreadPool().relativeTimeInMillis())); } + versionMap.putDeleteUnderLock(delete.uid().bytes(), + new DeleteVersionValue(plan.versionOfDeletion, plan.seqNoOfDeletion, delete.primaryTerm(), + engineConfig.getThreadPool().relativeTimeInMillis())); return new DeleteResult( plan.versionOfDeletion, getPrimaryTerm(), plan.seqNoOfDeletion, plan.currentlyDeleted == false); } catch (Exception ex) { @@ -1301,16 +1219,15 @@ public class InternalEngine extends Engine { } } - protected static final class DeletionStrategy { + private static final class DeletionStrategy { // of a rare double delete final boolean deleteFromLucene; - final boolean addStaleOpToLucene; final boolean currentlyDeleted; final long seqNoOfDeletion; final long versionOfDeletion; final Optional earlyResultOnPreflightError; - private DeletionStrategy(boolean deleteFromLucene, boolean addStaleOpToLucene, boolean currentlyDeleted, + private DeletionStrategy(boolean deleteFromLucene, boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion, DeleteResult earlyResultOnPreflightError) { assert (deleteFromLucene && earlyResultOnPreflightError != null) == false : @@ -1318,7 +1235,6 @@ public class InternalEngine extends Engine { "deleteFromLucene: " + deleteFromLucene + " earlyResultOnPreFlightError:" + earlyResultOnPreflightError; this.deleteFromLucene = deleteFromLucene; - this.addStaleOpToLucene = addStaleOpToLucene; this.currentlyDeleted = currentlyDeleted; this.seqNoOfDeletion = seqNoOfDeletion; this.versionOfDeletion = versionOfDeletion; @@ -1330,22 +1246,16 @@ public class InternalEngine extends Engine { VersionConflictEngineException e, long currentVersion, long term, boolean currentlyDeleted) { final long unassignedSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; final DeleteResult deleteResult = new DeleteResult(e, currentVersion, term, unassignedSeqNo, currentlyDeleted == false); - return new DeletionStrategy(false, false, currentlyDeleted, unassignedSeqNo, Versions.NOT_FOUND, deleteResult); + return new DeletionStrategy(false, currentlyDeleted, unassignedSeqNo, Versions.NOT_FOUND, deleteResult); } static DeletionStrategy processNormally(boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(true, false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); + return new DeletionStrategy(true, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); } - public static DeletionStrategy processButSkipLucene(boolean currentlyDeleted, - long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(false, false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); - } - - static DeletionStrategy processAsStaleOp(boolean addStaleOpToLucene, boolean currentlyDeleted, - long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(false, addStaleOpToLucene, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); + public static DeletionStrategy processButSkipLucene(boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion) { + return new DeletionStrategy(false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); } } @@ -1374,28 +1284,7 @@ public class InternalEngine extends Engine { assert noOp.seqNo() > SequenceNumbers.NO_OPS_PERFORMED; final long seqNo = noOp.seqNo(); try { - Exception failure = null; - if (softDeleteEnabled) { - try { - final ParsedDocument tombstone = engineConfig.getTombstoneDocSupplier().newNoopTombstoneDoc(noOp.reason()); - tombstone.updateSeqID(noOp.seqNo(), noOp.primaryTerm()); - // A noop tombstone does not require a _version but it's added to have a fully dense docvalues for the version field. - // 1L is selected to optimize the compression because it might probably be the most common value in version field. - tombstone.version().setLongValue(1L); - assert tombstone.docs().size() == 1 : "Tombstone should have a single doc [" + tombstone + "]"; - final ParseContext.Document doc = tombstone.docs().get(0); - assert doc.getField(SeqNoFieldMapper.TOMBSTONE_NAME) != null - : "Noop tombstone document but _tombstone field is not set [" + doc + " ]"; - doc.add(softDeletesField); - indexWriter.addDocument(doc); - } catch (Exception ex) { - if (maybeFailEngine("noop", ex)) { - throw ex; - } - failure = ex; - } - } - final NoOpResult noOpResult = failure != null ? new NoOpResult(getPrimaryTerm(), noOp.seqNo(), failure) : new NoOpResult(getPrimaryTerm(), noOp.seqNo()); + final NoOpResult noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo()); if (noOp.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { final Translog.Location location = translog.add(new Translog.NoOp(noOp.seqNo(), noOp.primaryTerm(), noOp.reason())); noOpResult.setTranslogLocation(location); @@ -1420,7 +1309,6 @@ public class InternalEngine extends Engine { // since it flushes the index as well (though, in terms of concurrency, we are allowed to do it) // both refresh types will result in an internal refresh but only the external will also // pass the new reader reference to the external reader manager. - final long localCheckpointBeforeRefresh = getLocalCheckpoint(); // this will also cause version map ram to be freed hence we always account for it. final long bytes = indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh(); @@ -1446,7 +1334,6 @@ public class InternalEngine extends Engine { } finally { store.decRef(); } - lastRefreshedCheckpointListener.updateRefreshedCheckpoint(localCheckpointBeforeRefresh); } } catch (AlreadyClosedException e) { failOnTragicEvent(e); @@ -1461,8 +1348,7 @@ public class InternalEngine extends Engine { } finally { writingBytes.addAndGet(-bytes); } - assert lastRefreshedCheckpoint() >= localCheckpointBeforeRefresh : "refresh checkpoint was not advanced; " + - "local_checkpoint=" + localCheckpointBeforeRefresh + " refresh_checkpoint=" + lastRefreshedCheckpoint(); + // TODO: maybe we should just put a scheduled job in threadPool? // We check for pruning in each delete request, but we also prune here e.g. in case a delete burst comes in and then no more deletes // for a long time: @@ -2044,11 +1930,7 @@ public class InternalEngine extends Engine { // pkg-private for testing IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOException { - if (Assertions.ENABLED) { - return new AssertingIndexWriter(directory, iwc); - } else { - return new IndexWriter(directory, iwc); - } + return new IndexWriter(directory, iwc); } private IndexWriterConfig getIndexWriterConfig() { @@ -2064,15 +1946,11 @@ public class InternalEngine extends Engine { } iwc.setInfoStream(verbose ? InfoStream.getDefault() : new LoggerInfoStream(logger)); iwc.setMergeScheduler(mergeScheduler); + MergePolicy mergePolicy = config().getMergePolicy(); // Give us the opportunity to upgrade old segments while performing // background merges - MergePolicy mergePolicy = config().getMergePolicy(); - if (softDeleteEnabled) { - iwc.setSoftDeletesField(Lucene.SOFT_DELETES_FIELD); - mergePolicy = new RecoverySourcePruneMergePolicy(SourceFieldMapper.RECOVERY_SOURCE_NAME, softDeletesPolicy::getRetentionQuery, - new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, softDeletesPolicy::getRetentionQuery, mergePolicy)); - } - iwc.setMergePolicy(new ElasticsearchMergePolicy(mergePolicy)); + mergePolicy = new ElasticsearchMergePolicy(mergePolicy); + iwc.setMergePolicy(mergePolicy); iwc.setSimilarity(engineConfig.getSimilarity()); iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().getMbFrac()); iwc.setCodec(engineConfig.getCodec()); @@ -2269,9 +2147,6 @@ public class InternalEngine extends Engine { commitData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(localCheckpointTracker.getMaxSeqNo())); commitData.put(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, Long.toString(maxUnsafeAutoIdTimestamp.get())); commitData.put(HISTORY_UUID_KEY, historyUUID); - if (softDeleteEnabled) { - commitData.put(Engine.MIN_RETAINED_SEQNO, Long.toString(softDeletesPolicy.getMinRetainedSeqNo())); - } logger.trace("committing writer with commit data [{}]", commitData); return commitData.entrySet().iterator(); }); @@ -2327,7 +2202,6 @@ public class InternalEngine extends Engine { final IndexSettings indexSettings = engineConfig.getIndexSettings(); translogDeletionPolicy.setRetentionAgeInMillis(indexSettings.getTranslogRetentionAge().getMillis()); translogDeletionPolicy.setRetentionSizeInBytes(indexSettings.getTranslogRetentionSize().getBytes()); - softDeletesPolicy.setRetentionOperations(indexSettings.getSoftDeleteRetentionOperations()); } public MergeStats getMergeStats() { @@ -2422,69 +2296,6 @@ public class InternalEngine extends Engine { return numDocUpdates.count(); } - @Override - public Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, - long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException { - // TODO: Should we defer the refresh until we really need it? - ensureOpen(); - if (lastRefreshedCheckpoint() < toSeqNo) { - refresh(source, SearcherScope.INTERNAL); - } - Searcher searcher = acquireSearcher(source, SearcherScope.INTERNAL); - try { - LuceneChangesSnapshot snapshot = new LuceneChangesSnapshot( - searcher, mapperService, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE, fromSeqNo, toSeqNo, requiredFullRange); - searcher = null; - return snapshot; - } catch (Exception e) { - try { - maybeFailEngine("acquire changes snapshot", e); - } catch (Exception inner) { - e.addSuppressed(inner); - } - throw e; - } finally { - IOUtils.close(searcher); - } - } - - @Override - public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException { - if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { - return getMinRetainedSeqNo() <= startingSeqNo; - } else { - final long currentLocalCheckpoint = getLocalCheckpointTracker().getCheckpoint(); - final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1); - try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(startingSeqNo)) { - Translog.Operation operation; - while ((operation = snapshot.next()) != null) { - if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - tracker.markSeqNoAsCompleted(operation.seqNo()); - } - } - } - return tracker.getCheckpoint() >= currentLocalCheckpoint; - } - } - - /** - * Returns the minimum seqno that is retained in the Lucene index. - * Operations whose seq# are at least this value should exist in the Lucene index. - */ - final long getMinRetainedSeqNo() { - assert softDeleteEnabled : Thread.currentThread().getName(); - return softDeletesPolicy.getMinRetainedSeqNo(); - } - - @Override - public Closeable acquireRetentionLockForPeerRecovery() { - if (softDeleteEnabled) { - return softDeletesPolicy.acquireRetentionLock(); - } else { - return translog.acquireRetentionLock(); - } - } - @Override public boolean isRecovering() { return pendingTranslogRecovery.get(); @@ -2500,69 +2311,4 @@ public class InternalEngine extends Engine { } return commitData; } - - private final class AssertingIndexWriter extends IndexWriter { - AssertingIndexWriter(Directory d, IndexWriterConfig conf) throws IOException { - super(d, conf); - } - @Override - public long updateDocument(Term term, Iterable doc) throws IOException { - assert softDeleteEnabled == false : "Call #updateDocument but soft-deletes is enabled"; - return super.updateDocument(term, doc); - } - @Override - public long updateDocuments(Term delTerm, Iterable> docs) throws IOException { - assert softDeleteEnabled == false : "Call #updateDocuments but soft-deletes is enabled"; - return super.updateDocuments(delTerm, docs); - } - @Override - public long deleteDocuments(Term... terms) throws IOException { - assert softDeleteEnabled == false : "Call #deleteDocuments but soft-deletes is enabled"; - return super.deleteDocuments(terms); - } - @Override - public long softUpdateDocument(Term term, Iterable doc, Field... softDeletes) throws IOException { - assert softDeleteEnabled : "Call #softUpdateDocument but soft-deletes is disabled"; - return super.softUpdateDocument(term, doc, softDeletes); - } - @Override - public long softUpdateDocuments(Term term, Iterable> docs, Field... softDeletes) throws IOException { - assert softDeleteEnabled : "Call #softUpdateDocuments but soft-deletes is disabled"; - return super.softUpdateDocuments(term, docs, softDeletes); - } - } - - /** - * Returned the last local checkpoint value has been refreshed internally. - */ - final long lastRefreshedCheckpoint() { - return lastRefreshedCheckpointListener.refreshedCheckpoint.get(); - } - - private final class LastRefreshedCheckpointListener implements ReferenceManager.RefreshListener { - final AtomicLong refreshedCheckpoint; - private long pendingCheckpoint; - - LastRefreshedCheckpointListener(long initialLocalCheckpoint) { - this.refreshedCheckpoint = new AtomicLong(initialLocalCheckpoint); - } - - @Override - public void beforeRefresh() { - // all changes until this point should be visible after refresh - pendingCheckpoint = localCheckpointTracker.getCheckpoint(); - } - - @Override - public void afterRefresh(boolean didRefresh) { - if (didRefresh) { - updateRefreshedCheckpoint(pendingCheckpoint); - } - } - - void updateRefreshedCheckpoint(long checkpoint) { - refreshedCheckpoint.updateAndGet(curr -> Math.max(curr, checkpoint)); - assert refreshedCheckpoint.get() >= checkpoint : refreshedCheckpoint.get() + " < " + checkpoint; - } - } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java deleted file mode 100644 index deebfba9ed4..00000000000 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ /dev/null @@ -1,368 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -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.index.Term; -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.util.ArrayUtil; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.index.fieldvisitor.FieldsVisitor; -import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.SourceFieldMapper; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.VersionFieldMapper; -import org.elasticsearch.index.translog.Translog; - -import java.io.Closeable; -import java.io.IOException; -import java.util.Comparator; -import java.util.List; -import java.util.Objects; -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; - private long lastSeenSeqNo; - private int skippedOperations; - private final boolean requiredFullRange; - - private final IndexSearcher indexSearcher; - private final MapperService mapperService; - private int docIndex = 0; - private final int totalHits; - private ScoreDoc[] scoreDocs; - private final ParallelArray parallelArray; - private final Closeable onClose; - - /** - * Creates a new "translog" snapshot from Lucene for reading operations whose seq# in the specified range. - * - * @param engineSearcher the internal engine searcher which will be taken over if the snapshot is opened successfully - * @param mapperService the mapper service which will be mainly used to resolve the document's type and uid - * @param searchBatchSize the number of documents should be returned by each search - * @param fromSeqNo the min requesting seq# - inclusive - * @param toSeqNo the maximum requesting seq# - inclusive - * @param requiredFullRange if true, the snapshot will strictly check for the existence of operations between fromSeqNo and toSeqNo - */ - LuceneChangesSnapshot(Engine.Searcher engineSearcher, MapperService mapperService, int searchBatchSize, - long fromSeqNo, long toSeqNo, boolean requiredFullRange) 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.mapperService = mapperService; - this.searchBatchSize = searchBatchSize; - this.fromSeqNo = fromSeqNo; - this.toSeqNo = toSeqNo; - this.lastSeenSeqNo = fromSeqNo - 1; - this.requiredFullRange = requiredFullRange; - this.indexSearcher = new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); - this.indexSearcher.setQueryCache(null); - this.parallelArray = new ParallelArray(searchBatchSize); - final TopDocs topDocs = searchOperations(null); - this.totalHits = Math.toIntExact(topDocs.totalHits); - this.scoreDocs = topDocs.scoreDocs; - fillParallelArray(scoreDocs, parallelArray); - } - - @Override - public void close() throws IOException { - onClose.close(); - } - - @Override - public int totalOperations() { - return totalHits; - } - - @Override - public int skippedOperations() { - return skippedOperations; - } - - @Override - public Translog.Operation next() throws IOException { - Translog.Operation op = null; - for (int idx = nextDocIndex(); idx != -1; idx = nextDocIndex()) { - op = readDocAsOp(idx); - if (op != null) { - break; - } - } - if (requiredFullRange) { - rangeCheck(op); - } - if (op != null) { - lastSeenSeqNo = op.seqNo(); - } - return op; - } - - private void rangeCheck(Translog.Operation op) { - if (op == null) { - if (lastSeenSeqNo < toSeqNo) { - throw new IllegalStateException("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 IllegalStateException("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(prev).scoreDocs; - fillParallelArray(scoreDocs, parallelArray); - docIndex = 0; - } - if (docIndex < scoreDocs.length) { - int idx = docIndex; - docIndex++; - return idx; - } - return -1; - } - - private void fillParallelArray(ScoreDoc[] scoreDocs, ParallelArray parallelArray) throws IOException { - if (scoreDocs.length > 0) { - for (int i = 0; i < scoreDocs.length; i++) { - scoreDocs[i].shardIndex = i; - } - // for better loading performance we sort the array by docID and - // then visit all leaves in order. - ArrayUtil.introSort(scoreDocs, Comparator.comparingInt(i -> i.doc)); - int docBase = -1; - int maxDoc = 0; - List leaves = indexSearcher.getIndexReader().leaves(); - int readerIndex = 0; - CombinedDocValues combinedDocValues = null; - LeafReaderContext leaf = null; - for (int i = 0; i < scoreDocs.length; i++) { - ScoreDoc scoreDoc = scoreDocs[i]; - if (scoreDoc.doc >= docBase + maxDoc) { - do { - leaf = leaves.get(readerIndex++); - docBase = leaf.docBase; - maxDoc = leaf.reader().maxDoc(); - } while (scoreDoc.doc >= docBase + maxDoc); - combinedDocValues = new CombinedDocValues(leaf.reader()); - } - final int segmentDocID = scoreDoc.doc - docBase; - final int index = scoreDoc.shardIndex; - parallelArray.leafReaderContexts[index] = leaf; - parallelArray.seqNo[index] = combinedDocValues.docSeqNo(segmentDocID); - parallelArray.primaryTerm[index] = combinedDocValues.docPrimaryTerm(segmentDocID); - parallelArray.version[index] = combinedDocValues.docVersion(segmentDocID); - parallelArray.isTombStone[index] = combinedDocValues.isTombstone(segmentDocID); - parallelArray.hasRecoverySource[index] = combinedDocValues.hasRecoverySource(segmentDocID); - } - // now sort back based on the shardIndex. we use this to store the previous index - ArrayUtil.introSort(scoreDocs, Comparator.comparingInt(i -> i.shardIndex)); - } - } - - private TopDocs searchOperations(ScoreDoc after) throws IOException { - final Query rangeQuery = LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, lastSeenSeqNo + 1, toSeqNo); - final Sort sortedBySeqNoThenByTerm = new Sort( - new SortField(SeqNoFieldMapper.NAME, SortField.Type.LONG), - new SortField(SeqNoFieldMapper.PRIMARY_TERM_NAME, SortField.Type.LONG, true) - ); - return indexSearcher.searchAfter(after, rangeQuery, searchBatchSize, sortedBySeqNoThenByTerm); - } - - private Translog.Operation readDocAsOp(int docIndex) throws IOException { - final LeafReaderContext leaf = parallelArray.leafReaderContexts[docIndex]; - final int segmentDocID = scoreDocs[docIndex].doc - leaf.docBase; - final long primaryTerm = parallelArray.primaryTerm[docIndex]; - // We don't have to read the nested child documents - those docs don't have primary terms. - if (primaryTerm == -1) { - skippedOperations++; - return null; - } - final long seqNo = parallelArray.seqNo[docIndex]; - // Only pick the first seen seq# - if (seqNo == lastSeenSeqNo) { - skippedOperations++; - return null; - } - final long version = parallelArray.version[docIndex]; - final String sourceField = parallelArray.hasRecoverySource[docIndex] ? SourceFieldMapper.RECOVERY_SOURCE_NAME : - SourceFieldMapper.NAME; - final FieldsVisitor fields = new FieldsVisitor(true, sourceField); - leaf.reader().document(segmentDocID, fields); - fields.postProcess(mapperService); - - final Translog.Operation op; - final boolean isTombstone = parallelArray.isTombStone[docIndex]; - if (isTombstone && fields.uid() == null) { - op = new Translog.NoOp(seqNo, primaryTerm, fields.source().utf8ToString()); - assert version == 1L : "Noop tombstone should have version 1L; actual version [" + version + "]"; - assert assertDocSoftDeleted(leaf.reader(), segmentDocID) : "Noop but soft_deletes field is not set [" + op + "]"; - } else { - final String id = fields.uid().id(); - final String type = fields.uid().type(); - final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); - if (isTombstone) { - op = new Translog.Delete(type, id, uid, seqNo, primaryTerm, version); - assert assertDocSoftDeleted(leaf.reader(), segmentDocID) : "Delete op but soft_deletes field is not set [" + op + "]"; - } else { - final BytesReference source = fields.source(); - if (source == null) { - // 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 IllegalStateException("source not found for seqno=" + seqNo + - " from_seqno=" + fromSeqNo + " to_seqno=" + toSeqNo); - } else { - skippedOperations++; - return null; - } - } - // TODO: pass the latest timestamp from engine. - final long autoGeneratedIdTimestamp = -1; - op = new Translog.Index(type, id, seqNo, primaryTerm, version, - source.toBytesRef().bytes, fields.routing(), autoGeneratedIdTimestamp); - } - } - assert fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo && lastSeenSeqNo < op.seqNo() : "Unexpected operation; " + - "last_seen_seqno [" + lastSeenSeqNo + "], from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "], op [" + op + "]"; - return op; - } - - private 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 long[] version; - final long[] seqNo; - final long[] primaryTerm; - final boolean[] isTombStone; - final boolean[] hasRecoverySource; - - ParallelArray(int size) { - version = new long[size]; - seqNo = new long[size]; - primaryTerm = new long[size]; - isTombStone = new boolean[size]; - hasRecoverySource = new boolean[size]; - leafReaderContexts = new LeafReaderContext[size]; - } - } - - private static final class CombinedDocValues { - private final NumericDocValues versionDV; - private final NumericDocValues seqNoDV; - private final NumericDocValues primaryTermDV; - private final NumericDocValues tombstoneDV; - private final NumericDocValues recoverySource; - - CombinedDocValues(LeafReader leafReader) throws IOException { - this.versionDV = Objects.requireNonNull(leafReader.getNumericDocValues(VersionFieldMapper.NAME), "VersionDV is missing"); - this.seqNoDV = Objects.requireNonNull(leafReader.getNumericDocValues(SeqNoFieldMapper.NAME), "SeqNoDV is missing"); - this.primaryTermDV = Objects.requireNonNull( - leafReader.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME), "PrimaryTermDV is missing"); - this.tombstoneDV = leafReader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); - this.recoverySource = leafReader.getNumericDocValues(SourceFieldMapper.RECOVERY_SOURCE_NAME); - } - - long docVersion(int segmentDocId) throws IOException { - assert versionDV.docID() < segmentDocId; - if (versionDV.advanceExact(segmentDocId) == false) { - throw new IllegalStateException("DocValues for field [" + VersionFieldMapper.NAME + "] is not found"); - } - return versionDV.longValue(); - } - - long docSeqNo(int segmentDocId) throws IOException { - assert seqNoDV.docID() < segmentDocId; - if (seqNoDV.advanceExact(segmentDocId) == false) { - throw new IllegalStateException("DocValues for field [" + SeqNoFieldMapper.NAME + "] is not found"); - } - return seqNoDV.longValue(); - } - - long docPrimaryTerm(int segmentDocId) throws IOException { - if (primaryTermDV == null) { - return -1L; - } - assert primaryTermDV.docID() < segmentDocId; - // Use -1 for docs which don't have primary term. The caller considers those docs as nested docs. - if (primaryTermDV.advanceExact(segmentDocId) == false) { - return -1; - } - return primaryTermDV.longValue(); - } - - boolean isTombstone(int segmentDocId) throws IOException { - if (tombstoneDV == null) { - return false; - } - assert tombstoneDV.docID() < segmentDocId; - return tombstoneDV.advanceExact(segmentDocId) && tombstoneDV.longValue() > 0; - } - - boolean hasRecoverySource(int segmentDocId) throws IOException { - if (recoverySource == null) { - return false; - } - assert recoverySource.docID() < segmentDocId; - return recoverySource.advanceExact(segmentDocId); - } - } -} diff --git a/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java b/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java deleted file mode 100644 index fde97562de8..00000000000 --- a/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java +++ /dev/null @@ -1,292 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.engine; - -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.codecs.StoredFieldsReader; -import org.apache.lucene.index.BinaryDocValues; -import org.apache.lucene.index.CodecReader; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FilterCodecReader; -import org.apache.lucene.index.FilterNumericDocValues; -import org.apache.lucene.index.MergePolicy; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.index.OneMergeWrappingMergePolicy; -import org.apache.lucene.index.SortedDocValues; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.index.SortedSetDocValues; -import org.apache.lucene.index.StoredFieldVisitor; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.ConjunctionDISI; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.DocValuesFieldExistsQuery; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.Weight; -import org.apache.lucene.util.BitSet; -import org.apache.lucene.util.BitSetIterator; - -import java.io.IOException; -import java.util.Arrays; -import java.util.function.Supplier; - -final class RecoverySourcePruneMergePolicy extends OneMergeWrappingMergePolicy { - RecoverySourcePruneMergePolicy(String recoverySourceField, Supplier retainSourceQuerySupplier, MergePolicy in) { - super(in, toWrap -> new OneMerge(toWrap.segments) { - @Override - public CodecReader wrapForMerge(CodecReader reader) throws IOException { - CodecReader wrapped = toWrap.wrapForMerge(reader); - return wrapReader(recoverySourceField, wrapped, retainSourceQuerySupplier); - } - }); - } - - // pkg private for testing - static CodecReader wrapReader(String recoverySourceField, CodecReader reader, Supplier retainSourceQuerySupplier) - throws IOException { - NumericDocValues recoverySource = reader.getNumericDocValues(recoverySourceField); - 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. - } - BooleanQuery.Builder builder = new BooleanQuery.Builder(); - builder.add(new DocValuesFieldExistsQuery(recoverySourceField), BooleanClause.Occur.FILTER); - builder.add(retainSourceQuerySupplier.get(), BooleanClause.Occur.FILTER); - IndexSearcher s = new IndexSearcher(reader); - s.setQueryCache(null); - Weight weight = s.createWeight(s.rewrite(builder.build()), false, 1.0f); - Scorer scorer = weight.scorer(reader.getContext()); - if (scorer != null) { - return new SourcePruningFilterCodecReader(recoverySourceField, reader, BitSet.of(scorer.iterator(), reader.maxDoc())); - } else { - return new SourcePruningFilterCodecReader(recoverySourceField, reader, null); - } - } - - private static class SourcePruningFilterCodecReader extends FilterCodecReader { - private final BitSet recoverySourceToKeep; - private final String recoverySourceField; - - SourcePruningFilterCodecReader(String recoverySourceField, CodecReader reader, BitSet recoverySourceToKeep) { - super(reader); - this.recoverySourceField = recoverySourceField; - this.recoverySourceToKeep = recoverySourceToKeep; - } - - @Override - public DocValuesProducer getDocValuesReader() { - DocValuesProducer docValuesReader = super.getDocValuesReader(); - return new FilterDocValuesProducer(docValuesReader) { - @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"; - final DocIdSetIterator intersection; - if (recoverySourceToKeep == null) { - // we can't return null here lucenes DocIdMerger expects an instance - intersection = DocIdSetIterator.empty(); - } else { - intersection = ConjunctionDISI.intersectIterators(Arrays.asList(numeric, - new BitSetIterator(recoverySourceToKeep, recoverySourceToKeep.length()))); - } - return new FilterNumericDocValues(numeric) { - @Override - public int nextDoc() throws IOException { - return intersection.nextDoc(); - } - - @Override - public int advance(int target) { - throw new UnsupportedOperationException(); - } - - @Override - public boolean advanceExact(int target) { - throw new UnsupportedOperationException(); - } - }; - - } - return numeric; - } - }; - } - - @Override - public StoredFieldsReader getFieldsReader() { - StoredFieldsReader fieldsReader = super.getFieldsReader(); - return new FilterStoredFieldsReader(fieldsReader) { - @Override - public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException { - if (recoverySourceToKeep != null && recoverySourceToKeep.get(docID)) { - super.visitDocument(docID, visitor); - } else { - super.visitDocument(docID, new FilterStoredFieldVisitor(visitor) { - @Override - public Status needsField(FieldInfo fieldInfo) throws IOException { - if (recoverySourceField.equals(fieldInfo.name)) { - return Status.NO; - } - return super.needsField(fieldInfo); - } - }); - } - } - }; - } - - @Override - public CacheHelper getCoreCacheHelper() { - return null; - } - - @Override - public CacheHelper getReaderCacheHelper() { - return null; - } - - private static class FilterDocValuesProducer extends DocValuesProducer { - private final DocValuesProducer in; - - FilterDocValuesProducer(DocValuesProducer in) { - this.in = in; - } - - @Override - public NumericDocValues getNumeric(FieldInfo field) throws IOException { - return in.getNumeric(field); - } - - @Override - public BinaryDocValues getBinary(FieldInfo field) throws IOException { - return in.getBinary(field); - } - - @Override - public SortedDocValues getSorted(FieldInfo field) throws IOException { - return in.getSorted(field); - } - - @Override - public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { - return in.getSortedNumeric(field); - } - - @Override - public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { - return in.getSortedSet(field); - } - - @Override - public void checkIntegrity() throws IOException { - in.checkIntegrity(); - } - - @Override - public void close() throws IOException { - in.close(); - } - - @Override - public long ramBytesUsed() { - return in.ramBytesUsed(); - } - } - - private static class FilterStoredFieldsReader extends StoredFieldsReader { - - private final StoredFieldsReader fieldsReader; - - FilterStoredFieldsReader(StoredFieldsReader fieldsReader) { - this.fieldsReader = fieldsReader; - } - - @Override - public long ramBytesUsed() { - return fieldsReader.ramBytesUsed(); - } - - @Override - public void close() throws IOException { - fieldsReader.close(); - } - - @Override - public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException { - fieldsReader.visitDocument(docID, visitor); - } - - @Override - public StoredFieldsReader clone() { - return fieldsReader.clone(); - } - - @Override - public void checkIntegrity() throws IOException { - fieldsReader.checkIntegrity(); - } - } - - private static class FilterStoredFieldVisitor extends StoredFieldVisitor { - private final StoredFieldVisitor visitor; - - FilterStoredFieldVisitor(StoredFieldVisitor visitor) { - this.visitor = visitor; - } - - @Override - public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { - visitor.binaryField(fieldInfo, value); - } - - @Override - public void stringField(FieldInfo fieldInfo, byte[] value) throws IOException { - visitor.stringField(fieldInfo, value); - } - - @Override - public void intField(FieldInfo fieldInfo, int value) throws IOException { - visitor.intField(fieldInfo, value); - } - - @Override - public void longField(FieldInfo fieldInfo, long value) throws IOException { - visitor.longField(fieldInfo, value); - } - - @Override - public void floatField(FieldInfo fieldInfo, float value) throws IOException { - visitor.floatField(fieldInfo, value); - } - - @Override - public void doubleField(FieldInfo fieldInfo, double value) throws IOException { - visitor.doubleField(fieldInfo, value); - } - - @Override - public Status needsField(FieldInfo fieldInfo) throws IOException { - return visitor.needsField(fieldInfo); - } - } - } -} diff --git a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java deleted file mode 100644 index af2ded8c466..00000000000 --- a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.engine; - -import org.apache.lucene.document.LongPoint; -import org.apache.lucene.search.Query; -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.seqno.SequenceNumbers; -import org.elasticsearch.index.translog.Translog; - -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.LongSupplier; - -/** - * A policy that controls how many soft-deleted documents should be retained for peer-recovery and querying history changes purpose. - */ -final class SoftDeletesPolicy { - private final LongSupplier globalCheckpointSupplier; - private long localCheckpointOfSafeCommit; - // This lock count is used to prevent `minRetainedSeqNo` from advancing. - private int retentionLockCount; - // The extra number of operations before the global checkpoint are retained - private long retentionOperations; - // The min seq_no value that is retained - ops after this seq# should exist in the Lucene index. - private long minRetainedSeqNo; - - SoftDeletesPolicy(LongSupplier globalCheckpointSupplier, long minRetainedSeqNo, long retentionOperations) { - this.globalCheckpointSupplier = globalCheckpointSupplier; - this.retentionOperations = retentionOperations; - this.minRetainedSeqNo = minRetainedSeqNo; - this.localCheckpointOfSafeCommit = SequenceNumbers.NO_OPS_PERFORMED; - this.retentionLockCount = 0; - } - - /** - * Updates the number of soft-deleted documents prior to the global checkpoint to be retained - * See {@link org.elasticsearch.index.IndexSettings#INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING} - */ - synchronized void setRetentionOperations(long retentionOperations) { - this.retentionOperations = retentionOperations; - } - - /** - * Sets the local checkpoint of the current safe commit - */ - synchronized void setLocalCheckpointOfSafeCommit(long newCheckpoint) { - if (newCheckpoint < this.localCheckpointOfSafeCommit) { - throw new IllegalArgumentException("Local checkpoint can't go backwards; " + - "new checkpoint [" + newCheckpoint + "]," + "current checkpoint [" + localCheckpointOfSafeCommit + "]"); - } - this.localCheckpointOfSafeCommit = newCheckpoint; - } - - /** - * Acquires a lock on soft-deleted documents to prevent them from cleaning up in merge processes. This is necessary to - * make sure that all operations that are being retained will be retained until the lock is released. - * This is a analogy to the translog's retention lock; see {@link Translog#acquireRetentionLock()} - */ - synchronized Releasable acquireRetentionLock() { - assert retentionLockCount >= 0 : "Invalid number of retention locks [" + retentionLockCount + "]"; - retentionLockCount++; - final AtomicBoolean released = new AtomicBoolean(); - return () -> { - if (released.compareAndSet(false, true)) { - releaseRetentionLock(); - } - }; - } - - private synchronized void releaseRetentionLock() { - assert retentionLockCount > 0 : "Invalid number of retention locks [" + retentionLockCount + "]"; - retentionLockCount--; - } - - /** - * Returns the min seqno that is retained in the Lucene index. - * Operations whose seq# is least this value should exist in the Lucene index. - */ - synchronized long getMinRetainedSeqNo() { - // Do not advance if the retention lock is held - if (retentionLockCount == 0) { - // This policy retains operations for two purposes: peer-recovery and querying changes history. - // - Peer-recovery is driven by the local checkpoint of the safe commit. In peer-recovery, the primary transfers a safe commit, - // then sends ops after the local checkpoint of that commit. This requires keeping all ops after localCheckpointOfSafeCommit; - // - Changes APIs are driven the combination of the global checkpoint and retention ops. Here we prefer using the global - // checkpoint instead of max_seqno because only operations up to the global checkpoint are exposed in the the changes APIs. - final long minSeqNoForQueryingChanges = globalCheckpointSupplier.getAsLong() - retentionOperations; - final long minSeqNoToRetain = Math.min(minSeqNoForQueryingChanges, localCheckpointOfSafeCommit) + 1; - // This can go backward as the retentionOperations value can be changed in settings. - minRetainedSeqNo = Math.max(minRetainedSeqNo, minSeqNoToRetain); - } - return minRetainedSeqNo; - } - - /** - * Returns a soft-deletes retention query that will be used in {@link org.apache.lucene.index.SoftDeletesRetentionMergePolicy} - * Documents including tombstones are soft-deleted and matched this query will be retained and won't cleaned up by merges. - */ - Query getRetentionQuery() { - return LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, getMinRetainedSeqNo(), Long.MAX_VALUE); - } -} diff --git a/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java b/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java index 462f8ce8e68..4c65635c61b 100644 --- a/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java +++ b/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java @@ -54,19 +54,13 @@ public class FieldsVisitor extends StoredFieldVisitor { RoutingFieldMapper.NAME)); private final boolean loadSource; - private final String sourceFieldName; private final Set requiredFields; protected BytesReference source; protected String type, id; protected Map> fieldsValues; public FieldsVisitor(boolean loadSource) { - this(loadSource, SourceFieldMapper.NAME); - } - - public FieldsVisitor(boolean loadSource, String sourceFieldName) { this.loadSource = loadSource; - this.sourceFieldName = sourceFieldName; requiredFields = new HashSet<>(); reset(); } @@ -109,7 +103,7 @@ public class FieldsVisitor extends StoredFieldVisitor { @Override public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { - if (sourceFieldName.equals(fieldInfo.name)) { + if (SourceFieldMapper.NAME.equals(fieldInfo.name)) { source = new BytesArray(value); } else if (IdFieldMapper.NAME.equals(fieldInfo.name)) { id = Uid.decodeId(value); @@ -181,7 +175,7 @@ public class FieldsVisitor extends StoredFieldVisitor { requiredFields.addAll(BASE_REQUIRED_FIELDS); if (loadSource) { - requiredFields.add(sourceFieldName); + requiredFields.add(SourceFieldMapper.NAME); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index 663aa7e6f9e..a0640ac68a9 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -19,14 +19,11 @@ package org.elasticsearch.index.mapper; -import org.apache.lucene.document.StoredField; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchGenerationException; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.text.Text; @@ -42,15 +39,12 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.stream.Stream; public class DocumentMapper implements ToXContentFragment { @@ -127,8 +121,6 @@ public class DocumentMapper implements ToXContentFragment { private final Map objectMappers; private final boolean hasNestedObjects; - private final MetadataFieldMapper[] deleteTombstoneMetadataFieldMappers; - private final MetadataFieldMapper[] noopTombstoneMetadataFieldMappers; public DocumentMapper(MapperService mapperService, Mapping mapping) { this.mapperService = mapperService; @@ -179,15 +171,6 @@ public class DocumentMapper implements ToXContentFragment { } catch (Exception e) { throw new ElasticsearchGenerationException("failed to serialize source for type [" + type + "]", e); } - - final Collection deleteTombstoneMetadataFields = Arrays.asList(VersionFieldMapper.NAME, IdFieldMapper.NAME, - TypeFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME); - this.deleteTombstoneMetadataFieldMappers = Stream.of(mapping.metadataMappers) - .filter(field -> deleteTombstoneMetadataFields.contains(field.name())).toArray(MetadataFieldMapper[]::new); - final Collection noopTombstoneMetadataFields = Arrays.asList( - VersionFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME); - this.noopTombstoneMetadataFieldMappers = Stream.of(mapping.metadataMappers) - .filter(field -> noopTombstoneMetadataFields.contains(field.name())).toArray(MetadataFieldMapper[]::new); } public Mapping mapping() { @@ -259,22 +242,7 @@ public class DocumentMapper implements ToXContentFragment { } public ParsedDocument parse(SourceToParse source) throws MapperParsingException { - return documentParser.parseDocument(source, mapping.metadataMappers); - } - - public ParsedDocument createDeleteTombstoneDoc(String index, String type, String id) throws MapperParsingException { - final SourceToParse emptySource = SourceToParse.source(index, type, id, new BytesArray("{}"), XContentType.JSON); - return documentParser.parseDocument(emptySource, deleteTombstoneMetadataFieldMappers).toTombstone(); - } - - public ParsedDocument createNoopTombstoneDoc(String index, String reason) throws MapperParsingException { - final String id = ""; // _id won't be used. - final SourceToParse sourceToParse = SourceToParse.source(index, type, id, new BytesArray("{}"), XContentType.JSON); - final ParsedDocument parsedDoc = documentParser.parseDocument(sourceToParse, noopTombstoneMetadataFieldMappers).toTombstone(); - // Store the reason of a noop as a raw string in the _source field - final BytesRef byteRef = new BytesRef(reason); - parsedDoc.rootDoc().add(new StoredField(SourceFieldMapper.NAME, byteRef.bytes, byteRef.offset, byteRef.length)); - return parsedDoc; + return documentParser.parseDocument(source); } /** diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java index 85123f602ed..0fd156c0905 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -55,7 +55,7 @@ final class DocumentParser { this.docMapper = docMapper; } - ParsedDocument parseDocument(SourceToParse source, MetadataFieldMapper[] metadataFieldsMappers) throws MapperParsingException { + ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { validateType(source); final Mapping mapping = docMapper.mapping(); @@ -64,9 +64,9 @@ final class DocumentParser { try (XContentParser parser = XContentHelper.createParser(docMapperParser.getXContentRegistry(), LoggingDeprecationHandler.INSTANCE, source.source(), xContentType)) { - context = new ParseContext.InternalParseContext(indexSettings, docMapperParser, docMapper, source, parser); + context = new ParseContext.InternalParseContext(indexSettings.getSettings(), docMapperParser, docMapper, source, parser); validateStart(parser); - internalParseDocument(mapping, metadataFieldsMappers, context, parser); + internalParseDocument(mapping, context, parser); validateEnd(parser); } catch (Exception e) { throw wrapInMapperParsingException(source, e); @@ -81,11 +81,10 @@ final class DocumentParser { return parsedDocument(source, context, createDynamicUpdate(mapping, docMapper, context.getDynamicMappers())); } - private static void internalParseDocument(Mapping mapping, MetadataFieldMapper[] metadataFieldsMappers, - ParseContext.InternalParseContext context, XContentParser parser) throws IOException { + private static void internalParseDocument(Mapping mapping, ParseContext.InternalParseContext context, XContentParser parser) throws IOException { final boolean emptyDoc = isEmptyDoc(mapping, parser); - for (MetadataFieldMapper metadataMapper : metadataFieldsMappers) { + for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { metadataMapper.preParse(context); } @@ -96,7 +95,7 @@ final class DocumentParser { parseObjectOrNested(context, mapping.root); } - for (MetadataFieldMapper metadataMapper : metadataFieldsMappers) { + for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { metadataMapper.postParse(context); } } @@ -496,7 +495,7 @@ final class DocumentParser { if (builder == null) { builder = new ObjectMapper.Builder(currentFieldName).enabled(true); } - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); objectMapper = builder.build(builderContext); context.addDynamicMapper(objectMapper); context.path().add(currentFieldName); @@ -539,7 +538,7 @@ final class DocumentParser { if (builder == null) { parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName); } else { - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); mapper = builder.build(builderContext); assert mapper != null; if (mapper instanceof ArrayValueMapperParser) { @@ -697,13 +696,13 @@ final class DocumentParser { if (parseableAsLong && context.root().numericDetection()) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG); if (builder == null) { - builder = newLongBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); + builder = newLongBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); } return builder; } else if (parseableAsDouble && context.root().numericDetection()) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DOUBLE); if (builder == null) { - builder = newFloatBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); + builder = newFloatBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); } return builder; } else if (parseableAsLong == false && parseableAsDouble == false && context.root().dateDetection()) { @@ -719,7 +718,7 @@ final class DocumentParser { } Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DATE); if (builder == null) { - builder = newDateBuilder(currentFieldName, dateTimeFormatter, context.indexSettings().getIndexVersionCreated()); + builder = newDateBuilder(currentFieldName, dateTimeFormatter, Version.indexCreated(context.indexSettings())); } if (builder instanceof DateFieldMapper.Builder) { DateFieldMapper.Builder dateBuilder = (DateFieldMapper.Builder) builder; @@ -742,7 +741,7 @@ final class DocumentParser { if (numberType == XContentParser.NumberType.INT || numberType == XContentParser.NumberType.LONG) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG); if (builder == null) { - builder = newLongBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); + builder = newLongBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); } return builder; } else if (numberType == XContentParser.NumberType.FLOAT || numberType == XContentParser.NumberType.DOUBLE) { @@ -751,7 +750,7 @@ final class DocumentParser { // no templates are defined, we use float by default instead of double // since this is much more space-efficient and should be enough most of // the time - builder = newFloatBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); + builder = newFloatBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); } return builder; } @@ -786,7 +785,7 @@ final class DocumentParser { return; } final String path = context.path().pathAsText(currentFieldName); - final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); + final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); final MappedFieldType existingFieldType = context.mapperService().fullName(path); final Mapper.Builder builder; if (existingFieldType != null) { @@ -884,8 +883,8 @@ final class DocumentParser { if (builder == null) { builder = new ObjectMapper.Builder(paths[i]).enabled(true); } - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), - context.path()); mapper = (ObjectMapper) builder.build(builderContext); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + mapper = (ObjectMapper) builder.build(builderContext); if (mapper.nested() != ObjectMapper.Nested.NO) { throw new MapperParsingException("It is forbidden to create dynamic nested objects ([" + context.path().pathAsText(paths[i]) + "]) through `copy_to` or dots in field names"); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java index 8389a306270..606777392de 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java @@ -24,6 +24,7 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; import org.apache.lucene.search.Query; import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.lucene.Lucene; @@ -204,12 +205,12 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper { } @Override - public void preParse(ParseContext context) { + public void preParse(ParseContext context) throws IOException { } @Override public void postParse(ParseContext context) throws IOException { - if (context.indexSettings().getIndexVersionCreated().before(Version.V_6_1_0)) { + if (context.indexSettings().getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).before(Version.V_6_1_0)) { super.parse(context); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java index cf8cc4022fd..b77ffee05ca 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -24,8 +24,9 @@ import com.carrotsearch.hppc.ObjectObjectMap; import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexableField; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.IndexSettings; import java.util.ArrayList; import java.util.Collection; @@ -195,7 +196,7 @@ public abstract class ParseContext implements Iterable{ } @Override - public IndexSettings indexSettings() { + public Settings indexSettings() { return in.indexSettings(); } @@ -314,7 +315,8 @@ public abstract class ParseContext implements Iterable{ private final List documents; - private final IndexSettings indexSettings; + @Nullable + private final Settings indexSettings; private final SourceToParse sourceToParse; @@ -332,8 +334,8 @@ public abstract class ParseContext implements Iterable{ private final Set ignoredFields = new HashSet<>(); - public InternalParseContext(IndexSettings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, - SourceToParse source, XContentParser parser) { + public InternalParseContext(@Nullable Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, + SourceToParse source, XContentParser parser) { this.indexSettings = indexSettings; this.docMapper = docMapper; this.docMapperParser = docMapperParser; @@ -345,7 +347,7 @@ public abstract class ParseContext implements Iterable{ this.version = null; this.sourceToParse = source; this.dynamicMappers = new ArrayList<>(); - this.maxAllowedNumNestedDocs = indexSettings.getValue(MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING); + this.maxAllowedNumNestedDocs = MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING.get(indexSettings); this.numNestedDocs = 0L; } @@ -355,7 +357,8 @@ public abstract class ParseContext implements Iterable{ } @Override - public IndexSettings indexSettings() { + @Nullable + public Settings indexSettings() { return this.indexSettings; } @@ -562,7 +565,8 @@ public abstract class ParseContext implements Iterable{ return false; } - public abstract IndexSettings indexSettings(); + @Nullable + public abstract Settings indexSettings(); public abstract SourceToParse sourceToParse(); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java b/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java index d2cf17ddd35..414cb3a98ec 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java @@ -83,17 +83,6 @@ public class ParsedDocument { this.seqID.primaryTerm.setLongValue(primaryTerm); } - /** - * Makes the processing document as a tombstone document rather than a regular document. - * Tombstone documents are stored in Lucene index to represent delete operations or Noops. - */ - ParsedDocument toTombstone() { - assert docs().size() == 1 : "Tombstone should have a single doc [" + docs() + "]"; - this.seqID.tombstoneField.setLongValue(1); - rootDoc().add(this.seqID.tombstoneField); - return this; - } - public String routing() { return this.routing; } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java index 5a0db4163bf..ac3ffe46272 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java @@ -69,29 +69,26 @@ public class SeqNoFieldMapper extends MetadataFieldMapper { public final Field seqNo; public final Field seqNoDocValue; public final Field primaryTerm; - public final Field tombstoneField; - public SequenceIDFields(Field seqNo, Field seqNoDocValue, Field primaryTerm, Field tombstoneField) { + public SequenceIDFields(Field seqNo, Field seqNoDocValue, Field primaryTerm) { Objects.requireNonNull(seqNo, "sequence number field cannot be null"); Objects.requireNonNull(seqNoDocValue, "sequence number dv field cannot be null"); Objects.requireNonNull(primaryTerm, "primary term field cannot be null"); this.seqNo = seqNo; this.seqNoDocValue = seqNoDocValue; this.primaryTerm = primaryTerm; - this.tombstoneField = tombstoneField; } public static SequenceIDFields emptySeqID() { return new SequenceIDFields(new LongPoint(NAME, SequenceNumbers.UNASSIGNED_SEQ_NO), new NumericDocValuesField(NAME, SequenceNumbers.UNASSIGNED_SEQ_NO), - new NumericDocValuesField(PRIMARY_TERM_NAME, 0), new NumericDocValuesField(TOMBSTONE_NAME, 0)); + new NumericDocValuesField(PRIMARY_TERM_NAME, 0)); } } public static final String NAME = "_seq_no"; public static final String CONTENT_TYPE = "_seq_no"; public static final String PRIMARY_TERM_NAME = "_primary_term"; - public static final String TOMBSTONE_NAME = "_tombstone"; public static class SeqNoDefaults { public static final String NAME = SeqNoFieldMapper.NAME; 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 7bfe793bba4..f2090613c09 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.mapper; -import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.StoredField; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; @@ -50,7 +49,6 @@ import java.util.function.Function; public class SourceFieldMapper extends MetadataFieldMapper { public static final String NAME = "_source"; - public static final String RECOVERY_SOURCE_NAME = "_recovery_source"; public static final String CONTENT_TYPE = "_source"; private final Function, Map> filter; @@ -226,8 +224,7 @@ public class SourceFieldMapper extends MetadataFieldMapper { @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { - BytesReference originalSource = context.sourceToParse().source(); - BytesReference source = originalSource; + BytesReference source = context.sourceToParse().source(); if (enabled && fieldType().stored() && source != null) { // Percolate and tv APIs may not set the source and that is ok, because these APIs will not index any data if (filter != null) { @@ -243,17 +240,8 @@ public class SourceFieldMapper extends MetadataFieldMapper { } BytesRef ref = source.toBytesRef(); fields.add(new StoredField(fieldType().name(), ref.bytes, ref.offset, ref.length)); - } else { - source = null; } - - if (originalSource != null && source != originalSource && context.indexSettings().isSoftDeleteEnabled()) { - // 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(); - fields.add(new StoredField(RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); - fields.add(new NumericDocValuesField(RECOVERY_SOURCE_NAME, 1)); - } - } + } @Override protected String contentType() { 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 ef5f9ab0ef3..e030c95b56e 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -92,14 +92,12 @@ import org.elasticsearch.index.fielddata.ShardFieldData; import org.elasticsearch.index.flush.FlushStats; import org.elasticsearch.index.get.GetStats; import org.elasticsearch.index.get.ShardGetService; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperForType; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; @@ -1622,33 +1620,25 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl } /** - * Acquires a lock on the translog files and Lucene soft-deleted documents to prevent them from being trimmed + * Acquires a lock on the translog files, preventing them from being trimmed. */ - public Closeable acquireRetentionLockForPeerRecovery() { - return getEngine().acquireRetentionLockForPeerRecovery(); + public Closeable acquireTranslogRetentionLock() { + return getEngine().acquireTranslogRetentionLock(); } /** - * Returns the estimated number of history operations whose seq# at least the provided seq# in this shard. + * Creates a new translog snapshot for reading translog operations whose seq# at least the provided seq#. + * The caller has to close the returned snapshot after finishing the reading. */ - public int estimateNumberOfHistoryOperations(String source, long startingSeqNo) throws IOException { - return getEngine().estimateNumberOfHistoryOperations(source, mapperService, startingSeqNo); + public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { + return getEngine().newTranslogSnapshotFromMinSeqNo(minSeqNo); } /** - * Creates a new history snapshot for reading operations since the provided starting seqno (inclusive). - * The returned snapshot can be retrieved from either Lucene index or translog files. + * Returns the estimated number of operations in translog whose seq# at least the provided seq#. */ - public Translog.Snapshot getHistoryOperations(String source, long startingSeqNo) throws IOException { - return getEngine().readHistoryOperations(source, mapperService, startingSeqNo); - } - - /** - * Checks if we have a completed history of operations since the given starting seqno (inclusive). - * This method should be called after acquiring the retention lock; See {@link #acquireRetentionLockForPeerRecovery()} - */ - public boolean hasCompleteHistoryOperations(String source, long startingSeqNo) throws IOException { - return getEngine().hasCompleteOperationHistory(source, mapperService, startingSeqNo); + public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { + return getEngine().estimateTranslogOperationsFromMinSeq(minSeqNo); } public List segments(boolean verbose) { @@ -2219,7 +2209,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), - indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, () -> operationPrimaryTerm, tombstoneDocSupplier()); + indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, () -> operationPrimaryTerm); } /** @@ -2658,19 +2648,4 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl refreshMetric.inc(System.nanoTime() - currentRefreshStartTime); } } - - private EngineConfig.TombstoneDocSupplier tombstoneDocSupplier() { - final RootObjectMapper.Builder noopRootMapper = new RootObjectMapper.Builder("__noop"); - final DocumentMapper noopDocumentMapper = new DocumentMapper.Builder(noopRootMapper, mapperService).build(mapperService); - return new EngineConfig.TombstoneDocSupplier() { - @Override - public ParsedDocument newDeleteTombstoneDoc(String type, String id) { - return docMapper(type).getDocumentMapper().createDeleteTombstoneDoc(shardId.getIndexName(), type, id); - } - @Override - public ParsedDocument newNoopTombstoneDoc(String reason) { - return noopDocumentMapper.createNoopTombstoneDoc(shardId.getIndexName(), reason); - } - }; - } } 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 016a8afff69..1edc0eb5dca 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java +++ b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java @@ -89,7 +89,7 @@ public class PrimaryReplicaSyncer extends AbstractComponent { // 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.getHistoryOperations("resync", startingSeqNo); + snapshot = indexShard.newTranslogSnapshotFromMinSeqNo(startingSeqNo); final Translog.Snapshot originalSnapshot = snapshot; final Translog.Snapshot wrappedSnapshot = new Translog.Snapshot() { @Override diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index ae3f90e63e7..e9acfe3d8b0 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -156,7 +156,6 @@ final class StoreRecovery { final Directory hardLinkOrCopyTarget = new org.apache.lucene.store.HardlinkCopyDirectoryWrapper(target); IndexWriterConfig iwc = new IndexWriterConfig(null) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) // we don't want merges to happen here - we call maybe merge on the engine // later once we stared it up otherwise we would need to wait for it here diff --git a/server/src/main/java/org/elasticsearch/index/store/Store.java b/server/src/main/java/org/elasticsearch/index/store/Store.java index 85975bc68c8..001e263ea8f 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -1009,6 +1009,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref } final String segmentId = IndexFileNames.parseSegmentName(meta.name()); final String extension = IndexFileNames.getExtension(meta.name()); + assert FIELD_INFOS_FILE_EXTENSION.equals(extension) == false || IndexFileNames.stripExtension(IndexFileNames.stripSegmentName(meta.name())).isEmpty() : "FieldInfos are generational but updateable DV are not supported in elasticsearch"; if (IndexFileNames.SEGMENTS.equals(segmentId) || DEL_FILE_EXTENSION.equals(extension) || LIV_FILE_EXTENSION.equals(extension)) { // only treat del files as per-commit files fnm files are generational but only for upgradable DV perCommitStoreFiles.add(meta); @@ -1594,7 +1595,6 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref throws IOException { assert openMode == IndexWriterConfig.OpenMode.APPEND || commit == null : "can't specify create flag with a commit"; IndexWriterConfig iwc = new IndexWriterConfig(null) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) .setIndexCommit(commit) // we don't want merges to happen here - we call maybe merge on the engine diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index f17acac3789..618aa546e42 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -1261,8 +1261,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC ", type='" + type + '\'' + ", seqNo=" + seqNo + ", primaryTerm=" + primaryTerm + - ", version=" + version + - ", autoGeneratedIdTimestamp=" + autoGeneratedIdTimestamp + '}'; } @@ -1405,7 +1403,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC "uid=" + uid + ", seqNo=" + seqNo + ", primaryTerm=" + primaryTerm + - ", version=" + version + '}'; } } diff --git a/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java b/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java index e0cfe9eaaff..f48f2ceb792 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -40,7 +40,6 @@ import java.nio.file.Path; import java.nio.file.StandardOpenOption; import java.util.HashMap; import java.util.Map; -import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.LongSupplier; @@ -193,24 +192,7 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable { new BufferedChecksumStreamInput(data.streamInput(), "assertion")); Translog.Operation prvOp = Translog.readOperation( new BufferedChecksumStreamInput(previous.v1().streamInput(), "assertion")); - // TODO: We haven't had timestamp for Index operations in Lucene yet, we need to loosen this check without timestamp. - final boolean sameOp; - if (newOp instanceof Translog.Index && prvOp instanceof Translog.Index) { - final Translog.Index o1 = (Translog.Index) prvOp; - final Translog.Index o2 = (Translog.Index) newOp; - sameOp = Objects.equals(o1.id(), o2.id()) && Objects.equals(o1.type(), o2.type()) - && Objects.equals(o1.source(), o2.source()) && Objects.equals(o1.routing(), o2.routing()) - && o1.primaryTerm() == o2.primaryTerm() && o1.seqNo() == o2.seqNo() - && o1.version() == o2.version(); - } else if (newOp instanceof Translog.Delete && prvOp instanceof Translog.Delete) { - final Translog.Delete o1 = (Translog.Delete) newOp; - final Translog.Delete o2 = (Translog.Delete) prvOp; - sameOp = Objects.equals(o1.id(), o2.id()) && Objects.equals(o1.type(), o2.type()) - && o1.primaryTerm() == o2.primaryTerm() && o1.seqNo() == o2.seqNo() && o1.version() == o2.version(); - } else { - sameOp = false; - } - if (sameOp == false) { + if (newOp.equals(prvOp) == false) { throw new AssertionError( "seqNo [" + seqNo + "] was processed twice in generation [" + generation + "], with different data. " + "prvOp [" + prvOp + "], newOp [" + newOp + "]", previous.v2()); diff --git a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java index a90f8af0af4..86995ae7c5a 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java @@ -32,7 +32,6 @@ import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.Lock; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.store.NativeFSLockFactory; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cli.EnvironmentAwareCommand; @@ -178,7 +177,6 @@ public class TruncateTranslogCommand extends EnvironmentAwareCommand { terminal.println("Marking index with the new history uuid"); // commit the new histroy id IndexWriterConfig iwc = new IndexWriterConfig(null) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) // we don't want merges to happen here - we call maybe merge on the engine // later once we stared it up otherwise we would need to wait for it here 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 10f796e5e15..352f07d5764 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -146,11 +146,11 @@ public class RecoverySourceHandler { assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; }, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger); - try (Closeable ignored = shard.acquireRetentionLockForPeerRecovery()) { + try (Closeable ignored = shard.acquireTranslogRetentionLock()) { final long startingSeqNo; final long requiredSeqNoRangeStart; final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && - isTargetSameHistory() && shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo()); + isTargetSameHistory() && isTranslogReadyForSequenceNumberBasedRecovery(); if (isSequenceNumberBasedRecovery) { logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); startingSeqNo = request.startingSeqNo(); @@ -162,16 +162,14 @@ public class RecoverySourceHandler { } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e); } - // We must have everything above the local checkpoint in the commit + // we set this to 0 to create a translog roughly according to the retention policy + // on the target. Note that it will still filter out legacy operations with no sequence numbers + startingSeqNo = 0; + // but we must have everything above the local checkpoint in the commit requiredSeqNoRangeStart = Long.parseLong(phase1Snapshot.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1; - // If soft-deletes enabled, we need to transfer only operations after the local_checkpoint of the commit to have - // the same history on the target. However, with translog, we need to set this to 0 to create a translog roughly - // according to the retention policy on the target. Note that it will still filter out legacy operations without seqNo. - startingSeqNo = shard.indexSettings().isSoftDeleteEnabled() ? requiredSeqNoRangeStart : 0; try { - final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo); - phase1(phase1Snapshot.getIndexCommit(), () -> estimateNumOps); + phase1(phase1Snapshot.getIndexCommit(), () -> shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e); } finally { @@ -188,8 +186,7 @@ public class RecoverySourceHandler { try { // For a sequence based recovery, the target can keep its local translog - prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, - shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); + prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); } @@ -210,13 +207,11 @@ public class RecoverySourceHandler { */ cancellableThreads.execute(() -> shard.waitForOpsToComplete(endingSeqNo)); - if (logger.isTraceEnabled()) { - logger.trace("all operations up to [{}] completed, which will be used as an ending sequence number", endingSeqNo); - logger.trace("snapshot translog for recovery; current size is [{}]", - shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); - } + logger.trace("all operations up to [{}] completed, which will be used as an ending sequence number", endingSeqNo); + + logger.trace("snapshot translog for recovery; current size is [{}]", shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); final long targetLocalCheckpoint; - try (Translog.Snapshot snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo)) { + try(Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) { targetLocalCheckpoint = phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot); } catch (Exception e) { throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); @@ -273,6 +268,36 @@ public class RecoverySourceHandler { }); } + /** + * Determines if the source translog is ready for a sequence-number-based peer recovery. The main condition here is that the source + * translog contains all operations above the local checkpoint on the target. We already know the that translog contains or will contain + * all ops above the source local checkpoint, so we can stop check there. + * + * @return {@code true} if the source is ready for a sequence-number-based recovery + * @throws IOException if an I/O exception occurred reading the translog snapshot + */ + boolean isTranslogReadyForSequenceNumberBasedRecovery() throws IOException { + final long startingSeqNo = request.startingSeqNo(); + assert startingSeqNo >= 0; + final long localCheckpoint = shard.getLocalCheckpoint(); + logger.trace("testing sequence numbers in range: [{}, {}]", startingSeqNo, localCheckpoint); + // the start recovery request is initialized with the starting sequence number set to the target shard's local checkpoint plus one + if (startingSeqNo - 1 <= localCheckpoint) { + final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1); + try (Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) { + Translog.Operation operation; + while ((operation = snapshot.next()) != null) { + if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { + tracker.markSeqNoAsCompleted(operation.seqNo()); + } + } + } + return tracker.getCheckpoint() >= localCheckpoint; + } else { + return false; + } + } + /** * Perform phase1 of the recovery operations. Once this {@link IndexCommit} * snapshot has been performed no commit operations (files being fsync'd) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 9469f657c96..a4d6518e9af 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1492,7 +1492,6 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp // empty shard would cause exceptions to be thrown. Since there is no data to restore from an empty // shard anyway, we just create the empty shard here and then exit. IndexWriter writer = new IndexWriter(store.directory(), new IndexWriterConfig(null) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setOpenMode(IndexWriterConfig.OpenMode.CREATE) .setCommitOnClose(true)); writer.close(); diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 6acdbad2cce..702d63d0d94 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -64,7 +64,6 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.repositories.IndexId; @@ -121,8 +120,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateApp SETTING_NUMBER_OF_SHARDS, SETTING_VERSION_CREATED, SETTING_INDEX_UUID, - SETTING_CREATION_DATE, - IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey())); + SETTING_CREATION_DATE)); // It's OK to change some settings, but we shouldn't allow simply removing them private static final Set UNREMOVABLE_SETTINGS; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 9786c0eaf52..90173455c3b 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -392,7 +392,6 @@ public class PrimaryAllocationIT extends ESIntegTestCase { assertThat(shard.getLocalCheckpoint(), equalTo(numDocs + moreDocs)); } }, 30, TimeUnit.SECONDS); - internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); } } diff --git a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java index 890f6ef163b..753aedea01e 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java @@ -33,23 +33,18 @@ import org.apache.lucene.index.NoDeletionPolicy; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; -import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TermQuery; -import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.Weight; import org.apache.lucene.store.Directory; import org.apache.lucene.store.MMapDirectory; import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.util.Bits; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.test.ESTestCase; import java.io.IOException; -import java.io.StringReader; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -58,8 +53,6 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; -import static org.hamcrest.Matchers.equalTo; - public class LuceneTests extends ESTestCase { public void testWaitForIndex() throws Exception { final MockDirectoryWrapper dir = newMockDirectory(); @@ -413,88 +406,4 @@ public class LuceneTests extends ESTestCase { // add assume's here if needed for certain platforms, but we should know if it does not work. assertTrue("MMapDirectory does not support unmapping: " + MMapDirectory.UNMAP_NOT_SUPPORTED_REASON, MMapDirectory.UNMAP_SUPPORTED); } - - public void testWrapAllDocsLive() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) - .setMergePolicy(new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, MatchAllDocsQuery::new, newMergePolicy())); - IndexWriter writer = new IndexWriter(dir, config); - int numDocs = between(1, 10); - Set liveDocs = new HashSet<>(); - for (int i = 0; i < numDocs; i++) { - String id = Integer.toString(i); - Document doc = new Document(); - doc.add(new StringField("id", id, Store.YES)); - writer.addDocument(doc); - liveDocs.add(id); - } - for (int i = 0; i < numDocs; i++) { - if (randomBoolean()) { - String id = Integer.toString(i); - Document doc = new Document(); - doc.add(new StringField("id", "v2-" + id, Store.YES)); - if (randomBoolean()) { - doc.add(Lucene.newSoftDeletesField()); - } - writer.softUpdateDocument(new Term("id", id), doc, Lucene.newSoftDeletesField()); - liveDocs.add("v2-" + id); - } - } - try (DirectoryReader unwrapped = DirectoryReader.open(writer)) { - DirectoryReader reader = Lucene.wrapAllDocsLive(unwrapped); - assertThat(reader.numDocs(), equalTo(liveDocs.size())); - IndexSearcher searcher = new IndexSearcher(reader); - Set actualDocs = new HashSet<>(); - TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), Integer.MAX_VALUE); - for (ScoreDoc scoreDoc : topDocs.scoreDocs) { - actualDocs.add(reader.document(scoreDoc.doc).get("id")); - } - assertThat(actualDocs, equalTo(liveDocs)); - } - IOUtils.close(writer, dir); - } - - public void testWrapLiveDocsNotExposeAbortedDocuments() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) - .setMergePolicy(new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, MatchAllDocsQuery::new, newMergePolicy())); - IndexWriter writer = new IndexWriter(dir, config); - int numDocs = between(1, 10); - List liveDocs = new ArrayList<>(); - for (int i = 0; i < numDocs; i++) { - String id = Integer.toString(i); - Document doc = new Document(); - doc.add(new StringField("id", id, Store.YES)); - if (randomBoolean()) { - doc.add(Lucene.newSoftDeletesField()); - } - writer.addDocument(doc); - liveDocs.add(id); - } - int abortedDocs = between(1, 10); - for (int i = 0; i < abortedDocs; i++) { - try { - Document doc = new Document(); - doc.add(new StringField("id", "aborted-" + i, Store.YES)); - StringReader reader = new StringReader(""); - doc.add(new TextField("other", reader)); - reader.close(); // mark the indexing hit non-aborting error - writer.addDocument(doc); - fail("index should have failed"); - } catch (Exception ignored) { } - } - try (DirectoryReader unwrapped = DirectoryReader.open(writer)) { - DirectoryReader reader = Lucene.wrapAllDocsLive(unwrapped); - assertThat(reader.maxDoc(), equalTo(numDocs + abortedDocs)); - assertThat(reader.numDocs(), equalTo(liveDocs.size())); - IndexSearcher searcher = new IndexSearcher(reader); - List actualDocs = new ArrayList<>(); - TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), Integer.MAX_VALUE); - for (ScoreDoc scoreDoc : topDocs.scoreDocs) { - actualDocs.add(reader.document(scoreDoc.doc).get("id")); - } - assertThat(actualDocs, equalTo(liveDocs)); - } - IOUtils.close(writer, dir); - } } diff --git a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java index ac2f2b0d4f3..6bdd8ea3f2e 100644 --- a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java +++ b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java @@ -109,7 +109,6 @@ public abstract class AbstractDisruptionTestCase extends ESIntegTestCase { protected void beforeIndexDeletion() throws Exception { if (disableBeforeIndexDeletion == false) { super.beforeIndexDeletion(); - internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); assertSeqNos(); } } diff --git a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index b0b6c35f92a..d098c4918a7 100644 --- a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -40,7 +40,6 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; -import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -398,8 +397,7 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { .get(); logger.info("--> indexing docs"); - int numDocs = randomIntBetween(1, 1024); - for (int i = 0; i < numDocs; i++) { + for (int i = 0; i < randomIntBetween(1, 1024); i++) { client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); } @@ -421,15 +419,12 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { } logger.info("--> restart replica node"); - boolean softDeleteEnabled = internalCluster().getInstance(IndicesService.class, primaryNode) - .indexServiceSafe(resolveIndex("test")).getShard(0).indexSettings().isSoftDeleteEnabled(); - int moreDocs = randomIntBetween(1, 1024); internalCluster().restartNode(replicaNode, new RestartCallback() { @Override public Settings onNodeStopped(String nodeName) throws Exception { // index some more documents; we expect to reuse the files that already exist on the replica - for (int i = 0; i < moreDocs; i++) { + for (int i = 0; i < randomIntBetween(1, 1024); i++) { client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); } @@ -437,12 +432,8 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { client(primaryNode).admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder() .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) ).get(); client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get(); - if (softDeleteEnabled) { // We need an extra flush to advance the min_retained_seqno of the SoftDeletesPolicy - client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get(); - } return super.onNodeStopped(nodeName); } }); diff --git a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java index b0b4ec3930a..28fa440d96a 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java @@ -32,7 +32,6 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -307,7 +306,7 @@ public class IndexServiceTests extends ESSingleNodeTestCase { .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), -1)) .get(); IndexShard shard = indexService.getShard(0); - assertBusy(() -> assertThat(IndexShardTestCase.getTranslog(shard).totalOperations(), equalTo(0))); + assertBusy(() -> assertThat(shard.estimateTranslogOperationsFromMinSeq(0L), equalTo(0))); } public void testIllegalFsyncInterval() { diff --git a/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java b/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java index 64a2fa69bcb..b7da5add2ac 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java @@ -553,12 +553,4 @@ public class IndexSettingsTests extends ESTestCase { ); assertThat(index.getDefaultFields(), equalTo(Arrays.asList("body", "title"))); } - - public void testUpdateSoftDeletesFails() { - IndexScopedSettings settings = new IndexScopedSettings(Settings.EMPTY, IndexScopedSettings.BUILT_IN_INDEX_SETTINGS); - IllegalArgumentException error = expectThrows(IllegalArgumentException.class, () -> - settings.updateSettings(Settings.builder().put("index.soft_deletes.enabled", randomBoolean()).build(), - Settings.builder(), Settings.builder(), "index")); - assertThat(error.getMessage(), equalTo("final index setting [index.soft_deletes.enabled], not updateable")); - } } diff --git a/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index 3f9fc9a0429..ea7de50b7b3 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -51,24 +51,20 @@ public class CombinedDeletionPolicyTests extends ESTestCase { public void testKeepCommitsAfterGlobalCheckpoint() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); - final int extraRetainedOps = between(0, 100); - final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, extraRetainedOps); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); final LongArrayList maxSeqNoList = new LongArrayList(); final LongArrayList translogGenList = new LongArrayList(); final List commitList = new ArrayList<>(); int totalCommits = between(2, 20); long lastMaxSeqNo = 0; - long lastCheckpoint = lastMaxSeqNo; long lastTranslogGen = 0; final UUID translogUUID = UUID.randomUUID(); for (int i = 0; i < totalCommits; i++) { lastMaxSeqNo += between(1, 10000); - lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); lastTranslogGen += between(1, 100); - commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); maxSeqNoList.add(lastMaxSeqNo); translogGenList.add(lastTranslogGen); } @@ -89,19 +85,14 @@ public class CombinedDeletionPolicyTests extends ESTestCase { } assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(translogGenList.get(keptIndex))); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), - equalTo(Math.min(getLocalCheckpoint(commitList.get(keptIndex)) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } public void testAcquireIndexCommit() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); - final int extraRetainedOps = between(0, 100); - final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, extraRetainedOps); final UUID translogUUID = UUID.randomUUID(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); long lastMaxSeqNo = between(1, 1000); - long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); long lastTranslogGen = between(1, 20); int safeIndex = 0; List commitList = new ArrayList<>(); @@ -111,9 +102,8 @@ public class CombinedDeletionPolicyTests extends ESTestCase { int newCommits = between(1, 10); for (int n = 0; n < newCommits; n++) { lastMaxSeqNo += between(1, 1000); - lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); lastTranslogGen += between(1, 20); - commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); } // Advance the global checkpoint to between [safeIndex, safeIndex + 1) safeIndex = randomIntBetween(safeIndex, commitList.size() - 1); @@ -124,9 +114,6 @@ public class CombinedDeletionPolicyTests extends ESTestCase { globalCheckpoint.set(randomLongBetween(lower, upper)); commitList.forEach(this::resetDeletion); indexPolicy.onCommit(commitList); - IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), - equalTo(Math.min(getLocalCheckpoint(safeCommit) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); // Captures and releases some commits int captures = between(0, 5); for (int n = 0; n < captures; n++) { @@ -145,7 +132,7 @@ public class CombinedDeletionPolicyTests extends ESTestCase { snapshottingCommits.remove(snapshot); final long pendingSnapshots = snapshottingCommits.stream().filter(snapshot::equals).count(); final IndexCommit lastCommit = commitList.get(commitList.size() - 1); - safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); + final IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); assertThat(indexPolicy.releaseCommit(snapshot), equalTo(pendingSnapshots == 0 && snapshot.equals(lastCommit) == false && snapshot.equals(safeCommit) == false)); } @@ -156,8 +143,6 @@ public class CombinedDeletionPolicyTests extends ESTestCase { equalTo(Long.parseLong(commitList.get(safeIndex).getUserData().get(Translog.TRANSLOG_GENERATION_KEY)))); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(Long.parseLong(commitList.get(commitList.size() - 1).getUserData().get(Translog.TRANSLOG_GENERATION_KEY)))); - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), - equalTo(Math.min(getLocalCheckpoint(commitList.get(safeIndex)) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } snapshottingCommits.forEach(indexPolicy::releaseCommit); globalCheckpoint.set(randomLongBetween(lastMaxSeqNo, Long.MAX_VALUE)); @@ -169,27 +154,25 @@ public class CombinedDeletionPolicyTests extends ESTestCase { assertThat(commitList.get(commitList.size() - 1).isDeleted(), equalTo(false)); assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(lastTranslogGen)); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); - IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), - equalTo(Math.min(getLocalCheckpoint(safeCommit) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } public void testLegacyIndex() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); - final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); final UUID translogUUID = UUID.randomUUID(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); long legacyTranslogGen = randomNonNegativeLong(); IndexCommit legacyCommit = mockLegacyIndexCommit(translogUUID, legacyTranslogGen); - assertThat(CombinedDeletionPolicy.findSafeCommitPoint(singletonList(legacyCommit), globalCheckpoint.get()), - equalTo(legacyCommit)); + indexPolicy.onCommit(singletonList(legacyCommit)); + verify(legacyCommit, never()).delete(); + assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(legacyTranslogGen)); + assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(legacyTranslogGen)); long safeTranslogGen = randomLongBetween(legacyTranslogGen, Long.MAX_VALUE); long maxSeqNo = randomLongBetween(1, Long.MAX_VALUE); - final IndexCommit freshCommit = mockIndexCommit(randomLongBetween(-1, maxSeqNo), maxSeqNo, translogUUID, safeTranslogGen); + final IndexCommit freshCommit = mockIndexCommit(maxSeqNo, translogUUID, safeTranslogGen); globalCheckpoint.set(randomLongBetween(0, maxSeqNo - 1)); indexPolicy.onCommit(Arrays.asList(legacyCommit, freshCommit)); @@ -206,32 +189,25 @@ public class CombinedDeletionPolicyTests extends ESTestCase { verify(freshCommit, times(0)).delete(); assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(safeTranslogGen)); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(safeTranslogGen)); - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), equalTo(getLocalCheckpoint(freshCommit) + 1)); } public void testDeleteInvalidCommits() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(randomNonNegativeLong()); - final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); final int invalidCommits = between(1, 10); final List commitList = new ArrayList<>(); for (int i = 0; i < invalidCommits; i++) { - long maxSeqNo = randomNonNegativeLong(); - commitList.add(mockIndexCommit(randomLongBetween(-1, maxSeqNo), maxSeqNo, UUID.randomUUID(), randomNonNegativeLong())); + commitList.add(mockIndexCommit(randomNonNegativeLong(), UUID.randomUUID(), randomNonNegativeLong())); } final UUID expectedTranslogUUID = UUID.randomUUID(); long lastTranslogGen = 0; final int validCommits = between(1, 10); - long lastMaxSeqNo = between(1, 1000); - long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); for (int i = 0; i < validCommits; i++) { lastTranslogGen += between(1, 1000); - lastMaxSeqNo += between(1, 1000); - lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); - commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, expectedTranslogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(randomNonNegativeLong(), expectedTranslogUUID, lastTranslogGen)); } // We should never keep invalid commits regardless of the value of the global checkpoint. @@ -239,26 +215,21 @@ public class CombinedDeletionPolicyTests extends ESTestCase { for (int i = 0; i < invalidCommits - 1; i++) { verify(commitList.get(i), times(1)).delete(); } - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), - equalTo(getLocalCheckpoint(CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get())) + 1)); } public void testCheckUnreferencedCommits() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); - final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); final UUID translogUUID = UUID.randomUUID(); final TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); final List commitList = new ArrayList<>(); int totalCommits = between(2, 20); long lastMaxSeqNo = between(1, 1000); - long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); long lastTranslogGen = between(1, 50); for (int i = 0; i < totalCommits; i++) { lastMaxSeqNo += between(1, 10000); lastTranslogGen += between(1, 100); - lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); - commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); } IndexCommit safeCommit = randomFrom(commitList); globalCheckpoint.set(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO))); @@ -285,9 +256,8 @@ public class CombinedDeletionPolicyTests extends ESTestCase { } } - IndexCommit mockIndexCommit(long localCheckpoint, long maxSeqNo, UUID translogUUID, long translogGen) throws IOException { + IndexCommit mockIndexCommit(long maxSeqNo, UUID translogUUID, long translogGen) throws IOException { final Map userData = new HashMap<>(); - userData.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(localCheckpoint)); userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); userData.put(Translog.TRANSLOG_UUID_KEY, translogUUID.toString()); userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(translogGen)); @@ -308,10 +278,6 @@ public class CombinedDeletionPolicyTests extends ESTestCase { }).when(commit).delete(); } - private long getLocalCheckpoint(IndexCommit commit) throws IOException { - return Long.parseLong(commit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); - } - IndexCommit mockLegacyIndexCommit(UUID translogUUID, long translogGen) throws IOException { final Map userData = new HashMap<>(); userData.put(Translog.TRANSLOG_UUID_KEY, translogUUID.toString()); @@ -321,5 +287,4 @@ public class CombinedDeletionPolicyTests extends ESTestCase { resetDeletion(commit); return commit; } - } 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 d3aead9e44e..76e05ba1e0b 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.engine; -import java.io.Closeable; import java.io.IOException; import java.io.UncheckedIOException; import java.nio.charset.Charset; @@ -78,12 +77,10 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.LogByteSizeMergePolicy; import org.apache.lucene.index.LogDocMergePolicy; -import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PointValues; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.index.TieredMergePolicy; import org.apache.lucene.search.IndexSearcher; @@ -117,7 +114,6 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.Loggers; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; @@ -137,7 +133,6 @@ import org.elasticsearch.index.fieldvisitor.FieldsVisitor; import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.Mapper.BuilderContext; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.ParseContext; @@ -177,10 +172,8 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.isIn; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @@ -254,13 +247,8 @@ public class InternalEngineTests extends EngineTestCase { } public void testSegments() throws Exception { - Settings settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); - IndexSettings indexSettings = IndexSettingsModule.newIndexSettings( - IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); try (Store store = createStore(); - InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null))) { + InternalEngine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { List segments = engine.segments(false); assertThat(segments.isEmpty(), equalTo(true)); assertThat(engine.segmentsStats(false).getCount(), equalTo(0L)); @@ -1323,13 +1311,9 @@ public class InternalEngineTests extends EngineTestCase { assertThat(indexResult.getVersion(), equalTo(1L)); } - public void testForceMergeWithoutSoftDeletes() throws IOException { - Settings settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); - IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); + public void testForceMerge() throws IOException { try (Store store = createStore(); - Engine engine = createEngine(config(IndexSettingsModule.newIndexSettings(indexMetaData), store, createTempDir(), + Engine engine = createEngine(config(defaultSettings, store, createTempDir(), new LogByteSizeMergePolicy(), null))) { // use log MP here we test some behavior in ESMP int numDocs = randomIntBetween(10, 100); for (int i = 0; i < numDocs; i++) { @@ -1370,165 +1354,6 @@ public class InternalEngineTests extends EngineTestCase { } } - public void testForceMergeWithSoftDeletesRetention() throws Exception { - final long retainedExtraOps = randomLongBetween(0, 10); - Settings.Builder settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), retainedExtraOps); - final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - final MapperService mapperService = createMapperService("test"); - final Set liveDocs = new HashSet<>(); - try (Store store = createStore(); - InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get))) { - int numDocs = scaledRandomIntBetween(10, 100); - for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null); - engine.index(indexForDoc(doc)); - liveDocs.add(doc.id()); - } - for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null); - if (randomBoolean()) { - engine.delete(new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get())); - liveDocs.remove(doc.id()); - } - if (randomBoolean()) { - engine.index(indexForDoc(doc)); - liveDocs.add(doc.id()); - } - if (randomBoolean()) { - engine.flush(randomBoolean(), true); - } - } - engine.flush(); - - long localCheckpoint = engine.getLocalCheckpoint(); - globalCheckpoint.set(randomLongBetween(0, localCheckpoint)); - engine.syncTranslog(); - final long safeCommitCheckpoint; - try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { - safeCommitCheckpoint = Long.parseLong(safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); - } - engine.forceMerge(true, 1, false, false, false); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - Map ops = readAllOperationsInLucene(engine, mapperService) - .stream().collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); - for (long seqno = 0; seqno <= localCheckpoint; seqno++) { - long minSeqNoToRetain = Math.min(globalCheckpoint.get() + 1 - retainedExtraOps, safeCommitCheckpoint + 1); - String msg = "seq# [" + seqno + "], global checkpoint [" + globalCheckpoint + "], retained-ops [" + retainedExtraOps + "]"; - if (seqno < minSeqNoToRetain) { - Translog.Operation op = ops.get(seqno); - if (op != null) { - assertThat(op, instanceOf(Translog.Index.class)); - assertThat(msg, ((Translog.Index) op).id(), isIn(liveDocs)); - assertEquals(msg, ((Translog.Index) op).source(), B_1); - } - } else { - assertThat(msg, ops.get(seqno), notNullValue()); - } - } - settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0); - indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); - engine.onSettingsChanged(); - globalCheckpoint.set(localCheckpoint); - engine.syncTranslog(); - - engine.forceMerge(true, 1, false, false, false); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - assertThat(readAllOperationsInLucene(engine, mapperService), hasSize(liveDocs.size())); - } - } - - public void testForceMergeWithSoftDeletesRetentionAndRecoverySource() throws Exception { - final long retainedExtraOps = randomLongBetween(0, 10); - Settings.Builder settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), retainedExtraOps); - final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - final MapperService mapperService = createMapperService("test"); - final boolean omitSourceAllTheTime = randomBoolean(); - final Set liveDocs = new HashSet<>(); - final Set liveDocsWithSource = new HashSet<>(); - try (Store store = createStore(); - InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, - globalCheckpoint::get))) { - int numDocs = scaledRandomIntBetween(10, 100); - for (int i = 0; i < numDocs; i++) { - boolean useRecoverySource = randomBoolean() || omitSourceAllTheTime; - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null, useRecoverySource); - engine.index(indexForDoc(doc)); - liveDocs.add(doc.id()); - if (useRecoverySource == false) { - liveDocsWithSource.add(Integer.toString(i)); - } - } - for (int i = 0; i < numDocs; i++) { - boolean useRecoverySource = randomBoolean() || omitSourceAllTheTime; - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null, useRecoverySource); - if (randomBoolean()) { - engine.delete(new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get())); - liveDocs.remove(doc.id()); - liveDocsWithSource.remove(doc.id()); - } - if (randomBoolean()) { - engine.index(indexForDoc(doc)); - liveDocs.add(doc.id()); - if (useRecoverySource == false) { - liveDocsWithSource.add(doc.id()); - } else { - liveDocsWithSource.remove(doc.id()); - } - } - if (randomBoolean()) { - engine.flush(randomBoolean(), true); - } - } - engine.flush(); - globalCheckpoint.set(randomLongBetween(0, engine.getLocalCheckpoint())); - engine.syncTranslog(); - final long minSeqNoToRetain; - try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { - long safeCommitLocalCheckpoint = Long.parseLong( - safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); - minSeqNoToRetain = Math.min(globalCheckpoint.get() + 1 - retainedExtraOps, safeCommitLocalCheckpoint + 1); - } - engine.forceMerge(true, 1, false, false, false); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - Map ops = readAllOperationsInLucene(engine, mapperService) - .stream().collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); - for (long seqno = 0; seqno <= engine.getLocalCheckpoint(); seqno++) { - String msg = "seq# [" + seqno + "], global checkpoint [" + globalCheckpoint + "], retained-ops [" + retainedExtraOps + "]"; - if (seqno < minSeqNoToRetain) { - Translog.Operation op = ops.get(seqno); - if (op != null) { - assertThat(op, instanceOf(Translog.Index.class)); - assertThat(msg, ((Translog.Index) op).id(), isIn(liveDocs)); - } - } else { - Translog.Operation op = ops.get(seqno); - assertThat(msg, op, notNullValue()); - if (op instanceof Translog.Index) { - assertEquals(msg, ((Translog.Index) op).source(), B_1); - } - } - } - settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0); - indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); - engine.onSettingsChanged(); - globalCheckpoint.set(engine.getLocalCheckpoint()); - engine.syncTranslog(); - engine.forceMerge(true, 1, false, false, false); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - assertThat(readAllOperationsInLucene(engine, mapperService), hasSize(liveDocsWithSource.size())); - } - } - public void testForceMergeAndClose() throws IOException, InterruptedException { int numIters = randomIntBetween(2, 10); for (int j = 0; j < numIters; j++) { @@ -1597,10 +1422,126 @@ public class InternalEngineTests extends EngineTestCase { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } + protected List generateSingleDocHistory(boolean forReplica, VersionType versionType, + long primaryTerm, + int minOpCount, int maxOpCount, String docId) { + final int numOfOps = randomIntBetween(minOpCount, maxOpCount); + final List ops = new ArrayList<>(); + final Term id = newUid(docId); + final int startWithSeqNo = 0; + final String valuePrefix = (forReplica ? "r_" : "p_" ) + docId + "_"; + final boolean incrementTermWhenIntroducingSeqNo = randomBoolean(); + for (int i = 0; i < numOfOps; i++) { + final Engine.Operation op; + final long version; + switch (versionType) { + case INTERNAL: + version = forReplica ? i : Versions.MATCH_ANY; + break; + case EXTERNAL: + version = i; + break; + case EXTERNAL_GTE: + version = randomBoolean() ? Math.max(i - 1, 0) : i; + break; + case FORCE: + version = randomNonNegativeLong(); + break; + default: + throw new UnsupportedOperationException("unknown version type: " + versionType); + } + if (randomBoolean()) { + op = new Engine.Index(id, testParsedDocument(docId, null, testDocumentWithTextField(valuePrefix + i), B_1, null), + forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, + version, + forReplica ? null : versionType, + forReplica ? REPLICA : PRIMARY, + System.currentTimeMillis(), -1, false + ); + } else { + op = new Engine.Delete("test", docId, id, + forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, + version, + forReplica ? null : versionType, + forReplica ? REPLICA : PRIMARY, + System.currentTimeMillis()); + } + ops.add(op); + } + return ops; + } + public void testOutOfOrderDocsOnReplica() throws IOException { final List ops = generateSingleDocHistory(true, randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE, VersionType.FORCE), 2, 2, 20, "1"); - assertOpsOnReplica(ops, replicaEngine, true, logger); + assertOpsOnReplica(ops, replicaEngine, true); + } + + private void assertOpsOnReplica(List ops, InternalEngine replicaEngine, boolean shuffleOps) throws IOException { + final Engine.Operation lastOp = ops.get(ops.size() - 1); + final String lastFieldValue; + if (lastOp instanceof Engine.Index) { + Engine.Index index = (Engine.Index) lastOp; + lastFieldValue = index.docs().get(0).get("value"); + } else { + // delete + lastFieldValue = null; + } + if (shuffleOps) { + int firstOpWithSeqNo = 0; + while (firstOpWithSeqNo < ops.size() && ops.get(firstOpWithSeqNo).seqNo() < 0) { + firstOpWithSeqNo++; + } + // shuffle ops but make sure legacy ops are first + shuffle(ops.subList(0, firstOpWithSeqNo), random()); + shuffle(ops.subList(firstOpWithSeqNo, ops.size()), random()); + } + boolean firstOp = true; + for (Engine.Operation op : ops) { + logger.info("performing [{}], v [{}], seq# [{}], term [{}]", + op.operationType().name().charAt(0), op.version(), op.seqNo(), op.primaryTerm()); + if (op instanceof Engine.Index) { + Engine.IndexResult result = replicaEngine.index((Engine.Index) op); + // replicas don't really care to about creation status of documents + // this allows to ignore the case where a document was found in the live version maps in + // a delete state and return false for the created flag in favor of code simplicity + // as deleted or not. This check is just signal regression so a decision can be made if it's + // intentional + assertThat(result.isCreated(), equalTo(firstOp)); + assertThat(result.getVersion(), equalTo(op.version())); + assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); + + } else { + Engine.DeleteResult result = replicaEngine.delete((Engine.Delete) op); + // Replicas don't really care to about found status of documents + // this allows to ignore the case where a document was found in the live version maps in + // a delete state and return true for the found flag in favor of code simplicity + // his check is just signal regression so a decision can be made if it's + // intentional + assertThat(result.isFound(), equalTo(firstOp == false)); + assertThat(result.getVersion(), equalTo(op.version())); + assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); + } + if (randomBoolean()) { + engine.refresh("test"); + } + if (randomBoolean()) { + engine.flush(); + engine.refresh("test"); + } + firstOp = false; + } + + assertVisibleCount(replicaEngine, lastFieldValue == null ? 0 : 1); + if (lastFieldValue != null) { + try (Searcher searcher = replicaEngine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); + assertThat(collector.getTotalHits(), equalTo(1)); + } + } } public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, InterruptedException { @@ -1628,12 +1569,11 @@ public class InternalEngineTests extends EngineTestCase { } // randomly interleave final AtomicLong seqNoGenerator = new AtomicLong(); - BiFunction seqNoUpdater = (operation, newSeqNo) -> { + Function seqNoUpdater = operation -> { + final long newSeqNo = seqNoGenerator.getAndIncrement(); if (operation instanceof Engine.Index) { Engine.Index index = (Engine.Index) operation; - Document doc = testDocumentWithTextField(index.docs().get(0).get("value")); - ParsedDocument parsedDocument = testParsedDocument(index.id(), index.routing(), doc, index.source(), null); - return new Engine.Index(index.uid(), parsedDocument, newSeqNo, index.primaryTerm(), index.version(), + return new Engine.Index(index.uid(), index.parsedDoc(), newSeqNo, index.primaryTerm(), index.version(), index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry()); } else { Engine.Delete delete = (Engine.Delete) operation; @@ -1646,12 +1586,12 @@ public class InternalEngineTests extends EngineTestCase { Iterator iter2 = opsDoc2.iterator(); while (iter1.hasNext() && iter2.hasNext()) { final Engine.Operation next = randomBoolean() ? iter1.next() : iter2.next(); - allOps.add(seqNoUpdater.apply(next, seqNoGenerator.getAndIncrement())); + allOps.add(seqNoUpdater.apply(next)); } - iter1.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o, seqNoGenerator.getAndIncrement()))); - iter2.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o, seqNoGenerator.getAndIncrement()))); + iter1.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o))); + iter2.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o))); // insert some duplicates - randomSubsetOf(allOps).forEach(op -> allOps.add(seqNoUpdater.apply(op, op.seqNo()))); + allOps.addAll(randomSubsetOf(allOps)); shuffle(allOps, random()); concurrentlyApplyOps(allOps, engine); @@ -1683,6 +1623,42 @@ public class InternalEngineTests extends EngineTestCase { assertVisibleCount(engine, totalExpectedOps); } + private void concurrentlyApplyOps(List ops, InternalEngine engine) throws InterruptedException { + Thread[] thread = new Thread[randomIntBetween(3, 5)]; + CountDownLatch startGun = new CountDownLatch(thread.length); + AtomicInteger offset = new AtomicInteger(-1); + for (int i = 0; i < thread.length; i++) { + thread[i] = new Thread(() -> { + startGun.countDown(); + try { + startGun.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + int docOffset; + while ((docOffset = offset.incrementAndGet()) < ops.size()) { + try { + final Engine.Operation op = ops.get(docOffset); + if (op instanceof Engine.Index) { + engine.index((Engine.Index) op); + } else { + engine.delete((Engine.Delete) op); + } + if ((docOffset + 1) % 4 == 0) { + engine.refresh("test"); + } + } catch (IOException e) { + throw new AssertionError(e); + } + } + }); + thread[i].start(); + } + for (int i = 0; i < thread.length; i++) { + thread[i].join(); + } + } + public void testInternalVersioningOnPrimary() throws IOException { final List ops = generateSingleDocHistory(false, VersionType.INTERNAL, 2, 2, 20, "1"); assertOpsOnPrimary(ops, Versions.NOT_FOUND, true, engine); @@ -1893,7 +1869,7 @@ public class InternalEngineTests extends EngineTestCase { final boolean deletedOnReplica = lastReplicaOp instanceof Engine.Delete; final long finalReplicaVersion = lastReplicaOp.version(); final long finalReplicaSeqNo = lastReplicaOp.seqNo(); - assertOpsOnReplica(replicaOps, replicaEngine, true, logger); + assertOpsOnReplica(replicaOps, replicaEngine, true); final int opsOnPrimary = assertOpsOnPrimary(primaryOps, finalReplicaVersion, deletedOnReplica, replicaEngine); final long currentSeqNo = getSequenceID(replicaEngine, new Engine.Get(false, false, "type", lastReplicaOp.uid().text(), lastReplicaOp.uid())).v1(); @@ -2698,16 +2674,14 @@ public class InternalEngineTests extends EngineTestCase { Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } - EngineConfig config = engine.config(); assertVisibleCount(engine, numDocs); engine.close(); - trimUnsafeCommits(config); - try (InternalEngine engine = new InternalEngine(config)) { - engine.skipTranslogRecovery(); - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(0L)); - } + trimUnsafeCommits(engine.config()); + engine = new InternalEngine(engine.config()); + engine.skipTranslogRecovery(); + try (Engine.Searcher searcher = engine.acquireSearcher("test")) { + TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); + assertThat(topDocs.totalHits, equalTo(0L)); } } @@ -2837,7 +2811,7 @@ public class InternalEngineTests extends EngineTestCase { new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm::get, tombstoneDocSupplier()); + new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm::get); try { InternalEngine internalEngine = new InternalEngine(brokenConfig); fail("translog belongs to a different engine"); @@ -2966,12 +2940,6 @@ public class InternalEngineTests extends EngineTestCase { } } - @Override - public long softUpdateDocument(Term term, Iterable doc, Field... softDeletes) throws IOException { - maybeThrowFailure(); - return super.softUpdateDocument(term, doc, softDeletes); - } - @Override public long deleteDocuments(Term... terms) throws IOException { maybeThrowFailure(); @@ -3172,10 +3140,10 @@ public class InternalEngineTests extends EngineTestCase { } public void testDoubleDeliveryReplicaAppendingOnly() throws IOException { - final Supplier doc = () -> testParsedDocument("1", null, testDocumentWithTextField(), + final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); - Engine.Index operation = appendOnlyReplica(doc.get(), false, 1, randomIntBetween(0, 5)); - Engine.Index retry = appendOnlyReplica(doc.get(), true, 1, randomIntBetween(0, 5)); + Engine.Index operation = appendOnlyReplica(doc, false, 1, randomIntBetween(0, 5)); + Engine.Index retry = appendOnlyReplica(doc, true, 1, randomIntBetween(0, 5)); // operations with a seq# equal or lower to the local checkpoint are not indexed to lucene // and the version lookup is skipped final boolean belowLckp = operation.seqNo() == 0 && retry.seqNo() == 0; @@ -3214,8 +3182,8 @@ public class InternalEngineTests extends EngineTestCase { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); assertEquals(1, topDocs.totalHits); } - operation = randomAppendOnly(doc.get(), false, 1); - retry = randomAppendOnly(doc.get(), true, 1); + operation = randomAppendOnly(doc, false, 1); + retry = randomAppendOnly(doc, true, 1); if (randomBoolean()) { Engine.IndexResult indexResult = engine.index(operation); assertNotNull(indexResult.getTranslogLocation()); @@ -3280,8 +3248,6 @@ public class InternalEngineTests extends EngineTestCase { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); assertEquals(1, topDocs.totalHits); } - List ops = readAllOperationsInLucene(engine, createMapperService("test")); - assertThat(ops.stream().map(o -> o.seqNo()).collect(Collectors.toList()), hasItem(20L)); } public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOException { @@ -3750,22 +3716,20 @@ public class InternalEngineTests extends EngineTestCase { final List operations = new ArrayList<>(); final int numberOfOperations = randomIntBetween(16, 32); + final Document document = testDocumentWithTextField(); final AtomicLong sequenceNumber = new AtomicLong(); final Engine.Operation.Origin origin = randomFrom(LOCAL_TRANSLOG_RECOVERY, PEER_RECOVERY, PRIMARY, REPLICA); final LongSupplier sequenceNumberSupplier = origin == PRIMARY ? () -> SequenceNumbers.UNASSIGNED_SEQ_NO : sequenceNumber::getAndIncrement; - final Supplier doc = () -> { - final Document document = testDocumentWithTextField(); - document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); - return testParsedDocument("1", null, document, B_1, null); - }; - final Term uid = newUid("1"); + document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); + final ParsedDocument doc = testParsedDocument("1", null, document, B_1, null); + final Term uid = newUid(doc); final BiFunction searcherFactory = engine::acquireSearcher; for (int i = 0; i < numberOfOperations; i++) { if (randomBoolean()) { final Engine.Index index = new Engine.Index( uid, - doc.get(), + doc, sequenceNumberSupplier.getAsLong(), 1, i, @@ -3841,9 +3805,7 @@ public class InternalEngineTests extends EngineTestCase { maxSeqNo, localCheckpoint); trimUnsafeCommits(engine.config()); - EngineConfig noopEngineConfig = copy(engine.config(), new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, - () -> new MatchAllDocsQuery(), engine.config().getMergePolicy())); - noOpEngine = new InternalEngine(noopEngineConfig, supplier) { + noOpEngine = new InternalEngine(engine.config(), supplier) { @Override protected long doGenerateSeqNoForOperation(Operation operation) { throw new UnsupportedOperationException(); @@ -3851,7 +3813,7 @@ public class InternalEngineTests extends EngineTestCase { }; noOpEngine.recoverFromTranslog(Long.MAX_VALUE); final int gapsFilled = noOpEngine.fillSeqNoGaps(primaryTerm.get()); - final String reason = "filling gaps"; + final String reason = randomAlphaOfLength(16); noOpEngine.noOp(new Engine.NoOp(maxSeqNo + 1, primaryTerm.get(), LOCAL_TRANSLOG_RECOVERY, System.nanoTime(), reason)); assertThat(noOpEngine.getLocalCheckpoint(), equalTo((long) (maxSeqNo + 1))); assertThat(noOpEngine.getTranslog().stats().getUncommittedOperations(), equalTo(gapsFilled)); @@ -3873,77 +3835,11 @@ public class InternalEngineTests extends EngineTestCase { assertThat(noOp.seqNo(), equalTo((long) (maxSeqNo + 2))); assertThat(noOp.primaryTerm(), equalTo(primaryTerm.get())); assertThat(noOp.reason(), equalTo(reason)); - if (engine.engineConfig.getIndexSettings().isSoftDeleteEnabled()) { - MapperService mapperService = createMapperService("test"); - List operationsFromLucene = readAllOperationsInLucene(noOpEngine, mapperService); - assertThat(operationsFromLucene, hasSize(maxSeqNo + 2 - localCheckpoint)); // fills n gap and 2 manual noop. - for (int i = 0; i < operationsFromLucene.size(); i++) { - assertThat(operationsFromLucene.get(i), equalTo(new Translog.NoOp(localCheckpoint + 1 + i, primaryTerm.get(), "filling gaps"))); - } - assertConsistentHistoryBetweenTranslogAndLuceneIndex(noOpEngine, mapperService); - } } finally { IOUtils.close(noOpEngine); } } - /** - * Verifies that a segment containing only no-ops can be used to look up _version and _seqno. - */ - public void testSegmentContainsOnlyNoOps() throws Exception { - Engine.NoOpResult noOpResult = engine.noOp(new Engine.NoOp(1, primaryTerm.get(), - randomFrom(Engine.Operation.Origin.values()), randomNonNegativeLong(), "test")); - assertThat(noOpResult.getFailure(), nullValue()); - engine.refresh("test"); - Engine.DeleteResult deleteResult = engine.delete(replicaDeleteForDoc("id", 1, 2, randomNonNegativeLong())); - assertThat(deleteResult.getFailure(), nullValue()); - engine.refresh("test"); - } - - /** - * A simple test to check that random combination of operations can coexist in segments and be lookup. - * This is needed as some fields in Lucene may not exist if a segment misses operation types and this code is to check for that. - * For example, a segment containing only no-ops does not have neither _uid or _version. - */ - public void testRandomOperations() throws Exception { - int numOps = between(10, 100); - for (int i = 0; i < numOps; i++) { - String id = Integer.toString(randomIntBetween(1, 10)); - ParsedDocument doc = createParsedDoc(id, null); - Engine.Operation.TYPE type = randomFrom(Engine.Operation.TYPE.values()); - switch (type) { - case INDEX: - Engine.IndexResult index = engine.index(replicaIndexForDoc(doc, between(1, 100), i, randomBoolean())); - assertThat(index.getFailure(), nullValue()); - break; - case DELETE: - Engine.DeleteResult delete = engine.delete(replicaDeleteForDoc(doc.id(), between(1, 100), i, randomNonNegativeLong())); - assertThat(delete.getFailure(), nullValue()); - break; - case NO_OP: - Engine.NoOpResult noOp = engine.noOp(new Engine.NoOp(i, primaryTerm.get(), - randomFrom(Engine.Operation.Origin.values()), randomNonNegativeLong(), "")); - assertThat(noOp.getFailure(), nullValue()); - break; - default: - throw new IllegalStateException("Invalid op [" + type + "]"); - } - if (randomBoolean()) { - engine.refresh("test"); - } - if (randomBoolean()) { - engine.flush(); - } - if (randomBoolean()) { - engine.forceMerge(randomBoolean(), between(1, 10), randomBoolean(), false, false); - } - } - if (engine.engineConfig.getIndexSettings().isSoftDeleteEnabled()) { - List operations = readAllOperationsInLucene(engine, createMapperService("test")); - assertThat(operations, hasSize(numOps)); - } - } - public void testMinGenerationForSeqNo() throws IOException, BrokenBarrierException, InterruptedException { engine.close(); final int numberOfTriplets = randomIntBetween(1, 32); @@ -4509,7 +4405,7 @@ public class InternalEngineTests extends EngineTestCase { globalCheckpoint.set(randomLongBetween(engine.getLocalCheckpoint(), Long.MAX_VALUE)); engine.syncTranslog(); assertThat(DirectoryReader.listCommits(store.directory()), contains(commits.get(commits.size() - 1))); - assertThat(engine.getTranslog().totalOperations(), equalTo(0)); + assertThat(engine.estimateTranslogOperationsFromMinSeq(0L), equalTo(0)); } } @@ -4872,154 +4768,6 @@ public class InternalEngineTests extends EngineTestCase { } } - public void testLuceneHistoryOnPrimary() throws Exception { - final List operations = generateSingleDocHistory(false, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "1"); - assertOperationHistoryInLucene(operations); - } - - public void testLuceneHistoryOnReplica() throws Exception { - final List operations = generateSingleDocHistory(true, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); - Randomness.shuffle(operations); - assertOperationHistoryInLucene(operations); - } - - private void assertOperationHistoryInLucene(List operations) throws IOException { - final MergePolicy keepSoftDeleteDocsMP = new SoftDeletesRetentionMergePolicy( - Lucene.SOFT_DELETES_FIELD, () -> new MatchAllDocsQuery(), engine.config().getMergePolicy()); - Settings.Builder settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); - final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); - Set expectedSeqNos = new HashSet<>(); - try (Store store = createStore(); - Engine engine = createEngine(config(indexSettings, store, createTempDir(), keepSoftDeleteDocsMP, null))) { - for (Engine.Operation op : operations) { - if (op instanceof Engine.Index) { - Engine.IndexResult indexResult = engine.index((Engine.Index) op); - assertThat(indexResult.getFailure(), nullValue()); - expectedSeqNos.add(indexResult.getSeqNo()); - } else { - Engine.DeleteResult deleteResult = engine.delete((Engine.Delete) op); - assertThat(deleteResult.getFailure(), nullValue()); - expectedSeqNos.add(deleteResult.getSeqNo()); - } - if (rarely()) { - engine.refresh("test"); - } - if (rarely()) { - engine.flush(); - } - if (rarely()) { - engine.forceMerge(true); - } - } - MapperService mapperService = createMapperService("test"); - List actualOps = readAllOperationsInLucene(engine, mapperService); - assertThat(actualOps.stream().map(o -> o.seqNo()).collect(Collectors.toList()), containsInAnyOrder(expectedSeqNos.toArray())); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - } - } - - public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException { - IOUtils.close(engine, store); - Settings.Builder settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); - final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - final List operations = generateSingleDocHistory(true, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); - Randomness.shuffle(operations); - Set existingSeqNos = new HashSet<>(); - store = createStore(); - engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get)); - assertThat(engine.getMinRetainedSeqNo(), equalTo(0L)); - long lastMinRetainedSeqNo = engine.getMinRetainedSeqNo(); - for (Engine.Operation op : operations) { - final Engine.Result result; - if (op instanceof Engine.Index) { - result = engine.index((Engine.Index) op); - } else { - result = engine.delete((Engine.Delete) op); - } - existingSeqNos.add(result.getSeqNo()); - if (randomBoolean()) { - globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getCheckpoint())); - } - if (rarely()) { - settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); - indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); - engine.onSettingsChanged(); - } - if (rarely()) { - engine.refresh("test"); - } - if (rarely()) { - engine.flush(true, true); - assertThat(Long.parseLong(engine.getLastCommittedSegmentInfos().userData.get(Engine.MIN_RETAINED_SEQNO)), - equalTo(engine.getMinRetainedSeqNo())); - } - if (rarely()) { - engine.forceMerge(randomBoolean()); - } - try (Closeable ignored = engine.acquireRetentionLockForPeerRecovery()) { - long minRetainSeqNos = engine.getMinRetainedSeqNo(); - assertThat(minRetainSeqNos, lessThanOrEqualTo(globalCheckpoint.get() + 1)); - Long[] expectedOps = existingSeqNos.stream().filter(seqno -> seqno >= minRetainSeqNos).toArray(Long[]::new); - Set actualOps = readAllOperationsInLucene(engine, createMapperService("test")).stream() - .map(Translog.Operation::seqNo).collect(Collectors.toSet()); - assertThat(actualOps, containsInAnyOrder(expectedOps)); - } - try (Engine.IndexCommitRef commitRef = engine.acquireSafeIndexCommit()) { - IndexCommit safeCommit = commitRef.getIndexCommit(); - if (safeCommit.getUserData().containsKey(Engine.MIN_RETAINED_SEQNO)) { - lastMinRetainedSeqNo = Long.parseLong(safeCommit.getUserData().get(Engine.MIN_RETAINED_SEQNO)); - } - } - } - if (randomBoolean()) { - engine.close(); - } else { - engine.flushAndClose(); - } - trimUnsafeCommits(engine.config()); - try (InternalEngine recoveringEngine = new InternalEngine(engine.config())) { - assertThat(recoveringEngine.getMinRetainedSeqNo(), equalTo(lastMinRetainedSeqNo)); - } - } - - public void testLastRefreshCheckpoint() throws Exception { - AtomicBoolean done = new AtomicBoolean(); - Thread[] refreshThreads = new Thread[between(1, 8)]; - CountDownLatch latch = new CountDownLatch(refreshThreads.length); - for (int i = 0; i < refreshThreads.length; i++) { - latch.countDown(); - refreshThreads[i] = new Thread(() -> { - while (done.get() == false) { - long checkPointBeforeRefresh = engine.getLocalCheckpoint(); - engine.refresh("test", randomFrom(Engine.SearcherScope.values())); - assertThat(engine.lastRefreshedCheckpoint(), greaterThanOrEqualTo(checkPointBeforeRefresh)); - } - }); - refreshThreads[i].start(); - } - latch.await(); - List ops = generateSingleDocHistory(true, VersionType.EXTERNAL, 1, 10, 1000, "1"); - concurrentlyApplyOps(ops, engine); - done.set(true); - for (Thread thread : refreshThreads) { - thread.join(); - } - engine.refresh("test"); - assertThat(engine.lastRefreshedCheckpoint(), equalTo(engine.getLocalCheckpoint())); - } - private static void trimUnsafeCommits(EngineConfig config) throws IOException { final Store store = config.getStore(); final TranslogConfig translogConfig = config.getTranslogConfig(); diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java deleted file mode 100644 index 2d097366a27..00000000000 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ /dev/null @@ -1,289 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.engine; - -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.translog.SnapshotMatchers; -import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.test.IndexSettingsModule; -import org.junit.Before; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; - -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; - -public class LuceneChangesSnapshotTests extends EngineTestCase { - private MapperService mapperService; - - @Before - public void createMapper() throws Exception { - mapperService = createMapperService("test"); - } - - @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", mapperService, fromSeqNo, toSeqNo, true)) { - 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", mapperService, fromSeqNo, toSeqNo, false)) { - 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.type(), doc.id(), newUid(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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, false)) { - 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { - 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, false)) { - 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { - 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { - 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", mapperService, fromSeqNo, toSeqNo, randomBoolean())) { - assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); - } - } - - public void testDedupByPrimaryTerm() throws Exception { - Map latestOperations = new HashMap<>(); - List terms = Arrays.asList(between(1, 1000), between(1000, 2000)); - int totalOps = 0; - for (long term : terms) { - final List ops = generateSingleDocHistory(true, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE), term, 2, 20, "1"); - primaryTerm.set(Math.max(primaryTerm.get(), term)); - engine.rollTranslogGeneration(); - for (Engine.Operation op : ops) { - // We need to simulate a rollback here as only ops after local checkpoint get into the engine - if (op.seqNo() <= engine.getLocalCheckpointTracker().getCheckpoint()) { - engine.getLocalCheckpointTracker().resetCheckpoint(randomLongBetween(-1, op.seqNo() - 1)); - engine.rollTranslogGeneration(); - } - if (op instanceof Engine.Index) { - engine.index((Engine.Index) op); - } else if (op instanceof Engine.Delete) { - engine.delete((Engine.Delete) op); - } - latestOperations.put(op.seqNo(), op.primaryTerm()); - if (rarely()) { - engine.refresh("test"); - } - if (rarely()) { - engine.flush(); - } - totalOps++; - } - } - long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); - try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", mapperService, 0, maxSeqNo, false)) { - Translog.Operation op; - while ((op = snapshot.next()) != null) { - assertThat(op.toString(), op.primaryTerm(), equalTo(latestOperations.get(op.seqNo()))); - } - assertThat(snapshot.skippedOperations(), equalTo(totalOps - latestOperations.size())); - } - } - - 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 = scaledRandomIntBetween(1, 1000); - for (int i = 0; i < numOps; i++) { - String id = Integer.toString(randomIntBetween(1, 10)); - 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.type(), doc.id(), newUid(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(); - concurrentlyApplyOps(operations, engine); - assertThat(engine.getLocalCheckpointTracker().getCheckpoint(), equalTo(operations.size() - 1L)); - isDone.set(true); - for (Follower follower : followers) { - follower.join(); - } - } - - class Follower extends Thread { - private final Engine leader; - private final TranslogHandler translogHandler; - private final AtomicBoolean isDone; - private final CountDownLatch readLatch; - - Follower(Engine leader, AtomicBoolean isDone, CountDownLatch readLatch) { - this.leader = leader; - this.isDone = isDone; - this.readLatch = readLatch; - this.translogHandler = new TranslogHandler(xContentRegistry(), IndexSettingsModule.newIndexSettings(shardId.getIndexName(), - engine.engineConfig.getIndexSettings().getSettings())); - } - - void pullOperations(Engine follower) throws IOException { - long leaderCheckpoint = leader.getLocalCheckpoint(); - long followerCheckpoint = follower.getLocalCheckpoint(); - 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", mapperService, fromSeqNo, toSeqNo, true)) { - translogHandler.run(follower, snapshot); - } - } - } - - @Override - public void run() { - try (Store store = createStore(); - InternalEngine follower = createEngine(store, createTempDir())) { - readLatch.countDown(); - readLatch.await(); - while (isDone.get() == false || - follower.getLocalCheckpointTracker().getCheckpoint() < leader.getLocalCheckpoint()) { - pullOperations(follower); - } - assertConsistentHistoryBetweenTranslogAndLuceneIndex(follower, mapperService); - assertThat(getDocIds(follower, true), equalTo(getDocIds(leader, true))); - } 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.error("Reading [{}]", op); - assert operations.stream().allMatch(o -> o.seqNo() < newOp.seqNo()) : "Operations [" + operations + "], op [" + op + "]"; - operations.add(newOp); - } - return operations; - } -} diff --git a/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java deleted file mode 100644 index c46b47b87d0..00000000000 --- a/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.engine; - -import org.apache.lucene.document.Document; -import org.apache.lucene.document.Field; -import org.apache.lucene.document.NumericDocValuesField; -import org.apache.lucene.document.StoredField; -import org.apache.lucene.document.StringField; -import org.apache.lucene.index.CodecReader; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.IndexableField; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.MergePolicy; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.index.SegmentCommitInfo; -import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.StandardDirectoryReader; -import org.apache.lucene.index.Term; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.MatchNoDocsQuery; -import org.apache.lucene.search.TermQuery; -import org.apache.lucene.store.Directory; -import org.apache.lucene.util.InfoStream; -import org.apache.lucene.util.NullInfoStream; -import org.elasticsearch.test.ESTestCase; - -import java.io.IOException; -import java.util.Collections; -import java.util.Set; -import java.util.stream.Collectors; - -public class RecoverySourcePruneMergePolicyTests extends ESTestCase { - - public void testPruneAll() throws IOException { - try (Directory dir = newDirectory()) { - IndexWriterConfig iwc = newIndexWriterConfig(); - RecoverySourcePruneMergePolicy mp = new RecoverySourcePruneMergePolicy("extra_source", MatchNoDocsQuery::new, - newLogMergePolicy()); - iwc.setMergePolicy(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("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); - assertEquals(1, document.getFields().size()); - assertEquals("source", document.getFields().get(0).name()); - } - 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 && reader instanceof StandardDirectoryReader) { - CodecReader codecReader = (CodecReader) leafReader; - StandardDirectoryReader sdr = (StandardDirectoryReader) reader; - 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 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)); - } - } - } - } - } - - - public void testPruneSome() throws IOException { - try (Directory dir = newDirectory()) { - IndexWriterConfig iwc = newIndexWriterConfig(); - iwc.setMergePolicy(new RecoverySourcePruneMergePolicy("extra_source", - () -> 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 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() == 3) { - assertTrue(collect.contains("extra_source")); - assertEquals("true", document.getField("even").stringValue()); - assertEquals(i, extra_source.nextDoc()); - } else { - assertEquals(2, document.getFields().size()); - } - } - assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); - } - } - } - } -} diff --git a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java deleted file mode 100644 index f3590100382..00000000000 --- a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.engine; - -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.index.seqno.SequenceNumbers; -import org.elasticsearch.test.ESTestCase; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; - -import static org.hamcrest.Matchers.equalTo; - -public class SoftDeletesPolicyTests extends ESTestCase { - /** - * Makes sure we won't advance the retained seq# if the retention lock is held - */ - public void testSoftDeletesRetentionLock() { - long retainedOps = between(0, 10000); - AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - long safeCommitCheckpoint = globalCheckpoint.get(); - SoftDeletesPolicy policy = new SoftDeletesPolicy(globalCheckpoint::get, between(1, 10000), retainedOps); - long minRetainedSeqNo = policy.getMinRetainedSeqNo(); - List locks = new ArrayList<>(); - int iters = scaledRandomIntBetween(10, 1000); - for (int i = 0; i < iters; i++) { - if (randomBoolean()) { - locks.add(policy.acquireRetentionLock()); - } - // Advances the global checkpoint and the local checkpoint of a safe commit - globalCheckpoint.addAndGet(between(0, 1000)); - safeCommitCheckpoint = randomLongBetween(safeCommitCheckpoint, globalCheckpoint.get()); - policy.setLocalCheckpointOfSafeCommit(safeCommitCheckpoint); - if (rarely()) { - retainedOps = between(0, 10000); - policy.setRetentionOperations(retainedOps); - } - // Release some locks - List releasingLocks = randomSubsetOf(locks); - locks.removeAll(releasingLocks); - releasingLocks.forEach(Releasable::close); - - // We only expose the seqno to the merge policy if the retention lock is not held. - policy.getRetentionQuery(); - if (locks.isEmpty()) { - long retainedSeqNo = Math.min(safeCommitCheckpoint, globalCheckpoint.get() - retainedOps) + 1; - minRetainedSeqNo = Math.max(minRetainedSeqNo, retainedSeqNo); - } - assertThat(policy.getMinRetainedSeqNo(), equalTo(minRetainedSeqNo)); - } - - locks.forEach(Releasable::close); - long retainedSeqNo = Math.min(safeCommitCheckpoint, globalCheckpoint.get() - retainedOps) + 1; - minRetainedSeqNo = Math.max(minRetainedSeqNo, retainedSeqNo); - assertThat(policy.getMinRetainedSeqNo(), equalTo(minRetainedSeqNo)); - } -} diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java index 5a46b9a889f..76ca6aa7ea8 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java @@ -31,7 +31,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -312,18 +311,15 @@ public class DocumentParserTests extends ESSingleNodeTestCase { // creates an object mapper, which is about 100x harder than it should be.... ObjectMapper createObjectMapper(MapperService mapperService, String name) throws Exception { - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); - ParseContext context = new ParseContext.InternalParseContext(settings, + ParseContext context = new ParseContext.InternalParseContext( + Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(), mapperService.documentMapperParser(), mapperService.documentMapper("type"), null, null); String[] nameParts = name.split("\\."); for (int i = 0; i < nameParts.length - 1; ++i) { context.path().add(nameParts[i]); } Mapper.Builder builder = new ObjectMapper.Builder(nameParts[nameParts.length - 1]).enabled(true); - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); return (ObjectMapper)builder.build(builderContext); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java index b11e4876f9e..cb2ed785699 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.BooleanFieldMapper.BooleanFieldType; import org.elasticsearch.index.mapper.DateFieldMapper.DateFieldType; import org.elasticsearch.index.mapper.NumberFieldMapper.NumberFieldType; @@ -216,10 +215,7 @@ public class DynamicMappingTests extends ESSingleNodeTestCase { } private Mapper parse(DocumentMapper mapper, DocumentMapperParser parser, XContentBuilder builder) throws Exception { - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); SourceToParse source = SourceToParse.source("test", mapper.type(), "some_id", BytesReference.bytes(builder), builder.contentType()); try (XContentParser xContentParser = createParser(JsonXContent.jsonXContent, source.source())) { ParseContext.InternalParseContext ctx = new ParseContext.InternalParseContext(settings, parser, mapper, source, xContentParser); 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 fba71dd1e52..1d1e423afc1 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.index.replication; -import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; @@ -42,7 +41,6 @@ import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.engine.InternalEngineTests; import org.elasticsearch.index.engine.SegmentsStats; import org.elasticsearch.index.engine.VersionConflictEngineException; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -142,9 +140,7 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase } public void testInheritMaxValidAutoIDTimestampOnRecovery() throws Exception { - //TODO: Enables this test with soft-deletes once we have timestamp - Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); - try (ReplicationGroup shards = createGroup(0, settings)) { + try (ReplicationGroup shards = createGroup(0)) { shards.startAll(); final IndexRequest indexRequest = new IndexRequest(index.getName(), "type").source("{}", XContentType.JSON); indexRequest.onRetry(); // force an update of the timestamp @@ -350,13 +346,7 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase final AtomicBoolean throwAfterIndexedOneDoc = new AtomicBoolean(); // need one document to trigger delete in IW. @Override public long addDocument(Iterable doc) throws IOException { - boolean isTombstone = false; - for (IndexableField field : doc) { - if (SeqNoFieldMapper.TOMBSTONE_NAME.equals(field.name())) { - isTombstone = true; - } - } - if (isTombstone == false && throwAfterIndexedOneDoc.getAndSet(true)) { + if (throwAfterIndexedOneDoc.getAndSet(true)) { throw indexException; } else { return super.addDocument(doc); @@ -366,10 +356,6 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase public long deleteDocuments(Term... terms) throws IOException { throw deleteException; } - @Override - public long softUpdateDocument(Term term, Iterable doc, Field...fields) throws IOException { - throw deleteException; // a delete uses softUpdateDocument API if soft-deletes enabled - } }, null, null, config); try (ReplicationGroup shards = new ReplicationGroup(buildIndexMetaData(0)) { @Override @@ -404,9 +390,6 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase try (Translog.Snapshot snapshot = getTranslog(shard).newSnapshot()) { assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); } - try (Translog.Snapshot snapshot = shard.getHistoryOperations("test", 0)) { - assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); - } } // unlike previous failures, these two failures replicated directly from the replication channel. indexResp = shards.index(new IndexRequest(index.getName(), "type", "any").source("{}", XContentType.JSON)); @@ -421,9 +404,6 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase try (Translog.Snapshot snapshot = getTranslog(shard).newSnapshot()) { assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); } - try (Translog.Snapshot snapshot = shard.getHistoryOperations("test", 0)) { - assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); - } } shards.assertAllEqual(1); } @@ -521,9 +501,8 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase recoverReplica(replica3, replica2, true); try (Translog.Snapshot snapshot = getTranslog(replica3).newSnapshot()) { assertThat(snapshot.totalOperations(), equalTo(initDocs + 1)); - final List expectedOps = new ArrayList<>(initOperations); - expectedOps.add(op2); - assertThat(snapshot, containsOperationsInAnyOrder(expectedOps)); + assertThat(snapshot.next(), equalTo(op2)); + assertThat("Remaining of snapshot should contain init operations", snapshot, containsOperationsInAnyOrder(initOperations)); assertThat("Peer-recovery should not send overridden operations", snapshot.skippedOperations(), equalTo(0)); } // TODO: We should assert the content of shards in the ReplicationGroup. diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index 28122665e9b..2d198c32ba7 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -98,8 +98,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC } public void testRecoveryOfDisconnectedReplica() throws Exception { - Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); - try (ReplicationGroup shards = createGroup(1, settings)) { + try (ReplicationGroup shards = createGroup(1)) { shards.startAll(); int docs = shards.indexDocs(randomInt(50)); shards.flush(); @@ -267,7 +266,6 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC builder.settings(Settings.builder().put(newPrimary.indexSettings().getSettings()) .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) ); newPrimary.indexSettings().updateIndexMetaData(builder.build()); newPrimary.onSettingsChanged(); @@ -277,12 +275,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC shards.syncGlobalCheckpoint(); assertThat(newPrimary.getLastSyncedGlobalCheckpoint(), equalTo(newPrimary.seqNoStats().getMaxSeqNo())); }); - newPrimary.flush(new FlushRequest().force(true)); - if (replica.indexSettings().isSoftDeleteEnabled()) { - // We need an extra flush to advance the min_retained_seqno on the new primary so ops-based won't happen. - // The min_retained_seqno only advances when a merge asks for the retention query. - newPrimary.flush(new FlushRequest().force(true)); - } + newPrimary.flush(new FlushRequest()); uncommittedOpsOnPrimary = shards.indexDocs(randomIntBetween(0, 10)); totalDocs += uncommittedOpsOnPrimary; } 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 50f95bf4d47..2228e1b017f 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -22,7 +22,6 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.TermQuery; @@ -31,7 +30,6 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; -import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Constants; import org.elasticsearch.Assertions; import org.elasticsearch.Version; @@ -91,13 +89,8 @@ import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; @@ -167,7 +160,6 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasKey; -import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.hasToString; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.lessThan; @@ -245,8 +237,7 @@ public class IndexShardTests extends IndexShardTestCase { assertNotNull(shardPath); // fail shard shard.failShard("test shard fail", new CorruptIndexException("", "")); - shard.close("do not assert history", false); - shard.store().close(); + closeShards(shard); // check state file still exists ShardStateMetaData shardStateMetaData = load(logger, shardPath.getShardStatePath()); assertEquals(shardStateMetaData, getShardStateMetadata(shard)); @@ -2403,8 +2394,7 @@ public class IndexShardTests extends IndexShardTestCase { public void testDocStats() throws IOException, InterruptedException { IndexShard indexShard = null; try { - indexShard = newStartedShard( - Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0).build()); + indexShard = newStartedShard(); final long numDocs = randomIntBetween(2, 32); // at least two documents so we have docs to delete final long numDocsToDelete = randomLongBetween(1, numDocs); for (int i = 0; i < numDocs; i++) { @@ -2434,16 +2424,7 @@ public class IndexShardTests extends IndexShardTestCase { deleteDoc(indexShard, "_doc", id); indexDoc(indexShard, "_doc", id); } - // Need to update and sync the global checkpoint as the soft-deletes retention MergePolicy depends on it. - if (indexShard.indexSettings.isSoftDeleteEnabled()) { - if (indexShard.routingEntry().primary()) { - indexShard.updateGlobalCheckpointForShard(indexShard.routingEntry().allocationId().getId(), - indexShard.getLocalCheckpoint()); - } else { - indexShard.updateGlobalCheckpointOnReplica(indexShard.getLocalCheckpoint(), "test"); - } - indexShard.sync(); - } + // flush the buffered deletes final FlushRequest flushRequest = new FlushRequest(); flushRequest.force(false); @@ -2981,7 +2962,6 @@ public class IndexShardTests extends IndexShardTestCase { assertThat(breaker.getUsed(), greaterThan(preRefreshBytes)); indexDoc(primary, "_doc", "4", "{\"foo\": \"potato\"}"); - indexDoc(primary, "_doc", "5", "{\"foo\": \"potato\"}"); // Forces a refresh with the INTERNAL scope ((InternalEngine) primary.getEngine()).writeIndexingBuffer(); @@ -2993,13 +2973,6 @@ public class IndexShardTests extends IndexShardTestCase { // Deleting a doc causes its memory to be freed from the breaker deleteDoc(primary, "_doc", "0"); - // Here we are testing that a fully deleted segment should be dropped and its memory usage is freed. - // In order to instruct the merge policy not to keep a fully deleted segment, - // we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything. - if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { - primary.sync(); - flushShard(primary); - } primary.refresh("force refresh"); ss = primary.segmentStats(randomBoolean()); @@ -3091,7 +3064,6 @@ public class IndexShardTests extends IndexShardTestCase { // Close remaining searchers IOUtils.close(searchers); - primary.refresh("test"); SegmentsStats ss = primary.segmentStats(randomBoolean()); CircuitBreaker breaker = primary.circuitBreakerService.getBreaker(CircuitBreaker.ACCOUNTING); @@ -3209,28 +3181,4 @@ public class IndexShardTests extends IndexShardTestCase { } - public void testSupplyTombstoneDoc() throws Exception { - IndexShard shard = newStartedShard(); - String id = randomRealisticUnicodeOfLengthBetween(1, 10); - ParsedDocument deleteTombstone = shard.getEngine().config().getTombstoneDocSupplier().newDeleteTombstoneDoc("doc", id); - assertThat(deleteTombstone.docs(), hasSize(1)); - ParseContext.Document deleteDoc = deleteTombstone.docs().get(0); - assertThat(deleteDoc.getFields().stream().map(IndexableField::name).collect(Collectors.toList()), - containsInAnyOrder(IdFieldMapper.NAME, VersionFieldMapper.NAME, - SeqNoFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME)); - assertThat(deleteDoc.getField(IdFieldMapper.NAME).binaryValue(), equalTo(Uid.encodeId(id))); - assertThat(deleteDoc.getField(SeqNoFieldMapper.TOMBSTONE_NAME).numericValue().longValue(), equalTo(1L)); - - final String reason = randomUnicodeOfLength(200); - ParsedDocument noopTombstone = shard.getEngine().config().getTombstoneDocSupplier().newNoopTombstoneDoc(reason); - assertThat(noopTombstone.docs(), hasSize(1)); - ParseContext.Document noopDoc = noopTombstone.docs().get(0); - assertThat(noopDoc.getFields().stream().map(IndexableField::name).collect(Collectors.toList()), - containsInAnyOrder(VersionFieldMapper.NAME, SourceFieldMapper.NAME, SeqNoFieldMapper.TOMBSTONE_NAME, - SeqNoFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME)); - assertThat(noopDoc.getField(SeqNoFieldMapper.TOMBSTONE_NAME).numericValue().longValue(), equalTo(1L)); - assertThat(noopDoc.getField(SourceFieldMapper.NAME).binaryValue(), equalTo(new BytesRef(reason))); - - closeShards(shard); - } } diff --git a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java index 29b16ca28f4..ae2cc84e487 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java @@ -106,22 +106,17 @@ public class PrimaryReplicaSyncerTests extends IndexShardTestCase { .isPresent(), is(false)); } + + assertEquals(globalCheckPoint == numDocs - 1 ? 0 : numDocs, resyncTask.getTotalOperations()); if (syncNeeded && globalCheckPoint < numDocs - 1) { - if (shard.indexSettings.isSoftDeleteEnabled()) { - assertThat(resyncTask.getSkippedOperations(), equalTo(0)); - assertThat(resyncTask.getResyncedOperations(), equalTo(resyncTask.getTotalOperations())); - assertThat(resyncTask.getTotalOperations(), equalTo(Math.toIntExact(numDocs - 1 - globalCheckPoint))); - } else { - int skippedOps = Math.toIntExact(globalCheckPoint + 1); // everything up to global checkpoint included - assertThat(resyncTask.getSkippedOperations(), equalTo(skippedOps)); - assertThat(resyncTask.getResyncedOperations(), equalTo(numDocs - skippedOps)); - assertThat(resyncTask.getTotalOperations(), equalTo(globalCheckPoint == numDocs - 1 ? 0 : numDocs)); - } + long skippedOps = globalCheckPoint + 1; // everything up to global checkpoint included + assertEquals(skippedOps, resyncTask.getSkippedOperations()); + assertEquals(numDocs - skippedOps, resyncTask.getResyncedOperations()); } else { - assertThat(resyncTask.getSkippedOperations(), equalTo(0)); - assertThat(resyncTask.getResyncedOperations(), equalTo(0)); - assertThat(resyncTask.getTotalOperations(), equalTo(0)); + assertEquals(0, resyncTask.getSkippedOperations()); + assertEquals(0, resyncTask.getResyncedOperations()); } + closeShards(shard); } 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 b93f170174c..774b272121a 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -42,7 +42,6 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; -import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; import org.elasticsearch.index.mapper.IdFieldMapper; @@ -131,8 +130,7 @@ public class RefreshListenersTests extends ESTestCase { indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), Collections.singletonList(listeners), Collections.emptyList(), null, - (e, s) -> 0, new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, () -> primaryTerm, - EngineTestCase.tombstoneDocSupplier()); + (e, s) -> 0, new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, () -> primaryTerm); engine = new InternalEngine(config); engine.recoverFromTranslog(Long.MAX_VALUE); listeners.setCurrentRefreshLocationSupplier(engine::getTranslogLastWriteLocation); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index 81afab4bb8f..89a8813e3e0 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -67,7 +67,6 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; -import org.junit.After; import java.io.IOException; import java.util.ArrayList; @@ -111,11 +110,6 @@ public class IndexRecoveryIT extends ESIntegTestCase { RecoverySettingsChunkSizePlugin.class); } - @After - public void assertConsistentHistoryInLuceneIndex() throws Exception { - internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); - } - private void assertRecoveryStateWithoutStage(RecoveryState state, int shardId, RecoverySource recoverySource, boolean primary, String sourceNode, String targetNode) { assertThat(state.getShardId().getId(), equalTo(shardId)); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java index b6f5a7b6451..4b1419375e6 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -25,7 +25,6 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.NoMergePolicy; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; @@ -92,7 +91,6 @@ public class PeerRecoveryTargetServiceTests extends IndexShardTestCase { replica.close("test", false); final List commits = DirectoryReader.listCommits(replica.store().directory()); IndexWriterConfig iwc = new IndexWriterConfig(null) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) .setMergePolicy(NoMergePolicy.INSTANCE) .setOpenMode(IndexWriterConfig.OpenMode.APPEND); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 0351111c305..f0644b029c3 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -411,6 +411,12 @@ public class RecoverySourceHandlerTests extends ESTestCase { recoverySettings.getChunkSize().bytesAsInt(), Settings.EMPTY) { + + @Override + boolean isTranslogReadyForSequenceNumberBasedRecovery() throws IOException { + return randomBoolean(); + } + @Override public void phase1(final IndexCommit snapshot, final Supplier translogOps) { phase1Called.set(true); 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 45535e19672..5547a629ab2 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.SourceToParse; @@ -64,13 +63,13 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { int docs = shards.indexDocs(10); getTranslog(shards.getPrimary()).rollGeneration(); shards.flush(); - int moreDocs = shards.indexDocs(randomInt(10)); + if (randomBoolean()) { + docs += shards.indexDocs(10); + } shards.addReplica(); shards.startAll(); final IndexShard replica = shards.getReplicas().get(0); - boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); - assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? moreDocs : docs + moreDocs)); - shards.assertAllEqual(docs + moreDocs); + assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(docs)); } } @@ -102,12 +101,12 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { // rolling/flushing is async assertBusy(() -> { assertThat(replica.getLastSyncedGlobalCheckpoint(), equalTo(19L)); - assertThat(getTranslog(replica).totalOperations(), equalTo(0)); + assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(0)); }); } } - public void testRecoveryWithOutOfOrderDeleteWithTranslog() throws Exception { + public void testRecoveryWithOutOfOrderDelete() throws Exception { /* * The flow of this test: * - delete #1 @@ -119,8 +118,7 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { * - index #5 * - If flush and the translog retention disabled, delete #1 will be removed while index #0 is still retained and replayed. */ - Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); - try (ReplicationGroup shards = createGroup(1, settings)) { + try (ReplicationGroup shards = createGroup(1)) { shards.startAll(); // create out of order delete and index op on replica final IndexShard orgReplica = shards.getReplicas().get(0); @@ -172,63 +170,7 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(newReplica); shards.assertAllEqual(3); - assertThat(getTranslog(newReplica).totalOperations(), equalTo(translogOps)); - } - } - - public void testRecoveryWithOutOfOrderDeleteWithSoftDeletes() throws Exception { - Settings settings = Settings.builder() - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 10) - // If soft-deletes is enabled, delete#1 will be reclaimed because its segment (segment_1) is fully deleted - // index#0 will be retained if merge is disabled; otherwise it will be reclaimed because gcp=3 and retained_ops=0 - .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false).build(); - try (ReplicationGroup shards = createGroup(1, settings)) { - shards.startAll(); - // create out of order delete and index op on replica - final IndexShard orgReplica = shards.getReplicas().get(0); - final String indexName = orgReplica.shardId().getIndexName(); - - // delete #1 - orgReplica.applyDeleteOperationOnReplica(1, 2, "type", "id"); - orgReplica.flush(new FlushRequest().force(true)); // isolate delete#1 in its own translog generation and lucene segment - // index #0 - orgReplica.applyIndexOperationOnReplica(0, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, - SourceToParse.source(indexName, "type", "id", new BytesArray("{}"), XContentType.JSON)); - // index #3 - orgReplica.applyIndexOperationOnReplica(3, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, - SourceToParse.source(indexName, "type", "id-3", new BytesArray("{}"), XContentType.JSON)); - // Flushing a new commit with local checkpoint=1 allows to delete the translog gen #1. - orgReplica.flush(new FlushRequest().force(true).waitIfOngoing(true)); - // index #2 - orgReplica.applyIndexOperationOnReplica(2, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, - SourceToParse.source(indexName, "type", "id-2", new BytesArray("{}"), XContentType.JSON)); - orgReplica.updateGlobalCheckpointOnReplica(3L, "test"); - // index #5 -> force NoOp #4. - orgReplica.applyIndexOperationOnReplica(5, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, - SourceToParse.source(indexName, "type", "id-5", new BytesArray("{}"), XContentType.JSON)); - - if (randomBoolean()) { - if (randomBoolean()) { - logger.info("--> flushing shard (translog/soft-deletes will be trimmed)"); - IndexMetaData.Builder builder = IndexMetaData.builder(orgReplica.indexSettings().getIndexMetaData()); - builder.settings(Settings.builder().put(orgReplica.indexSettings().getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0)); - orgReplica.indexSettings().updateIndexMetaData(builder.build()); - orgReplica.onSettingsChanged(); - } - flushShard(orgReplica); - } - - final IndexShard orgPrimary = shards.getPrimary(); - shards.promoteReplicaToPrimary(orgReplica).get(); // wait for primary/replica sync to make sure seq# gap is closed. - - IndexShard newReplica = shards.addReplicaWithExistingPath(orgPrimary.shardPath(), orgPrimary.routingEntry().currentNodeId()); - shards.recoverReplica(newReplica); - shards.assertAllEqual(3); - try (Translog.Snapshot snapshot = newReplica.getHistoryOperations("test", 0)) { - assertThat(snapshot, SnapshotMatchers.size(6)); - } + assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(translogOps)); } } @@ -280,8 +222,7 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(newReplica); // file based recovery should be made assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty())); - boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); - assertThat(getTranslog(newReplica).totalOperations(), equalTo(softDeletesEnabled ? nonFlushedDocs : numDocs)); + assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); // history uuid was restored assertThat(newReplica.getHistoryUUID(), equalTo(historyUUID)); @@ -385,8 +326,7 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(replica); // Make sure the flushing will eventually be completed (eg. `shouldPeriodicallyFlush` is false) assertBusy(() -> assertThat(getEngine(replica).shouldPeriodicallyFlush(), equalTo(false))); - boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); - assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? 0 : numDocs)); + assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); shards.assertAllEqual(numDocs); } } diff --git a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java index ce162b9600c..fa591411bba 100644 --- a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java @@ -43,7 +43,6 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexModule; -import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.MergeSchedulerConfig; @@ -51,7 +50,6 @@ import org.elasticsearch.index.VersionType; import org.elasticsearch.index.cache.query.QueryCacheStats; import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesQueryCache; import org.elasticsearch.indices.IndicesRequestCache; @@ -71,7 +69,6 @@ import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.Random; -import java.util.Set; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; @@ -118,7 +115,6 @@ public class IndexStatsIT extends ESIntegTestCase { return Settings.builder().put(super.indexSettings()) .put(IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.getKey(), true) .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) .build(); } @@ -1010,15 +1006,10 @@ public class IndexStatsIT extends ESIntegTestCase { @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/32506") public void testFilterCacheStats() throws Exception { - Settings settings = Settings.builder().put(indexSettings()).put("number_of_replicas", 0).build(); - assertAcked(prepareCreate("index").setSettings(settings).get()); - indexRandom(false, true, + assertAcked(prepareCreate("index").setSettings(Settings.builder().put(indexSettings()).put("number_of_replicas", 0).build()).get()); + indexRandom(true, client().prepareIndex("index", "type", "1").setSource("foo", "bar"), client().prepareIndex("index", "type", "2").setSource("foo", "baz")); - if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { - persistGlobalCheckpoint("index"); // Need to persist the global checkpoint for the soft-deletes retention MP. - } - refresh(); ensureGreen(); IndicesStatsResponse response = client().admin().indices().prepareStats("index").setQueryCache(true).get(); @@ -1049,13 +1040,6 @@ public class IndexStatsIT extends ESIntegTestCase { assertEquals(DocWriteResponse.Result.DELETED, client().prepareDelete("index", "type", "1").get().getResult()); assertEquals(DocWriteResponse.Result.DELETED, client().prepareDelete("index", "type", "2").get().getResult()); - // Here we are testing that a fully deleted segment should be dropped and its cached is evicted. - // In order to instruct the merge policy not to keep a fully deleted segment, - // we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything. - if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { - persistGlobalCheckpoint("index"); - flush("index"); - } refresh(); response = client().admin().indices().prepareStats("index").setQueryCache(true).get(); assertCumulativeQueryCacheStats(response); @@ -1189,21 +1173,4 @@ public class IndexStatsIT extends ESIntegTestCase { assertThat(executionFailures.get(), emptyCollectionOf(Exception.class)); } - - /** - * Persist the global checkpoint on all shards of the given index into disk. - * This makes sure that the persisted global checkpoint on those shards will equal to the in-memory value. - */ - private void persistGlobalCheckpoint(String index) throws Exception { - final Set nodes = internalCluster().nodesInclude(index); - for (String node : nodes) { - final IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node); - for (IndexService indexService : indexServices) { - for (IndexShard indexShard : indexService) { - indexShard.sync(); - assertThat(indexShard.getLastSyncedGlobalCheckpoint(), equalTo(indexShard.getGlobalCheckpoint())); - } - } - } - } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java b/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java index c25cad61e07..23c56688e00 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -27,7 +27,6 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.snapshots.mockstore.MockRepository; import org.elasticsearch.test.ESIntegTestCase; -import org.junit.After; import java.io.IOException; import java.nio.file.FileVisitResult; @@ -59,11 +58,6 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase { return Arrays.asList(MockRepository.Plugin.class); } - @After - public void assertConsistentHistoryInLuceneIndex() throws Exception { - internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); - } - public static long getFailureCount(String repository) { long failureCount = 0; for (RepositoriesService repositoriesService : diff --git a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index 632a1ecbee1..1230d594b98 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -122,7 +122,6 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF import static org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.IndexSettings.INDEX_REFRESH_INTERVAL_SETTING; -import static org.elasticsearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; import static org.elasticsearch.index.query.QueryBuilders.matchQuery; import static org.elasticsearch.index.shard.IndexShardTests.getEngineFromShard; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -2049,9 +2048,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES))); // only one shard - final Settings indexSettings = Settings.builder() - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).build(); - assertAcked(prepareCreate("test").setSettings(indexSettings)); + assertAcked(prepareCreate("test").setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1))); ensureGreen(); logger.info("--> indexing"); @@ -2097,13 +2094,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas SnapshotStatus snapshotStatus = client.admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("test-2").get().getSnapshots().get(0); List shards = snapshotStatus.getShards(); for (SnapshotIndexShardStatus status : shards) { - // we flush before the snapshot such that we have to process the segments_N files plus the .del file - if (INDEX_SOFT_DELETES_SETTING.get(indexSettings)) { - // soft-delete generates DV files. - assertThat(status.getStats().getProcessedFileCount(), greaterThan(2)); - } else { - assertThat(status.getStats().getProcessedFileCount(), equalTo(2)); - } + assertThat(status.getStats().getProcessedFileCount(), equalTo(2)); // we flush before the snapshot such that we have to process the segments_N files plus the .del file } } } diff --git a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java index 588118db4ae..caf4f725fa4 100644 --- a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java +++ b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java @@ -26,7 +26,6 @@ import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.VersionType; @@ -786,26 +785,4 @@ public class SimpleVersioningIT extends ESIntegTestCase { .getVersion(), equalTo(-1L)); } - - public void testSpecialVersioning() { - internalCluster().ensureAtLeastNumDataNodes(2); - createIndex("test", Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).build()); - IndexResponse doc1 = client().prepareIndex("test", "type", "1").setSource("field", "value1") - .setVersion(0).setVersionType(VersionType.EXTERNAL).execute().actionGet(); - assertThat(doc1.getVersion(), equalTo(0L)); - IndexResponse doc2 = client().prepareIndex("test", "type", "1").setSource("field", "value2") - .setVersion(Versions.MATCH_ANY).setVersionType(VersionType.INTERNAL).execute().actionGet(); - assertThat(doc2.getVersion(), equalTo(1L)); - client().prepareDelete("test", "type", "1").get(); //v2 - IndexResponse doc3 = client().prepareIndex("test", "type", "1").setSource("field", "value3") - .setVersion(Versions.MATCH_DELETED).setVersionType(VersionType.INTERNAL).execute().actionGet(); - assertThat(doc3.getVersion(), equalTo(3L)); - IndexResponse doc4 = client().prepareIndex("test", "type", "1").setSource("field", "value4") - .setVersion(4L).setVersionType(VersionType.EXTERNAL_GTE).execute().actionGet(); - assertThat(doc4.getVersion(), equalTo(4L)); - // Make sure that these versions are replicated correctly - client().admin().indices().prepareUpdateSettings("test") - .setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)).get(); - ensureGreen("test"); - } } 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 b558cd1ba90..b5ba5f18b39 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 @@ -19,18 +19,14 @@ package org.elasticsearch.index.engine; -import org.apache.logging.log4j.Logger; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.StoredField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.Term; @@ -38,41 +34,32 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.search.Sort; -import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TotalHitCountCollector; import org.apache.lucene.store.Directory; -import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.MapperTestUtils; -import org.elasticsearch.index.VersionType; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -93,30 +80,17 @@ import org.junit.Before; import java.io.IOException; import java.nio.charset.Charset; import java.nio.file.Path; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; -import java.util.function.Function; import java.util.function.LongSupplier; import java.util.function.ToLongBiFunction; -import java.util.stream.Collectors; import static java.util.Collections.emptyList; -import static java.util.Collections.shuffle; -import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; -import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.notNullValue; public abstract class EngineTestCase extends ESTestCase { @@ -154,20 +128,6 @@ public abstract class EngineTestCase extends ESTestCase { } } - protected Settings indexSettings() { - // TODO randomize more settings - return Settings.builder() - .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), "1h") // make sure this doesn't kick in on us - .put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName) - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), - between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), - randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) - .build(); - } - @Override @Before public void setUp() throws Exception { @@ -182,7 +142,13 @@ public abstract class EngineTestCase extends ESTestCase { } else { codecName = "default"; } - defaultSettings = IndexSettingsModule.newIndexSettings("test", indexSettings()); + defaultSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder() + .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), "1h") // make sure this doesn't kick in on us + .put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName) + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), + between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) + .build()); // TODO randomize more settings threadPool = new TestThreadPool(getClass().getName()); store = createStore(); storeReplica = createStore(); @@ -214,7 +180,7 @@ public abstract class EngineTestCase extends ESTestCase { new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService(), globalCheckpointSupplier, config.getPrimaryTermSupplier(), tombstoneDocSupplier()); + config.getCircuitBreakerService(), globalCheckpointSupplier, config.getPrimaryTermSupplier()); } public EngineConfig copy(EngineConfig config, Analyzer analyzer) { @@ -223,18 +189,7 @@ public abstract class EngineTestCase extends ESTestCase { new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier(), - config.getTombstoneDocSupplier()); - } - - public EngineConfig copy(EngineConfig config, MergePolicy mergePolicy) { - return new EngineConfig(config.getShardId(), config.getAllocationId(), config.getThreadPool(), config.getIndexSettings(), - config.getWarmer(), config.getStore(), mergePolicy, config.getAnalyzer(), config.getSimilarity(), - new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), - config.getTranslogConfig(), config.getFlushMergesAfter(), - config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier(), - config.getTombstoneDocSupplier()); + config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier()); } @Override @@ -243,11 +198,9 @@ public abstract class EngineTestCase extends ESTestCase { super.tearDown(); if (engine != null && engine.isClosed.get() == false) { engine.getTranslog().getDeletionPolicy().assertNoOpenTranslogRefs(); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, createMapperService("test")); } if (replicaEngine != null && replicaEngine.isClosed.get() == false) { replicaEngine.getTranslog().getDeletionPolicy().assertNoOpenTranslogRefs(); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(replicaEngine, createMapperService("test")); } IOUtils.close( replicaEngine, storeReplica, @@ -275,18 +228,8 @@ public abstract class EngineTestCase extends ESTestCase { return testParsedDocument(id, routing, testDocumentWithTextField(), new BytesArray("{ \"value\" : \"test\" }"), null); } - public static ParsedDocument createParsedDoc(String id, String routing, boolean recoverySource) { - return testParsedDocument(id, routing, testDocumentWithTextField(), new BytesArray("{ \"value\" : \"test\" }"), null, - recoverySource); - } - protected static ParsedDocument testParsedDocument( String id, String routing, ParseContext.Document document, BytesReference source, Mapping mappingUpdate) { - return testParsedDocument(id, routing, document, source, mappingUpdate, false); - } - protected static ParsedDocument testParsedDocument( - String id, String routing, ParseContext.Document document, BytesReference source, Mapping mappingUpdate, - boolean recoverySource) { Field uidField = new Field("_id", Uid.encodeId(id), IdFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); @@ -296,57 +239,11 @@ public abstract class EngineTestCase extends ESTestCase { document.add(seqID.seqNoDocValue); document.add(seqID.primaryTerm); BytesRef ref = source.toBytesRef(); - if (recoverySource) { - document.add(new StoredField(SourceFieldMapper.RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); - document.add(new NumericDocValuesField(SourceFieldMapper.RECOVERY_SOURCE_NAME, 1)); - } else { - document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length)); - } + document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length)); return new ParsedDocument(versionField, seqID, id, "test", routing, Arrays.asList(document), source, XContentType.JSON, mappingUpdate); } - /** - * Creates a tombstone document that only includes uid, seq#, term and version fields. - */ - public static EngineConfig.TombstoneDocSupplier tombstoneDocSupplier(){ - return new EngineConfig.TombstoneDocSupplier() { - @Override - public ParsedDocument newDeleteTombstoneDoc(String type, String id) { - final ParseContext.Document doc = new ParseContext.Document(); - Field uidField = new Field(IdFieldMapper.NAME, Uid.encodeId(id), IdFieldMapper.Defaults.FIELD_TYPE); - doc.add(uidField); - Field versionField = new NumericDocValuesField(VersionFieldMapper.NAME, 0); - doc.add(versionField); - SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); - doc.add(seqID.seqNo); - doc.add(seqID.seqNoDocValue); - doc.add(seqID.primaryTerm); - seqID.tombstoneField.setLongValue(1); - doc.add(seqID.tombstoneField); - return new ParsedDocument(versionField, seqID, id, type, null, - Collections.singletonList(doc), new BytesArray("{}"), XContentType.JSON, null); - } - - @Override - public ParsedDocument newNoopTombstoneDoc(String reason) { - final ParseContext.Document doc = new ParseContext.Document(); - SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); - doc.add(seqID.seqNo); - doc.add(seqID.seqNoDocValue); - doc.add(seqID.primaryTerm); - seqID.tombstoneField.setLongValue(1); - doc.add(seqID.tombstoneField); - Field versionField = new NumericDocValuesField(VersionFieldMapper.NAME, 0); - doc.add(versionField); - BytesRef byteRef = new BytesRef(reason); - doc.add(new StoredField(SourceFieldMapper.NAME, byteRef.bytes, byteRef.offset, byteRef.length)); - return new ParsedDocument(versionField, seqID, null, null, null, - Collections.singletonList(doc), null, XContentType.JSON, null); - } - }; - } - protected Store createStore() throws IOException { return createStore(newDirectory()); } @@ -564,7 +461,7 @@ public abstract class EngineTestCase extends ESTestCase { new NoneCircuitBreakerService(), globalCheckpointSupplier == null ? new ReplicationTracker(shardId, allocationId.getId(), indexSettings, SequenceNumbers.NO_OPS_PERFORMED, update -> {}) : - globalCheckpointSupplier, primaryTerm::get, tombstoneDocSupplier()); + globalCheckpointSupplier, primaryTerm::get); return config; } @@ -577,7 +474,7 @@ public abstract class EngineTestCase extends ESTestCase { return new BytesArray(string.getBytes(Charset.defaultCharset())); } - protected static Term newUid(String id) { + protected Term newUid(String id) { return new Term("_id", Uid.encodeId(id)); } @@ -602,279 +499,6 @@ public abstract class EngineTestCase extends ESTestCase { protected Engine.Delete replicaDeleteForDoc(String id, long version, long seqNo, long startTime) { return new Engine.Delete("test", id, newUid(id), seqNo, 1, version, null, Engine.Operation.Origin.REPLICA, startTime); } - protected static void assertVisibleCount(InternalEngine engine, int numDocs) throws IOException { - assertVisibleCount(engine, numDocs, true); - } - - protected static void assertVisibleCount(InternalEngine engine, int numDocs, boolean refresh) throws IOException { - if (refresh) { - engine.refresh("test"); - } - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - final TotalHitCountCollector collector = new TotalHitCountCollector(); - searcher.searcher().search(new MatchAllDocsQuery(), collector); - assertThat(collector.getTotalHits(), equalTo(numDocs)); - } - } - - public static List generateSingleDocHistory(boolean forReplica, VersionType versionType, - long primaryTerm, int minOpCount, int maxOpCount, String docId) { - final int numOfOps = randomIntBetween(minOpCount, maxOpCount); - final List ops = new ArrayList<>(); - final Term id = newUid(docId); - final int startWithSeqNo = 0; - final String valuePrefix = (forReplica ? "r_" : "p_" ) + docId + "_"; - final boolean incrementTermWhenIntroducingSeqNo = randomBoolean(); - for (int i = 0; i < numOfOps; i++) { - final Engine.Operation op; - final long version; - switch (versionType) { - case INTERNAL: - version = forReplica ? i : Versions.MATCH_ANY; - break; - case EXTERNAL: - version = i; - break; - case EXTERNAL_GTE: - version = randomBoolean() ? Math.max(i - 1, 0) : i; - break; - case FORCE: - version = randomNonNegativeLong(); - break; - default: - throw new UnsupportedOperationException("unknown version type: " + versionType); - } - if (randomBoolean()) { - op = new Engine.Index(id, testParsedDocument(docId, null, testDocumentWithTextField(valuePrefix + i), B_1, null), - forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, - forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, - version, - forReplica ? null : versionType, - forReplica ? REPLICA : PRIMARY, - System.currentTimeMillis(), -1, false - ); - } else { - op = new Engine.Delete("test", docId, id, - forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, - forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, - version, - forReplica ? null : versionType, - forReplica ? REPLICA : PRIMARY, - System.currentTimeMillis()); - } - ops.add(op); - } - return ops; - } - - public static void assertOpsOnReplica( - final List ops, - final InternalEngine replicaEngine, - boolean shuffleOps, - final Logger logger) throws IOException { - final Engine.Operation lastOp = ops.get(ops.size() - 1); - final String lastFieldValue; - if (lastOp instanceof Engine.Index) { - Engine.Index index = (Engine.Index) lastOp; - lastFieldValue = index.docs().get(0).get("value"); - } else { - // delete - lastFieldValue = null; - } - if (shuffleOps) { - int firstOpWithSeqNo = 0; - while (firstOpWithSeqNo < ops.size() && ops.get(firstOpWithSeqNo).seqNo() < 0) { - firstOpWithSeqNo++; - } - // shuffle ops but make sure legacy ops are first - shuffle(ops.subList(0, firstOpWithSeqNo), random()); - shuffle(ops.subList(firstOpWithSeqNo, ops.size()), random()); - } - boolean firstOp = true; - for (Engine.Operation op : ops) { - logger.info("performing [{}], v [{}], seq# [{}], term [{}]", - op.operationType().name().charAt(0), op.version(), op.seqNo(), op.primaryTerm()); - if (op instanceof Engine.Index) { - Engine.IndexResult result = replicaEngine.index((Engine.Index) op); - // replicas don't really care to about creation status of documents - // this allows to ignore the case where a document was found in the live version maps in - // a delete state and return false for the created flag in favor of code simplicity - // as deleted or not. This check is just signal regression so a decision can be made if it's - // intentional - assertThat(result.isCreated(), equalTo(firstOp)); - assertThat(result.getVersion(), equalTo(op.version())); - assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); - - } else { - Engine.DeleteResult result = replicaEngine.delete((Engine.Delete) op); - // Replicas don't really care to about found status of documents - // this allows to ignore the case where a document was found in the live version maps in - // a delete state and return true for the found flag in favor of code simplicity - // his check is just signal regression so a decision can be made if it's - // intentional - assertThat(result.isFound(), equalTo(firstOp == false)); - assertThat(result.getVersion(), equalTo(op.version())); - assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); - } - if (randomBoolean()) { - replicaEngine.refresh("test"); - } - if (randomBoolean()) { - replicaEngine.flush(); - replicaEngine.refresh("test"); - } - firstOp = false; - } - - assertVisibleCount(replicaEngine, lastFieldValue == null ? 0 : 1); - if (lastFieldValue != null) { - try (Engine.Searcher searcher = replicaEngine.acquireSearcher("test")) { - final TotalHitCountCollector collector = new TotalHitCountCollector(); - searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); - assertThat(collector.getTotalHits(), equalTo(1)); - } - } - } - - protected void concurrentlyApplyOps(List ops, InternalEngine engine) throws InterruptedException { - Thread[] thread = new Thread[randomIntBetween(3, 5)]; - CountDownLatch startGun = new CountDownLatch(thread.length); - AtomicInteger offset = new AtomicInteger(-1); - for (int i = 0; i < thread.length; i++) { - thread[i] = new Thread(() -> { - startGun.countDown(); - try { - startGun.await(); - } catch (InterruptedException e) { - throw new AssertionError(e); - } - int docOffset; - while ((docOffset = offset.incrementAndGet()) < ops.size()) { - try { - final Engine.Operation op = ops.get(docOffset); - if (op instanceof Engine.Index) { - engine.index((Engine.Index) op); - } else if (op instanceof Engine.Delete){ - engine.delete((Engine.Delete) op); - } else { - engine.noOp((Engine.NoOp) op); - } - if ((docOffset + 1) % 4 == 0) { - engine.refresh("test"); - } - if (rarely()) { - engine.flush(); - } - } catch (IOException e) { - throw new AssertionError(e); - } - } - }); - thread[i].start(); - } - for (int i = 0; i < thread.length; i++) { - thread[i].join(); - } - } - - /** - * Gets all docId from the given engine. - */ - public static Set getDocIds(Engine engine, boolean refresh) throws IOException { - if (refresh) { - engine.refresh("test_get_doc_ids"); - } - try (Engine.Searcher searcher = engine.acquireSearcher("test_get_doc_ids")) { - Set ids = new HashSet<>(); - for (LeafReaderContext leafContext : searcher.reader().leaves()) { - LeafReader reader = leafContext.reader(); - Bits liveDocs = reader.getLiveDocs(); - for (int i = 0; i < reader.maxDoc(); i++) { - if (liveDocs == null || liveDocs.get(i)) { - Document uuid = reader.document(i, Collections.singleton(IdFieldMapper.NAME)); - BytesRef binaryID = uuid.getBinaryValue(IdFieldMapper.NAME); - ids.add(Uid.decodeId(Arrays.copyOfRange(binaryID.bytes, binaryID.offset, binaryID.offset + binaryID.length))); - } - } - } - return ids; - } - } - - /** - * Reads all engine operations that have been processed by the engine from Lucene index. - * The returned operations are sorted and de-duplicated, thus each sequence number will be have at most one operation. - */ - public static List readAllOperationsInLucene(Engine engine, MapperService mapper) throws IOException { - final List operations = new ArrayList<>(); - long maxSeqNo = Math.max(0, ((InternalEngine)engine).getLocalCheckpointTracker().getMaxSeqNo()); - try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", mapper, 0, maxSeqNo, false)) { - Translog.Operation op; - while ((op = snapshot.next()) != null){ - operations.add(op); - } - } - return operations; - } - - /** - * Asserts the provided engine has a consistent document history between translog and Lucene index. - */ - public static void assertConsistentHistoryBetweenTranslogAndLuceneIndex(Engine engine, MapperService mapper) throws IOException { - if (mapper.documentMapper() == null || engine.config().getIndexSettings().isSoftDeleteEnabled() == false) { - return; - } - final long maxSeqNo = ((InternalEngine) engine).getLocalCheckpointTracker().getMaxSeqNo(); - if (maxSeqNo < 0) { - return; // nothing to check - } - final Map translogOps = new HashMap<>(); - try (Translog.Snapshot snapshot = EngineTestCase.getTranslog(engine).newSnapshot()) { - Translog.Operation op; - while ((op = snapshot.next()) != null) { - translogOps.put(op.seqNo(), op); - } - } - final Map luceneOps = readAllOperationsInLucene(engine, mapper).stream() - .collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); - final long globalCheckpoint = EngineTestCase.getTranslog(engine).getLastSyncedGlobalCheckpoint(); - final long retainedOps = engine.config().getIndexSettings().getSoftDeleteRetentionOperations(); - final long seqNoForRecovery; - try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { - seqNoForRecovery = Long.parseLong(safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1; - } - final long minSeqNoToRetain = Math.min(seqNoForRecovery, globalCheckpoint + 1 - retainedOps); - for (Translog.Operation translogOp : translogOps.values()) { - final Translog.Operation luceneOp = luceneOps.get(translogOp.seqNo()); - if (luceneOp == null) { - if (minSeqNoToRetain <= translogOp.seqNo() && translogOp.seqNo() <= maxSeqNo) { - fail("Operation not found seq# [" + translogOp.seqNo() + "], global checkpoint [" + globalCheckpoint + "], " + - "retention policy [" + retainedOps + "], maxSeqNo [" + maxSeqNo + "], translog op [" + translogOp + "]"); - } else { - continue; - } - } - assertThat(luceneOp, notNullValue()); - assertThat(luceneOp.toString(), luceneOp.primaryTerm(), equalTo(translogOp.primaryTerm())); - assertThat(luceneOp.opType(), equalTo(translogOp.opType())); - if (luceneOp.opType() == Translog.Operation.Type.INDEX) { - assertThat(luceneOp.getSource().source, equalTo(translogOp.getSource().source)); - } - } - } - - protected MapperService createMapperService(String type) throws IOException { - IndexMetaData indexMetaData = IndexMetaData.builder("test") - .settings(Settings.builder() - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)) - .putMapping(type, "{\"properties\": {}}") - .build(); - MapperService mapperService = MapperTestUtils.newMapperService(new NamedXContentRegistry(ClusterModule.getNamedXWriteables()), - createTempDir(), Settings.EMPTY, "test"); - mapperService.merge(indexMetaData, MapperService.MergeReason.MAPPING_UPDATE); - return mapperService; - } /** * Exposes a translog associated with the given engine for testing purpose. diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index f2afdff9c3a..3f1f5daf514 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -60,7 +60,6 @@ import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; @@ -100,14 +99,10 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase protected final Index index = new Index("test", "uuid"); private final ShardId shardId = new ShardId(index, 0); - protected final Map indexMapping = Collections.singletonMap("type", "{ \"type\": {} }"); + private final Map indexMapping = Collections.singletonMap("type", "{ \"type\": {} }"); protected ReplicationGroup createGroup(int replicas) throws IOException { - return createGroup(replicas, Settings.EMPTY); - } - - protected ReplicationGroup createGroup(int replicas, Settings settings) throws IOException { - IndexMetaData metaData = buildIndexMetaData(replicas, settings, indexMapping); + IndexMetaData metaData = buildIndexMetaData(replicas); return new ReplicationGroup(metaData); } @@ -116,17 +111,9 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } protected IndexMetaData buildIndexMetaData(int replicas, Map mappings) throws IOException { - return buildIndexMetaData(replicas, Settings.EMPTY, mappings); - } - - protected IndexMetaData buildIndexMetaData(int replicas, Settings indexSettings, Map mappings) throws IOException { Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, replicas) .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), - randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) - .put(indexSettings) .build(); IndexMetaData.Builder metaData = IndexMetaData.builder(index.getName()) .settings(settings) @@ -159,7 +146,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } }); - protected ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { + ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { final ShardRouting primaryRouting = this.createShardRouting("s0", true); primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting), () -> {}); replicas = new CopyOnWriteArrayList<>(); @@ -461,7 +448,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } } - protected abstract class ReplicationAction, + abstract class ReplicationAction, ReplicaRequest extends ReplicationRequest, Response extends ReplicationResponse> { private final Request request; @@ -469,7 +456,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase private final ReplicationGroup replicationGroup; private final String opType; - protected ReplicationAction(Request request, ActionListener listener, ReplicationGroup group, String opType) { + ReplicationAction(Request request, ActionListener listener, ReplicationGroup group, String opType) { this.request = request; this.listener = listener; this.replicationGroup = group; @@ -595,11 +582,11 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } } - protected class PrimaryResult implements ReplicationOperation.PrimaryResult { + class PrimaryResult implements ReplicationOperation.PrimaryResult { final ReplicaRequest replicaRequest; final Response finalResponse; - public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) { + PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) { this.replicaRequest = replicaRequest; this.finalResponse = finalResponse; } diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 2f4a3dfd6c1..d2a84589669 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -18,8 +18,13 @@ */ package org.elasticsearch.index.shard; +import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexNotFoundException; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.index.IndexRequest; @@ -52,8 +57,10 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngineFactory; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.SourceToParse; +import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.similarity.SimilarityService; @@ -173,63 +180,37 @@ public abstract class IndexShardTestCase extends ESTestCase { } /** - * Creates a new initializing shard. The shard will have its own unique data path. + * creates a new initializing shard. The shard will have its own unique data path. * - * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica (ready to recover from - * another shard) + * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica + * (ready to recover from another shard) */ protected IndexShard newShard(boolean primary) throws IOException { - return newShard(primary, Settings.EMPTY, new InternalEngineFactory()); + ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), primary, + ShardRoutingState.INITIALIZING, + primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); + return newShard(shardRouting); } /** - * Creates a new initializing shard. The shard will have its own unique data path. + * creates a new initializing shard. The shard will have its own unique data path. * - * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica (ready to recover from - * another shard) - * @param settings the settings to use for this shard - * @param engineFactory the engine factory to use for this shard - */ - protected IndexShard newShard(boolean primary, Settings settings, EngineFactory engineFactory) throws IOException { - final RecoverySource recoverySource = - primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE; - final ShardRouting shardRouting = - TestShardRouting.newShardRouting( - new ShardId("index", "_na_", 0), randomAlphaOfLength(10), primary, ShardRoutingState.INITIALIZING, recoverySource); - return newShard(shardRouting, settings, engineFactory); - } - - protected IndexShard newShard(ShardRouting shardRouting, final IndexingOperationListener... listeners) throws IOException { - return newShard(shardRouting, Settings.EMPTY, new InternalEngineFactory(), listeners); - } - - /** - * Creates a new initializing shard. The shard will have its own unique data path. - * - * @param shardRouting the {@link ShardRouting} to use for this shard - * @param settings the settings to use for this shard - * @param engineFactory the engine factory to use for this shard - * @param listeners an optional set of listeners to add to the shard + * @param shardRouting the {@link ShardRouting} to use for this shard + * @param listeners an optional set of listeners to add to the shard */ protected IndexShard newShard( final ShardRouting shardRouting, - final Settings settings, - final EngineFactory engineFactory, final IndexingOperationListener... listeners) throws IOException { assert shardRouting.initializing() : shardRouting; - Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), - randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) - .put(settings) - .build(); + Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .build(); IndexMetaData.Builder metaData = IndexMetaData.builder(shardRouting.getIndexName()) - .settings(indexSettings) + .settings(settings) .primaryTerm(0, primaryTerm) .putMapping("_doc", "{ \"properties\": {} }"); - return newShard(shardRouting, metaData.build(), engineFactory, listeners); + return newShard(shardRouting, metaData.build(), listeners); } /** @@ -244,7 +225,7 @@ public abstract class IndexShardTestCase extends ESTestCase { ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, randomAlphaOfLength(5), primary, ShardRoutingState.INITIALIZING, primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); - return newShard(shardRouting, Settings.EMPTY, new InternalEngineFactory(), listeners); + return newShard(shardRouting, listeners); } /** @@ -284,10 +265,9 @@ public abstract class IndexShardTestCase extends ESTestCase { * @param indexMetaData indexMetaData for the shard, including any mapping * @param listeners an optional set of listeners to add to the shard */ - protected IndexShard newShard( - ShardRouting routing, IndexMetaData indexMetaData, EngineFactory engineFactory, IndexingOperationListener... listeners) + protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, IndexingOperationListener... listeners) throws IOException { - return newShard(routing, indexMetaData, null, engineFactory, () -> {}, listeners); + return newShard(routing, indexMetaData, null, new InternalEngineFactory(), () -> {}, listeners); } /** @@ -392,39 +372,19 @@ public abstract class IndexShardTestCase extends ESTestCase { } /** - * Creates a new empty shard and starts it. The shard will randomly be a replica or a primary. + * creates a new empyu shard and starts it. The shard will be either a replica or a primary. */ protected IndexShard newStartedShard() throws IOException { return newStartedShard(randomBoolean()); } /** - * Creates a new empty shard and starts it - * @param settings the settings to use for this shard - */ - protected IndexShard newStartedShard(Settings settings) throws IOException { - return newStartedShard(randomBoolean(), settings, new InternalEngineFactory()); - } - - /** - * Creates a new empty shard and starts it. + * creates a new empty shard and starts it. * * @param primary controls whether the shard will be a primary or a replica. */ - protected IndexShard newStartedShard(final boolean primary) throws IOException { - return newStartedShard(primary, Settings.EMPTY, new InternalEngineFactory()); - } - - /** - * Creates a new empty shard with the specified settings and engine factory and starts it. - * - * @param primary controls whether the shard will be a primary or a replica. - * @param settings the settings to use for this shard - * @param engineFactory the engine factory to use for this shard - */ - protected IndexShard newStartedShard( - final boolean primary, final Settings settings, final EngineFactory engineFactory) throws IOException { - IndexShard shard = newShard(primary, settings, engineFactory); + protected IndexShard newStartedShard(boolean primary) throws IOException { + IndexShard shard = newShard(primary); if (primary) { recoverShardFromStore(shard); } else { @@ -441,7 +401,6 @@ public abstract class IndexShardTestCase extends ESTestCase { for (IndexShard shard : shards) { if (shard != null) { try { - assertConsistentHistoryBetweenTranslogAndLucene(shard); shard.close("test", false); } finally { IOUtils.close(shard.store()); @@ -623,7 +582,22 @@ public abstract class IndexShardTestCase extends ESTestCase { } protected Set getShardDocUIDs(final IndexShard shard) throws IOException { - return EngineTestCase.getDocIds(shard.getEngine(), true); + shard.refresh("get_uids"); + try (Engine.Searcher searcher = shard.acquireSearcher("test")) { + Set ids = new HashSet<>(); + for (LeafReaderContext leafContext : searcher.reader().leaves()) { + LeafReader reader = leafContext.reader(); + Bits liveDocs = reader.getLiveDocs(); + for (int i = 0; i < reader.maxDoc(); i++) { + if (liveDocs == null || liveDocs.get(i)) { + Document uuid = reader.document(i, Collections.singleton(IdFieldMapper.NAME)); + BytesRef binaryID = uuid.getBinaryValue(IdFieldMapper.NAME); + ids.add(Uid.decodeId(Arrays.copyOfRange(binaryID.bytes, binaryID.offset, binaryID.offset + binaryID.length))); + } + } + } + return ids; + } } protected void assertDocCount(IndexShard shard, int docDount) throws IOException { @@ -636,12 +610,6 @@ public abstract class IndexShardTestCase extends ESTestCase { assertThat(shardDocUIDs, hasSize(ids.length)); } - public static void assertConsistentHistoryBetweenTranslogAndLucene(IndexShard shard) throws IOException { - final Engine engine = shard.getEngineOrNull(); - if (engine != null) { - EngineTestCase.assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, shard.mapperService()); - } - } protected Engine.IndexResult indexDoc(IndexShard shard, String type, String id) throws IOException { return indexDoc(shard, type, id, "{}"); @@ -685,14 +653,11 @@ public abstract class IndexShardTestCase extends ESTestCase { } protected Engine.DeleteResult deleteDoc(IndexShard shard, String type, String id) throws IOException { - final Engine.DeleteResult result; if (shard.routingEntry().primary()) { - result = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, type, id, VersionType.INTERNAL); - shard.updateLocalCheckpointForShard(shard.routingEntry().allocationId().getId(), shard.getEngine().getLocalCheckpoint()); + return shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, type, id, VersionType.INTERNAL); } else { - result = shard.applyDeleteOperationOnReplica(shard.seqNoStats().getMaxSeqNo() + 1, 0L, type, id); + return shard.applyDeleteOperationOnReplica(shard.seqNoStats().getMaxSeqNo() + 1, 0L, type, id); } - return result; } protected void flushShard(IndexShard shard) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index be9e40ab420..322e2a128c9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -723,10 +723,6 @@ public abstract class ESIntegTestCase extends ESTestCase { } // always default delayed allocation to 0 to make sure we have tests are not delayed builder.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0); - builder.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()); - if (randomBoolean()) { - builder.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), between(0, 1000)); - } return builder.build(); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java index 19290f8cf11..9633f56dea9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java @@ -41,7 +41,6 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.node.MockNode; @@ -88,14 +87,6 @@ public abstract class ESSingleNodeTestCase extends ESTestCase { .setOrder(0) .setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)).get(); - client().admin().indices() - .preparePutTemplate("random-soft-deletes-template") - .setPatterns(Collections.singletonList("*")) - .setOrder(0) - .setSettings(Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), - randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) - ).get(); } private static void stopNode() throws IOException { diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 4c813372fae..306f79e5e16 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -1163,26 +1163,6 @@ public final class InternalTestCluster extends TestCluster { }); } - /** - * Asserts that the document history in Lucene index is consistent with Translog's on every index shard of the cluster. - * This assertion might be expensive, thus we prefer not to execute on every test but only interesting tests. - */ - public void assertConsistentHistoryBetweenTranslogAndLuceneIndex() throws IOException { - final Collection nodesAndClients = nodes.values(); - for (NodeAndClient nodeAndClient : nodesAndClients) { - IndicesService indexServices = getInstance(IndicesService.class, nodeAndClient.name); - for (IndexService indexService : indexServices) { - for (IndexShard indexShard : indexService) { - try { - IndexShardTestCase.assertConsistentHistoryBetweenTranslogAndLucene(indexShard); - } catch (AlreadyClosedException ignored) { - // shard is closed - } - } - } - } - } - private void randomlyResetClients() throws IOException { // only reset the clients on nightly tests, it causes heavy load... if (RandomizedTest.isNightly() && rarely(random)) { From ad4dd086d2cef4df3008fd1a3acffe5735231ce8 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 30 Aug 2018 22:11:23 -0400 Subject: [PATCH 07/16] Integrates soft-deletes into Elasticsearch (#33222) This PR integrates Lucene soft-deletes(LUCENE-8200) into Elasticsearch. Highlight works in this PR include: - Replace hard-deletes by soft-deletes in InternalEngine - Use _recovery_source if _source is disabled or modified (#31106) - Soft-deletes retention policy based on the global checkpoint (#30335) - Read operation history from Lucene instead of translog (#30120) - Use Lucene history in peer-recovery (#30522) Relates #30086 Closes #29530 --- These works have been done by the whole team; however, these individuals (lexical order) have significant contribution in coding and reviewing: Co-authored-by: Adrien Grand Co-authored-by: Boaz Leskes Co-authored-by: Jason Tedor Co-authored-by: Martijn van Groningen Co-authored-by: Nhat Nguyen Co-authored-by: Simon Willnauer --- .../percolator/CandidateQueryTests.java | 8 +- .../PercolatorFieldMapperTests.java | 30 +- .../elasticsearch/common/lucene/Lucene.java | 86 ++- .../uid/PerThreadIDVersionAndSeqNoLookup.java | 21 +- .../common/settings/IndexScopedSettings.java | 2 + .../elasticsearch/index/IndexSettings.java | 38 ++ .../index/engine/CombinedDeletionPolicy.java | 12 +- .../elasticsearch/index/engine/Engine.java | 28 +- .../index/engine/EngineConfig.java | 27 +- .../index/engine/InternalEngine.java | 390 +++++++++-- .../index/engine/LuceneChangesSnapshot.java | 368 +++++++++++ .../RecoverySourcePruneMergePolicy.java | 292 +++++++++ .../index/engine/SoftDeletesPolicy.java | 120 ++++ .../index/fieldvisitor/FieldsVisitor.java | 10 +- .../index/mapper/DocumentMapper.java | 34 +- .../index/mapper/DocumentParser.java | 33 +- .../index/mapper/FieldNamesFieldMapper.java | 5 +- .../index/mapper/ParseContext.java | 20 +- .../index/mapper/ParsedDocument.java | 11 + .../index/mapper/SeqNoFieldMapper.java | 7 +- .../index/mapper/SourceFieldMapper.java | 16 +- .../elasticsearch/index/shard/IndexShard.java | 47 +- .../index/shard/PrimaryReplicaSyncer.java | 2 +- .../index/shard/StoreRecovery.java | 1 + .../org/elasticsearch/index/store/Store.java | 2 +- .../index/translog/Translog.java | 3 + .../index/translog/TranslogWriter.java | 20 +- .../translog/TruncateTranslogCommand.java | 2 + .../recovery/RecoverySourceHandler.java | 59 +- .../blobstore/BlobStoreRepository.java | 1 + .../snapshots/RestoreService.java | 4 +- .../cluster/routing/PrimaryAllocationIT.java | 1 + .../common/lucene/LuceneTests.java | 91 +++ .../discovery/AbstractDisruptionTestCase.java | 1 + .../gateway/RecoveryFromGatewayIT.java | 13 +- .../index/IndexServiceTests.java | 3 +- .../index/IndexSettingsTests.java | 8 + .../engine/CombinedDeletionPolicyTests.java | 69 +- .../index/engine/InternalEngineTests.java | 620 ++++++++++++------ .../engine/LuceneChangesSnapshotTests.java | 289 ++++++++ .../RecoverySourcePruneMergePolicyTests.java | 161 +++++ .../index/engine/SoftDeletesPolicyTests.java | 75 +++ .../index/mapper/DocumentParserTests.java | 10 +- .../index/mapper/DynamicMappingTests.java | 6 +- .../IndexLevelReplicationTests.java | 29 +- .../RecoveryDuringReplicationTests.java | 11 +- .../index/shard/IndexShardTests.java | 58 +- .../shard/PrimaryReplicaSyncerTests.java | 21 +- .../index/shard/RefreshListenersTests.java | 4 +- .../indices/recovery/IndexRecoveryIT.java | 6 + .../PeerRecoveryTargetServiceTests.java | 2 + .../recovery/RecoverySourceHandlerTests.java | 6 - .../indices/recovery/RecoveryTests.java | 80 ++- .../indices/stats/IndexStatsIT.java | 37 +- .../AbstractSnapshotIntegTestCase.java | 6 + .../SharedClusterSnapshotRestoreIT.java | 13 +- .../versioning/SimpleVersioningIT.java | 23 + .../index/engine/EngineTestCase.java | 400 ++++++++++- .../ESIndexLevelReplicationTestCase.java | 27 +- .../index/shard/IndexShardTestCase.java | 131 ++-- .../elasticsearch/test/ESIntegTestCase.java | 4 + .../test/ESSingleNodeTestCase.java | 9 + .../test/InternalTestCluster.java | 20 + 63 files changed, 3433 insertions(+), 500 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java create mode 100644 server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java create mode 100644 server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java index e6d637aabb1..9c8979601e8 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java @@ -77,6 +77,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; @@ -87,6 +88,7 @@ import org.elasticsearch.common.geo.ShapeRelation; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; @@ -1109,7 +1111,11 @@ public class CandidateQueryTests extends ESSingleNodeTestCase { } private void addQuery(Query query, List docs) { - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(query, parseContext); ParseContext.Document queryDocument = parseContext.doc(); diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java index ecff48b344c..80524a2f862 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java @@ -42,6 +42,7 @@ import org.apache.lucene.search.join.ScoreMode; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -58,6 +59,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperParser; import org.elasticsearch.index.mapper.MapperParsingException; @@ -182,7 +184,11 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); ParseContext.Document document = parseContext.doc(); @@ -204,7 +210,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { bq.add(termQuery1, Occur.MUST); bq.add(termQuery2, Occur.MUST); - parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), + parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); document = parseContext.doc(); @@ -232,8 +238,12 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { bq.add(rangeQuery2, Occur.MUST); DocumentMapper documentMapper = mapperService.documentMapper("doc"); + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); ParseContext.Document document = parseContext.doc(); @@ -259,7 +269,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { .rangeQuery(15, 20, true, true, null, null, null, null); bq.add(rangeQuery2, Occur.MUST); - parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); document = parseContext.doc(); @@ -283,7 +293,11 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { TermRangeQuery query = new TermRangeQuery("field1", new BytesRef("a"), new BytesRef("z"), true, true); DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(query, parseContext); ParseContext.Document document = parseContext.doc(); @@ -298,7 +312,11 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { PhraseQuery phraseQuery = new PhraseQuery("field", "term"); DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(phraseQuery, parseContext); ParseContext.Document document = parseContext.doc(); diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index a24a6aea07f..1c1e5687893 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -27,8 +27,10 @@ import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.document.LatLonDocValuesField; +import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FilterDirectoryReader; import org.apache.lucene.index.FilterLeafReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexFileNames; @@ -96,6 +98,8 @@ public class Lucene { assert annotation == null : "DocValuesFormat " + LATEST_DOC_VALUES_FORMAT + " is deprecated" ; } + public static final String SOFT_DELETES_FIELD = "__soft_deletes"; + public static final NamedAnalyzer STANDARD_ANALYZER = new NamedAnalyzer("_standard", AnalyzerScope.GLOBAL, new StandardAnalyzer()); public static final NamedAnalyzer KEYWORD_ANALYZER = new NamedAnalyzer("_keyword", AnalyzerScope.GLOBAL, new KeywordAnalyzer()); @@ -140,7 +144,7 @@ public class Lucene { public static int getNumDocs(SegmentInfos info) { int numDocs = 0; for (SegmentCommitInfo si : info) { - numDocs += si.info.maxDoc() - si.getDelCount(); + numDocs += si.info.maxDoc() - si.getDelCount() - si.getSoftDelCount(); } return numDocs; } @@ -197,6 +201,7 @@ public class Lucene { } final CommitPoint cp = new CommitPoint(si, directory); try (IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(Lucene.STANDARD_ANALYZER) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setIndexCommit(cp) .setCommitOnClose(false) .setMergePolicy(NoMergePolicy.INSTANCE) @@ -220,6 +225,7 @@ public class Lucene { } } try (IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(Lucene.STANDARD_ANALYZER) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setMergePolicy(NoMergePolicy.INSTANCE) // no merges .setCommitOnClose(false) // no commits .setOpenMode(IndexWriterConfig.OpenMode.CREATE))) // force creation - don't append... @@ -829,4 +835,82 @@ public class Lucene { } }; } + + /** + * Wraps a directory reader to make all documents live except those were rolled back + * or hard-deleted due to non-aborting exceptions during indexing. + * The wrapped reader can be used to query all documents. + * + * @param in the input directory reader + * @return the wrapped reader + */ + public static DirectoryReader wrapAllDocsLive(DirectoryReader in) throws IOException { + return new DirectoryReaderWithAllLiveDocs(in); + } + + private static final class DirectoryReaderWithAllLiveDocs extends FilterDirectoryReader { + static final class LeafReaderWithLiveDocs extends FilterLeafReader { + final Bits liveDocs; + final int numDocs; + LeafReaderWithLiveDocs(LeafReader in, Bits liveDocs, int numDocs) { + super(in); + this.liveDocs = liveDocs; + this.numDocs = numDocs; + } + @Override + public Bits getLiveDocs() { + return liveDocs; + } + @Override + public int numDocs() { + return numDocs; + } + @Override + public CacheHelper getCoreCacheHelper() { + return in.getCoreCacheHelper(); + } + @Override + public CacheHelper getReaderCacheHelper() { + return null; // Modifying liveDocs + } + } + + DirectoryReaderWithAllLiveDocs(DirectoryReader in) throws IOException { + super(in, new SubReaderWrapper() { + @Override + public LeafReader wrap(LeafReader leaf) { + SegmentReader segmentReader = segmentReader(leaf); + Bits hardLiveDocs = segmentReader.getHardLiveDocs(); + if (hardLiveDocs == null) { + return new LeafReaderWithLiveDocs(leaf, null, leaf.maxDoc()); + } + // TODO: Can we avoid calculate numDocs by using SegmentReader#getSegmentInfo with LUCENE-8458? + int numDocs = 0; + for (int i = 0; i < hardLiveDocs.length(); i++) { + if (hardLiveDocs.get(i)) { + numDocs++; + } + } + return new LeafReaderWithLiveDocs(segmentReader, hardLiveDocs, numDocs); + } + }); + } + + @Override + protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { + return wrapAllDocsLive(in); + } + + @Override + public CacheHelper getReaderCacheHelper() { + return null; // Modifying liveDocs + } + } + + /** + * Returns a numeric docvalues which can be used to soft-delete documents. + */ + public static NumericDocValuesField newSoftDeletesField() { + return new NumericDocValuesField(SOFT_DELETES_FIELD, 1); + } } diff --git a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java index 38fcdfe5f1b..3a037bed62b 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java @@ -28,6 +28,7 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -66,15 +67,22 @@ final class PerThreadIDVersionAndSeqNoLookup { */ PerThreadIDVersionAndSeqNoLookup(LeafReader reader, String uidField) throws IOException { this.uidField = uidField; - Terms terms = reader.terms(uidField); + final Terms terms = reader.terms(uidField); if (terms == null) { - throw new IllegalArgumentException("reader misses the [" + uidField + "] field"); + // If a segment contains only no-ops, it does not have _uid but has both _soft_deletes and _tombstone fields. + final NumericDocValues softDeletesDV = reader.getNumericDocValues(Lucene.SOFT_DELETES_FIELD); + final NumericDocValues tombstoneDV = reader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); + if (softDeletesDV == null || tombstoneDV == null) { + throw new IllegalArgumentException("reader does not have _uid terms but not a no-op segment; " + + "_soft_deletes [" + softDeletesDV + "], _tombstone [" + tombstoneDV + "]"); + } + termsEnum = null; + } else { + termsEnum = terms.iterator(); } - termsEnum = terms.iterator(); if (reader.getNumericDocValues(VersionFieldMapper.NAME) == null) { - throw new IllegalArgumentException("reader misses the [" + VersionFieldMapper.NAME + "] field"); + throw new IllegalArgumentException("reader misses the [" + VersionFieldMapper.NAME + "] field; _uid terms [" + terms + "]"); } - Object readerKey = null; assert (readerKey = reader.getCoreCacheHelper().getKey()) != null; this.readerKey = readerKey; @@ -111,7 +119,8 @@ final class PerThreadIDVersionAndSeqNoLookup { * {@link DocIdSetIterator#NO_MORE_DOCS} is returned if not found * */ private int getDocID(BytesRef id, Bits liveDocs) throws IOException { - if (termsEnum.seekExact(id)) { + // termsEnum can possibly be null here if this leaf contains only no-ops. + if (termsEnum != null && termsEnum.seekExact(id)) { int docID = DocIdSetIterator.NO_MORE_DOCS; // there may be more than one matching docID, in the case of nested docs, so we want the last one: docsEnum = termsEnum.postings(docsEnum, 0); 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 46e3867f7ae..f3de294046c 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -129,6 +129,8 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.MAX_REGEX_LENGTH_SETTING, ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING, IndexSettings.INDEX_GC_DELETES_SETTING, + IndexSettings.INDEX_SOFT_DELETES_SETTING, + IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING, UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING, diff --git a/server/src/main/java/org/elasticsearch/index/IndexSettings.java b/server/src/main/java/org/elasticsearch/index/IndexSettings.java index 44cd743bbd4..3ea022bbebd 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/server/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -237,6 +237,21 @@ public final class IndexSettings { public static final Setting INDEX_GC_DELETES_SETTING = Setting.timeSetting("index.gc_deletes", DEFAULT_GC_DELETES, new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic, Property.IndexScope); + + /** + * Specifies if the index should use soft-delete instead of hard-delete for update/delete operations. + */ + public static final Setting INDEX_SOFT_DELETES_SETTING = + Setting.boolSetting("index.soft_deletes.enabled", false, Property.IndexScope, Property.Final); + + /** + * Controls how many soft-deleted documents will be kept around before being merged away. Keeping more deleted + * documents increases the chance of operation-based recoveries and allows querying a longer history of documents. + * If soft-deletes is enabled, an engine by default will retain all operations up to the global checkpoint. + **/ + public static final Setting INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING = + Setting.longSetting("index.soft_deletes.retention.operations", 0, 0, Property.IndexScope, Property.Dynamic); + /** * The maximum number of refresh listeners allows on this shard. */ @@ -289,6 +304,8 @@ public final class IndexSettings { private final IndexSortConfig indexSortConfig; private final IndexScopedSettings scopedSettings; private long gcDeletesInMillis = DEFAULT_GC_DELETES.millis(); + private final boolean softDeleteEnabled; + private volatile long softDeleteRetentionOperations; private volatile boolean warmerEnabled; private volatile int maxResultWindow; private volatile int maxInnerResultWindow; @@ -400,6 +417,8 @@ public final class IndexSettings { generationThresholdSize = scopedSettings.get(INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING); mergeSchedulerConfig = new MergeSchedulerConfig(this); gcDeletesInMillis = scopedSettings.get(INDEX_GC_DELETES_SETTING).getMillis(); + softDeleteEnabled = version.onOrAfter(Version.V_7_0_0_alpha1) && scopedSettings.get(INDEX_SOFT_DELETES_SETTING); + softDeleteRetentionOperations = scopedSettings.get(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING); warmerEnabled = scopedSettings.get(INDEX_WARMER_ENABLED_SETTING); maxResultWindow = scopedSettings.get(MAX_RESULT_WINDOW_SETTING); maxInnerResultWindow = scopedSettings.get(MAX_INNER_RESULT_WINDOW_SETTING); @@ -458,6 +477,7 @@ public final class IndexSettings { scopedSettings.addSettingsUpdateConsumer(INDEX_SEARCH_IDLE_AFTER, this::setSearchIdleAfter); scopedSettings.addSettingsUpdateConsumer(MAX_REGEX_LENGTH_SETTING, this::setMaxRegexLength); scopedSettings.addSettingsUpdateConsumer(DEFAULT_PIPELINE, this::setDefaultPipeline); + scopedSettings.addSettingsUpdateConsumer(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, this::setSoftDeleteRetentionOperations); } private void setSearchIdleAfter(TimeValue searchIdleAfter) { this.searchIdleAfter = searchIdleAfter; } @@ -841,4 +861,22 @@ public final class IndexSettings { public void setDefaultPipeline(String defaultPipeline) { this.defaultPipeline = defaultPipeline; } + + /** + * Returns true if soft-delete is enabled. + */ + public boolean isSoftDeleteEnabled() { + return softDeleteEnabled; + } + + private void setSoftDeleteRetentionOperations(long ops) { + this.softDeleteRetentionOperations = ops; + } + + /** + * Returns the number of extra operations (i.e. soft-deleted documents) to be kept for recoveries and history purpose. + */ + public long getSoftDeleteRetentionOperations() { + return this.softDeleteRetentionOperations; + } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index d0575c8a8c9..d10690379ed 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -46,14 +46,17 @@ import java.util.function.LongSupplier; public final class CombinedDeletionPolicy extends IndexDeletionPolicy { private final Logger logger; private final TranslogDeletionPolicy translogDeletionPolicy; + private final SoftDeletesPolicy softDeletesPolicy; private final LongSupplier globalCheckpointSupplier; private final ObjectIntHashMap snapshottedCommits; // Number of snapshots held against each commit point. private volatile IndexCommit safeCommit; // the most recent safe commit point - its max_seqno at most the persisted global checkpoint. private volatile IndexCommit lastCommit; // the most recent commit point - CombinedDeletionPolicy(Logger logger, TranslogDeletionPolicy translogDeletionPolicy, LongSupplier globalCheckpointSupplier) { + CombinedDeletionPolicy(Logger logger, TranslogDeletionPolicy translogDeletionPolicy, + SoftDeletesPolicy softDeletesPolicy, LongSupplier globalCheckpointSupplier) { this.logger = logger; this.translogDeletionPolicy = translogDeletionPolicy; + this.softDeletesPolicy = softDeletesPolicy; this.globalCheckpointSupplier = globalCheckpointSupplier; this.snapshottedCommits = new ObjectIntHashMap<>(); } @@ -80,7 +83,7 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { deleteCommit(commits.get(i)); } } - updateTranslogDeletionPolicy(); + updateRetentionPolicy(); } private void deleteCommit(IndexCommit commit) throws IOException { @@ -90,7 +93,7 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { assert commit.isDeleted() : "Deletion commit [" + commitDescription(commit) + "] was suppressed"; } - private void updateTranslogDeletionPolicy() throws IOException { + private void updateRetentionPolicy() throws IOException { assert Thread.holdsLock(this); logger.debug("Safe commit [{}], last commit [{}]", commitDescription(safeCommit), commitDescription(lastCommit)); assert safeCommit.isDeleted() == false : "The safe commit must not be deleted"; @@ -101,6 +104,9 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { assert minRequiredGen <= lastGen : "minRequiredGen must not be greater than lastGen"; translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastGen); translogDeletionPolicy.setMinTranslogGenerationForRecovery(minRequiredGen); + + softDeletesPolicy.setLocalCheckpointOfSafeCommit( + Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY))); } /** 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 4d95cf89ef0..08724d6e794 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -58,6 +58,7 @@ import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; @@ -97,6 +98,7 @@ public abstract class Engine implements Closeable { public static final String SYNC_COMMIT_ID = "sync_id"; public static final String HISTORY_UUID_KEY = "history_uuid"; + public static final String MIN_RETAINED_SEQNO = "min_retained_seq_no"; protected final ShardId shardId; protected final String allocationId; @@ -585,18 +587,32 @@ public abstract class Engine implements Closeable { public abstract void syncTranslog() throws IOException; - public abstract Closeable acquireTranslogRetentionLock(); + /** + * Acquires a lock on the translog files and Lucene soft-deleted documents to prevent them from being trimmed + */ + public abstract Closeable acquireRetentionLockForPeerRecovery(); /** - * Creates a new translog snapshot from this engine for reading translog operations whose seq# at least the provided seq#. - * The caller has to close the returned snapshot after finishing the reading. + * Creates a new history snapshot from Lucene for reading operations whose seqno in the requesting seqno range (both inclusive) */ - public abstract Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException; + public abstract Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, + long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException; /** - * Returns the estimated number of translog operations in this engine whose seq# at least the provided seq#. + * Creates a new history snapshot for reading operations since {@code startingSeqNo} (inclusive). + * The returned snapshot can be retrieved from either Lucene index or translog files. */ - public abstract int estimateTranslogOperationsFromMinSeq(long minSeqNo); + public abstract Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException; + + /** + * Returns the estimated number of history operations whose seq# at least {@code startingSeqNo}(inclusive) in this engine. + */ + public abstract int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException; + + /** + * Checks if this engine has every operations since {@code startingSeqNo}(inclusive) in its history (either Lucene or translog) + */ + public abstract boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException; public abstract TranslogStats getTranslogStats(); diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index 2deae61bd52..23a90553f60 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.unit.MemorySizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.codec.CodecService; +import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; @@ -80,6 +81,7 @@ public final class EngineConfig { private final CircuitBreakerService circuitBreakerService; private final LongSupplier globalCheckpointSupplier; private final LongSupplier primaryTermSupplier; + private final TombstoneDocSupplier tombstoneDocSupplier; /** * Index setting to change the low level lucene codec used for writing new segments. @@ -126,7 +128,8 @@ public final class EngineConfig { List externalRefreshListener, List internalRefreshListener, Sort indexSort, TranslogRecoveryRunner translogRecoveryRunner, CircuitBreakerService circuitBreakerService, - LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier) { + LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier, + TombstoneDocSupplier tombstoneDocSupplier) { this.shardId = shardId; this.allocationId = allocationId; this.indexSettings = indexSettings; @@ -164,6 +167,7 @@ public final class EngineConfig { this.circuitBreakerService = circuitBreakerService; this.globalCheckpointSupplier = globalCheckpointSupplier; this.primaryTermSupplier = primaryTermSupplier; + this.tombstoneDocSupplier = tombstoneDocSupplier; } /** @@ -373,4 +377,25 @@ public final class EngineConfig { public LongSupplier getPrimaryTermSupplier() { return primaryTermSupplier; } + + /** + * A supplier supplies tombstone documents which will be used in soft-update methods. + * The returned document consists only _uid, _seqno, _term and _version fields; other metadata fields are excluded. + */ + public interface TombstoneDocSupplier { + /** + * Creates a tombstone document for a delete operation. + */ + ParsedDocument newDeleteTombstoneDoc(String type, String id); + + /** + * Creates a tombstone document for a noop operation. + * @param reason the reason of an a noop + */ + ParsedDocument newNoopTombstoneDoc(String reason); + } + + public TombstoneDocSupplier getTombstoneDocSupplier() { + return tombstoneDocSupplier; + } } 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 023e659ffab..da4decc93b1 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -21,16 +21,20 @@ package org.elasticsearch.index.engine; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.ReferenceManager; @@ -42,6 +46,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.InfoStream; +import org.elasticsearch.Assertions; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.Nullable; @@ -61,7 +66,11 @@ import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParseContext; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; import org.elasticsearch.index.seqno.LocalCheckpointTracker; @@ -140,6 +149,10 @@ public class InternalEngine extends Engine { private final CounterMetric numDocDeletes = new CounterMetric(); private final CounterMetric numDocAppends = new CounterMetric(); private final CounterMetric numDocUpdates = new CounterMetric(); + private final NumericDocValuesField softDeletesField = Lucene.newSoftDeletesField(); + private final boolean softDeleteEnabled; + private final SoftDeletesPolicy softDeletesPolicy; + private final LastRefreshedCheckpointListener lastRefreshedCheckpointListener; /** * How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh. IndexingMemoryController polls this @@ -184,8 +197,10 @@ public class InternalEngine extends Engine { assert translog.getGeneration() != null; this.translog = translog; this.localCheckpointTracker = createLocalCheckpointTracker(localCheckpointTrackerSupplier); + this.softDeleteEnabled = engineConfig.getIndexSettings().isSoftDeleteEnabled(); + this.softDeletesPolicy = newSoftDeletesPolicy(); this.combinedDeletionPolicy = - new CombinedDeletionPolicy(logger, translogDeletionPolicy, translog::getLastSyncedGlobalCheckpoint); + new CombinedDeletionPolicy(logger, translogDeletionPolicy, softDeletesPolicy, translog::getLastSyncedGlobalCheckpoint); writer = createWriter(); bootstrapAppendOnlyInfoFromWriter(writer); historyUUID = loadHistoryUUID(writer); @@ -215,6 +230,8 @@ public class InternalEngine extends Engine { for (ReferenceManager.RefreshListener listener: engineConfig.getInternalRefreshListener()) { this.internalSearcherManager.addListener(listener); } + this.lastRefreshedCheckpointListener = new LastRefreshedCheckpointListener(localCheckpointTracker.getCheckpoint()); + this.internalSearcherManager.addListener(lastRefreshedCheckpointListener); success = true; } finally { if (success == false) { @@ -240,6 +257,18 @@ public class InternalEngine extends Engine { return localCheckpointTrackerSupplier.apply(maxSeqNo, localCheckpoint); } + private SoftDeletesPolicy newSoftDeletesPolicy() throws IOException { + final Map commitUserData = store.readLastCommittedSegmentsInfo().userData; + final long lastMinRetainedSeqNo; + if (commitUserData.containsKey(Engine.MIN_RETAINED_SEQNO)) { + lastMinRetainedSeqNo = Long.parseLong(commitUserData.get(Engine.MIN_RETAINED_SEQNO)); + } else { + lastMinRetainedSeqNo = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)) + 1; + } + return new SoftDeletesPolicy(translog::getLastSyncedGlobalCheckpoint, lastMinRetainedSeqNo, + engineConfig.getIndexSettings().getSoftDeleteRetentionOperations()); + } + /** * This reference manager delegates all it's refresh calls to another (internal) SearcherManager * The main purpose for this is that if we have external refreshes happening we don't issue extra @@ -451,19 +480,31 @@ public class InternalEngine extends Engine { revisitIndexDeletionPolicyOnTranslogSynced(); } + /** + * Creates a new history snapshot for reading operations since the provided seqno. + * The returned snapshot can be retrieved from either Lucene index or translog files. + */ @Override - public Closeable acquireTranslogRetentionLock() { - return getTranslog().acquireRetentionLock(); + public Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { + if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + return newChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false); + } else { + return getTranslog().newSnapshotFromMinSeqNo(startingSeqNo); + } } + /** + * Returns the estimated number of history operations whose seq# at least the provided seq# in this engine. + */ @Override - public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { - return getTranslog().newSnapshotFromMinSeqNo(minSeqNo); - } - - @Override - public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { - return getTranslog().estimateTotalOperationsFromMinSeq(minSeqNo); + public int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { + if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + try (Translog.Snapshot snapshot = newChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false)) { + return snapshot.totalOperations(); + } + } else { + return getTranslog().estimateTotalOperationsFromMinSeq(startingSeqNo); + } } @Override @@ -790,7 +831,7 @@ public class InternalEngine extends Engine { if (plan.earlyResultOnPreFlightError.isPresent()) { indexResult = plan.earlyResultOnPreFlightError.get(); assert indexResult.getResultType() == Result.Type.FAILURE : indexResult.getResultType(); - } else if (plan.indexIntoLucene) { + } else if (plan.indexIntoLucene || plan.addStaleOpToLucene) { indexResult = indexIntoLucene(index, plan); } else { indexResult = new IndexResult( @@ -801,8 +842,10 @@ public class InternalEngine extends Engine { if (indexResult.getResultType() == Result.Type.SUCCESS) { location = translog.add(new Translog.Index(index, indexResult)); } else if (indexResult.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - // if we have document failure, record it as a no-op in the translog with the generated seq_no - location = translog.add(new Translog.NoOp(indexResult.getSeqNo(), index.primaryTerm(), indexResult.getFailure().toString())); + // if we have document failure, record it as a no-op in the translog and Lucene with the generated seq_no + final NoOp noOp = new NoOp(indexResult.getSeqNo(), index.primaryTerm(), index.origin(), + index.startTime(), indexResult.getFailure().toString()); + location = innerNoOp(noOp).getTranslogLocation(); } else { location = null; } @@ -854,7 +897,6 @@ public class InternalEngine extends Engine { // unlike the primary, replicas don't really care to about creation status of documents // this allows to ignore the case where a document was found in the live version maps in // a delete state and return false for the created flag in favor of code simplicity - final OpVsLuceneDocStatus opVsLucene; if (index.seqNo() <= localCheckpointTracker.getCheckpoint()){ // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already @@ -863,16 +905,15 @@ public class InternalEngine extends Engine { // question may have been deleted in an out of order op that is not replayed. // See testRecoverFromStoreWithOutOfOrderDelete for an example of local recovery // See testRecoveryWithOutOfOrderDelete for an example of peer recovery - opVsLucene = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; - } else { - opVsLucene = compareOpToLuceneDocBasedOnSeqNo(index); - } - if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { plan = IndexingStrategy.processButSkipLucene(false, index.seqNo(), index.version()); } else { - plan = IndexingStrategy.processNormally( - opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, index.seqNo(), index.version() - ); + final OpVsLuceneDocStatus opVsLucene = compareOpToLuceneDocBasedOnSeqNo(index); + if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { + plan = IndexingStrategy.processAsStaleOp(softDeleteEnabled, index.seqNo(), index.version()); + } else { + plan = IndexingStrategy.processNormally(opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, + index.seqNo(), index.version()); + } } } return plan; @@ -921,7 +962,7 @@ public class InternalEngine extends Engine { throws IOException { assert plan.seqNoForIndexing >= 0 : "ops should have an assigned seq no.; origin: " + index.origin(); assert plan.versionForIndexing >= 0 : "version must be set. got " + plan.versionForIndexing; - assert plan.indexIntoLucene; + assert plan.indexIntoLucene || plan.addStaleOpToLucene; /* Update the document's sequence number and primary term; the sequence number here is derived here from either the sequence * number service if this is on the primary, or the existing document's sequence number if this is on the replica. The * primary term here has already been set, see IndexShard#prepareIndex where the Engine$Index operation is created. @@ -929,7 +970,9 @@ public class InternalEngine extends Engine { index.parsedDoc().updateSeqID(plan.seqNoForIndexing, index.primaryTerm()); index.parsedDoc().version().setLongValue(plan.versionForIndexing); try { - if (plan.useLuceneUpdateDocument) { + if (plan.addStaleOpToLucene) { + addStaleDocs(index.docs(), indexWriter); + } else if (plan.useLuceneUpdateDocument) { updateDocs(index.uid(), index.docs(), indexWriter); } else { // document does not exists, we can optimize for create, but double check if assertions are running @@ -993,16 +1036,29 @@ public class InternalEngine extends Engine { numDocAppends.inc(docs.size()); } - private static final class IndexingStrategy { + private void addStaleDocs(final List docs, final IndexWriter indexWriter) throws IOException { + assert softDeleteEnabled : "Add history documents but soft-deletes is disabled"; + for (ParseContext.Document doc : docs) { + doc.add(softDeletesField); // soft-deleted every document before adding to Lucene + } + if (docs.size() > 1) { + indexWriter.addDocuments(docs); + } else { + indexWriter.addDocument(docs.get(0)); + } + } + + protected static final class IndexingStrategy { final boolean currentNotFoundOrDeleted; final boolean useLuceneUpdateDocument; final long seqNoForIndexing; final long versionForIndexing; final boolean indexIntoLucene; + final boolean addStaleOpToLucene; final Optional earlyResultOnPreFlightError; private IndexingStrategy(boolean currentNotFoundOrDeleted, boolean useLuceneUpdateDocument, - boolean indexIntoLucene, long seqNoForIndexing, + boolean indexIntoLucene, boolean addStaleOpToLucene, long seqNoForIndexing, long versionForIndexing, IndexResult earlyResultOnPreFlightError) { assert useLuceneUpdateDocument == false || indexIntoLucene : "use lucene update is set to true, but we're not indexing into lucene"; @@ -1015,37 +1071,40 @@ public class InternalEngine extends Engine { this.seqNoForIndexing = seqNoForIndexing; this.versionForIndexing = versionForIndexing; this.indexIntoLucene = indexIntoLucene; + this.addStaleOpToLucene = addStaleOpToLucene; this.earlyResultOnPreFlightError = earlyResultOnPreFlightError == null ? Optional.empty() : Optional.of(earlyResultOnPreFlightError); } static IndexingStrategy optimizedAppendOnly(long seqNoForIndexing) { - return new IndexingStrategy(true, false, true, seqNoForIndexing, 1, null); + return new IndexingStrategy(true, false, true, false, seqNoForIndexing, 1, null); } static IndexingStrategy skipDueToVersionConflict( VersionConflictEngineException e, boolean currentNotFoundOrDeleted, long currentVersion, long term) { final IndexResult result = new IndexResult(e, currentVersion, term); return new IndexingStrategy( - currentNotFoundOrDeleted, false, false, SequenceNumbers.UNASSIGNED_SEQ_NO, Versions.NOT_FOUND, result); + currentNotFoundOrDeleted, false, false, false, SequenceNumbers.UNASSIGNED_SEQ_NO, Versions.NOT_FOUND, result); } static IndexingStrategy processNormally(boolean currentNotFoundOrDeleted, long seqNoForIndexing, long versionForIndexing) { return new IndexingStrategy(currentNotFoundOrDeleted, currentNotFoundOrDeleted == false, - true, seqNoForIndexing, versionForIndexing, null); + true, false, seqNoForIndexing, versionForIndexing, null); } static IndexingStrategy overrideExistingAsIfNotThere( long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(true, true, true, seqNoForIndexing, versionForIndexing, null); + return new IndexingStrategy(true, true, true, false, seqNoForIndexing, versionForIndexing, null); } - static IndexingStrategy processButSkipLucene(boolean currentNotFoundOrDeleted, - long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(currentNotFoundOrDeleted, false, - false, seqNoForIndexing, versionForIndexing, null); + static IndexingStrategy processButSkipLucene(boolean currentNotFoundOrDeleted, long seqNoForIndexing, long versionForIndexing) { + return new IndexingStrategy(currentNotFoundOrDeleted, false, false, false, seqNoForIndexing, versionForIndexing, null); + } + + static IndexingStrategy processAsStaleOp(boolean addStaleOpToLucene, long seqNoForIndexing, long versionForIndexing) { + return new IndexingStrategy(false, false, false, addStaleOpToLucene, seqNoForIndexing, versionForIndexing, null); } } @@ -1072,10 +1131,18 @@ public class InternalEngine extends Engine { } private void updateDocs(final Term uid, final List docs, final IndexWriter indexWriter) throws IOException { - if (docs.size() > 1) { - indexWriter.updateDocuments(uid, docs); + if (softDeleteEnabled) { + if (docs.size() > 1) { + indexWriter.softUpdateDocuments(uid, docs, softDeletesField); + } else { + indexWriter.softUpdateDocument(uid, docs.get(0), softDeletesField); + } } else { - indexWriter.updateDocument(uid, docs.get(0)); + if (docs.size() > 1) { + indexWriter.updateDocuments(uid, docs); + } else { + indexWriter.updateDocument(uid, docs.get(0)); + } } numDocUpdates.inc(docs.size()); } @@ -1099,7 +1166,7 @@ public class InternalEngine extends Engine { if (plan.earlyResultOnPreflightError.isPresent()) { deleteResult = plan.earlyResultOnPreflightError.get(); - } else if (plan.deleteFromLucene) { + } else if (plan.deleteFromLucene || plan.addStaleOpToLucene) { deleteResult = deleteInLucene(delete, plan); } else { deleteResult = new DeleteResult( @@ -1110,8 +1177,10 @@ public class InternalEngine extends Engine { if (deleteResult.getResultType() == Result.Type.SUCCESS) { location = translog.add(new Translog.Delete(delete, deleteResult)); } else if (deleteResult.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - location = translog.add(new Translog.NoOp(deleteResult.getSeqNo(), - delete.primaryTerm(), deleteResult.getFailure().toString())); + // if we have document failure, record it as a no-op in the translog and Lucene with the generated seq_no + final NoOp noOp = new NoOp(deleteResult.getSeqNo(), delete.primaryTerm(), delete.origin(), + delete.startTime(), deleteResult.getFailure().toString()); + location = innerNoOp(noOp).getTranslogLocation(); } else { location = null; } @@ -1142,7 +1211,7 @@ public class InternalEngine extends Engine { // unlike the primary, replicas don't really care to about found status of documents // this allows to ignore the case where a document was found in the live version maps in // a delete state and return true for the found flag in favor of code simplicity - final OpVsLuceneDocStatus opVsLucene; + final DeletionStrategy plan; if (delete.seqNo() <= localCheckpointTracker.getCheckpoint()) { // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already @@ -1151,18 +1220,15 @@ public class InternalEngine extends Engine { // question may have been deleted in an out of order op that is not replayed. // See testRecoverFromStoreWithOutOfOrderDelete for an example of local recovery // See testRecoveryWithOutOfOrderDelete for an example of peer recovery - opVsLucene = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; - } else { - opVsLucene = compareOpToLuceneDocBasedOnSeqNo(delete); - } - - final DeletionStrategy plan; - if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { plan = DeletionStrategy.processButSkipLucene(false, delete.seqNo(), delete.version()); } else { - plan = DeletionStrategy.processNormally( - opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, - delete.seqNo(), delete.version()); + final OpVsLuceneDocStatus opVsLucene = compareOpToLuceneDocBasedOnSeqNo(delete); + if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { + plan = DeletionStrategy.processAsStaleOp(softDeleteEnabled, false, delete.seqNo(), delete.version()); + } else { + plan = DeletionStrategy.processNormally(opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, + delete.seqNo(), delete.version()); + } } return plan; } @@ -1197,15 +1263,31 @@ public class InternalEngine extends Engine { private DeleteResult deleteInLucene(Delete delete, DeletionStrategy plan) throws IOException { try { - if (plan.currentlyDeleted == false) { + if (softDeleteEnabled) { + final ParsedDocument tombstone = engineConfig.getTombstoneDocSupplier().newDeleteTombstoneDoc(delete.type(), delete.id()); + assert tombstone.docs().size() == 1 : "Tombstone doc should have single doc [" + tombstone + "]"; + tombstone.updateSeqID(plan.seqNoOfDeletion, delete.primaryTerm()); + tombstone.version().setLongValue(plan.versionOfDeletion); + final ParseContext.Document doc = tombstone.docs().get(0); + assert doc.getField(SeqNoFieldMapper.TOMBSTONE_NAME) != null : + "Delete tombstone document but _tombstone field is not set [" + doc + " ]"; + doc.add(softDeletesField); + if (plan.addStaleOpToLucene || plan.currentlyDeleted) { + indexWriter.addDocument(doc); + } else { + indexWriter.softUpdateDocument(delete.uid(), doc, softDeletesField); + } + } else if (plan.currentlyDeleted == false) { // any exception that comes from this is a either an ACE or a fatal exception there // can't be any document failures coming from this indexWriter.deleteDocuments(delete.uid()); - numDocDeletes.inc(); } - versionMap.putDeleteUnderLock(delete.uid().bytes(), - new DeleteVersionValue(plan.versionOfDeletion, plan.seqNoOfDeletion, delete.primaryTerm(), - engineConfig.getThreadPool().relativeTimeInMillis())); + if (plan.deleteFromLucene) { + numDocDeletes.inc(); + versionMap.putDeleteUnderLock(delete.uid().bytes(), + new DeleteVersionValue(plan.versionOfDeletion, plan.seqNoOfDeletion, delete.primaryTerm(), + engineConfig.getThreadPool().relativeTimeInMillis())); + } return new DeleteResult( plan.versionOfDeletion, getPrimaryTerm(), plan.seqNoOfDeletion, plan.currentlyDeleted == false); } catch (Exception ex) { @@ -1219,15 +1301,16 @@ public class InternalEngine extends Engine { } } - private static final class DeletionStrategy { + protected static final class DeletionStrategy { // of a rare double delete final boolean deleteFromLucene; + final boolean addStaleOpToLucene; final boolean currentlyDeleted; final long seqNoOfDeletion; final long versionOfDeletion; final Optional earlyResultOnPreflightError; - private DeletionStrategy(boolean deleteFromLucene, boolean currentlyDeleted, + private DeletionStrategy(boolean deleteFromLucene, boolean addStaleOpToLucene, boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion, DeleteResult earlyResultOnPreflightError) { assert (deleteFromLucene && earlyResultOnPreflightError != null) == false : @@ -1235,6 +1318,7 @@ public class InternalEngine extends Engine { "deleteFromLucene: " + deleteFromLucene + " earlyResultOnPreFlightError:" + earlyResultOnPreflightError; this.deleteFromLucene = deleteFromLucene; + this.addStaleOpToLucene = addStaleOpToLucene; this.currentlyDeleted = currentlyDeleted; this.seqNoOfDeletion = seqNoOfDeletion; this.versionOfDeletion = versionOfDeletion; @@ -1246,16 +1330,22 @@ public class InternalEngine extends Engine { VersionConflictEngineException e, long currentVersion, long term, boolean currentlyDeleted) { final long unassignedSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; final DeleteResult deleteResult = new DeleteResult(e, currentVersion, term, unassignedSeqNo, currentlyDeleted == false); - return new DeletionStrategy(false, currentlyDeleted, unassignedSeqNo, Versions.NOT_FOUND, deleteResult); + return new DeletionStrategy(false, false, currentlyDeleted, unassignedSeqNo, Versions.NOT_FOUND, deleteResult); } static DeletionStrategy processNormally(boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(true, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); + return new DeletionStrategy(true, false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); } - public static DeletionStrategy processButSkipLucene(boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); + public static DeletionStrategy processButSkipLucene(boolean currentlyDeleted, + long seqNoOfDeletion, long versionOfDeletion) { + return new DeletionStrategy(false, false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); + } + + static DeletionStrategy processAsStaleOp(boolean addStaleOpToLucene, boolean currentlyDeleted, + long seqNoOfDeletion, long versionOfDeletion) { + return new DeletionStrategy(false, addStaleOpToLucene, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); } } @@ -1284,7 +1374,28 @@ public class InternalEngine extends Engine { assert noOp.seqNo() > SequenceNumbers.NO_OPS_PERFORMED; final long seqNo = noOp.seqNo(); try { - final NoOpResult noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo()); + Exception failure = null; + if (softDeleteEnabled) { + try { + final ParsedDocument tombstone = engineConfig.getTombstoneDocSupplier().newNoopTombstoneDoc(noOp.reason()); + tombstone.updateSeqID(noOp.seqNo(), noOp.primaryTerm()); + // A noop tombstone does not require a _version but it's added to have a fully dense docvalues for the version field. + // 1L is selected to optimize the compression because it might probably be the most common value in version field. + tombstone.version().setLongValue(1L); + assert tombstone.docs().size() == 1 : "Tombstone should have a single doc [" + tombstone + "]"; + final ParseContext.Document doc = tombstone.docs().get(0); + assert doc.getField(SeqNoFieldMapper.TOMBSTONE_NAME) != null + : "Noop tombstone document but _tombstone field is not set [" + doc + " ]"; + doc.add(softDeletesField); + indexWriter.addDocument(doc); + } catch (Exception ex) { + if (maybeFailEngine("noop", ex)) { + throw ex; + } + failure = ex; + } + } + final NoOpResult noOpResult = failure != null ? new NoOpResult(getPrimaryTerm(), noOp.seqNo(), failure) : new NoOpResult(getPrimaryTerm(), noOp.seqNo()); if (noOp.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { final Translog.Location location = translog.add(new Translog.NoOp(noOp.seqNo(), noOp.primaryTerm(), noOp.reason())); noOpResult.setTranslogLocation(location); @@ -1309,6 +1420,7 @@ public class InternalEngine extends Engine { // since it flushes the index as well (though, in terms of concurrency, we are allowed to do it) // both refresh types will result in an internal refresh but only the external will also // pass the new reader reference to the external reader manager. + final long localCheckpointBeforeRefresh = getLocalCheckpoint(); // this will also cause version map ram to be freed hence we always account for it. final long bytes = indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh(); @@ -1334,6 +1446,7 @@ public class InternalEngine extends Engine { } finally { store.decRef(); } + lastRefreshedCheckpointListener.updateRefreshedCheckpoint(localCheckpointBeforeRefresh); } } catch (AlreadyClosedException e) { failOnTragicEvent(e); @@ -1348,7 +1461,8 @@ public class InternalEngine extends Engine { } finally { writingBytes.addAndGet(-bytes); } - + assert lastRefreshedCheckpoint() >= localCheckpointBeforeRefresh : "refresh checkpoint was not advanced; " + + "local_checkpoint=" + localCheckpointBeforeRefresh + " refresh_checkpoint=" + lastRefreshedCheckpoint(); // TODO: maybe we should just put a scheduled job in threadPool? // We check for pruning in each delete request, but we also prune here e.g. in case a delete burst comes in and then no more deletes // for a long time: @@ -1930,7 +2044,11 @@ public class InternalEngine extends Engine { // pkg-private for testing IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOException { - return new IndexWriter(directory, iwc); + if (Assertions.ENABLED) { + return new AssertingIndexWriter(directory, iwc); + } else { + return new IndexWriter(directory, iwc); + } } private IndexWriterConfig getIndexWriterConfig() { @@ -1946,11 +2064,15 @@ public class InternalEngine extends Engine { } iwc.setInfoStream(verbose ? InfoStream.getDefault() : new LoggerInfoStream(logger)); iwc.setMergeScheduler(mergeScheduler); - MergePolicy mergePolicy = config().getMergePolicy(); // Give us the opportunity to upgrade old segments while performing // background merges - mergePolicy = new ElasticsearchMergePolicy(mergePolicy); - iwc.setMergePolicy(mergePolicy); + MergePolicy mergePolicy = config().getMergePolicy(); + if (softDeleteEnabled) { + iwc.setSoftDeletesField(Lucene.SOFT_DELETES_FIELD); + mergePolicy = new RecoverySourcePruneMergePolicy(SourceFieldMapper.RECOVERY_SOURCE_NAME, softDeletesPolicy::getRetentionQuery, + new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, softDeletesPolicy::getRetentionQuery, mergePolicy)); + } + iwc.setMergePolicy(new ElasticsearchMergePolicy(mergePolicy)); iwc.setSimilarity(engineConfig.getSimilarity()); iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().getMbFrac()); iwc.setCodec(engineConfig.getCodec()); @@ -2147,6 +2269,9 @@ public class InternalEngine extends Engine { commitData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(localCheckpointTracker.getMaxSeqNo())); commitData.put(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, Long.toString(maxUnsafeAutoIdTimestamp.get())); commitData.put(HISTORY_UUID_KEY, historyUUID); + if (softDeleteEnabled) { + commitData.put(Engine.MIN_RETAINED_SEQNO, Long.toString(softDeletesPolicy.getMinRetainedSeqNo())); + } logger.trace("committing writer with commit data [{}]", commitData); return commitData.entrySet().iterator(); }); @@ -2202,6 +2327,7 @@ public class InternalEngine extends Engine { final IndexSettings indexSettings = engineConfig.getIndexSettings(); translogDeletionPolicy.setRetentionAgeInMillis(indexSettings.getTranslogRetentionAge().getMillis()); translogDeletionPolicy.setRetentionSizeInBytes(indexSettings.getTranslogRetentionSize().getBytes()); + softDeletesPolicy.setRetentionOperations(indexSettings.getSoftDeleteRetentionOperations()); } public MergeStats getMergeStats() { @@ -2296,6 +2422,69 @@ public class InternalEngine extends Engine { return numDocUpdates.count(); } + @Override + public Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, + long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException { + // TODO: Should we defer the refresh until we really need it? + ensureOpen(); + if (lastRefreshedCheckpoint() < toSeqNo) { + refresh(source, SearcherScope.INTERNAL); + } + Searcher searcher = acquireSearcher(source, SearcherScope.INTERNAL); + try { + LuceneChangesSnapshot snapshot = new LuceneChangesSnapshot( + searcher, mapperService, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE, fromSeqNo, toSeqNo, requiredFullRange); + searcher = null; + return snapshot; + } catch (Exception e) { + try { + maybeFailEngine("acquire changes snapshot", e); + } catch (Exception inner) { + e.addSuppressed(inner); + } + throw e; + } finally { + IOUtils.close(searcher); + } + } + + @Override + public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException { + if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + return getMinRetainedSeqNo() <= startingSeqNo; + } else { + final long currentLocalCheckpoint = getLocalCheckpointTracker().getCheckpoint(); + final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1); + try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(startingSeqNo)) { + Translog.Operation operation; + while ((operation = snapshot.next()) != null) { + if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { + tracker.markSeqNoAsCompleted(operation.seqNo()); + } + } + } + return tracker.getCheckpoint() >= currentLocalCheckpoint; + } + } + + /** + * Returns the minimum seqno that is retained in the Lucene index. + * Operations whose seq# are at least this value should exist in the Lucene index. + */ + final long getMinRetainedSeqNo() { + assert softDeleteEnabled : Thread.currentThread().getName(); + return softDeletesPolicy.getMinRetainedSeqNo(); + } + + @Override + public Closeable acquireRetentionLockForPeerRecovery() { + if (softDeleteEnabled) { + return softDeletesPolicy.acquireRetentionLock(); + } else { + return translog.acquireRetentionLock(); + } + } + @Override public boolean isRecovering() { return pendingTranslogRecovery.get(); @@ -2311,4 +2500,69 @@ public class InternalEngine extends Engine { } return commitData; } + + private final class AssertingIndexWriter extends IndexWriter { + AssertingIndexWriter(Directory d, IndexWriterConfig conf) throws IOException { + super(d, conf); + } + @Override + public long updateDocument(Term term, Iterable doc) throws IOException { + assert softDeleteEnabled == false : "Call #updateDocument but soft-deletes is enabled"; + return super.updateDocument(term, doc); + } + @Override + public long updateDocuments(Term delTerm, Iterable> docs) throws IOException { + assert softDeleteEnabled == false : "Call #updateDocuments but soft-deletes is enabled"; + return super.updateDocuments(delTerm, docs); + } + @Override + public long deleteDocuments(Term... terms) throws IOException { + assert softDeleteEnabled == false : "Call #deleteDocuments but soft-deletes is enabled"; + return super.deleteDocuments(terms); + } + @Override + public long softUpdateDocument(Term term, Iterable doc, Field... softDeletes) throws IOException { + assert softDeleteEnabled : "Call #softUpdateDocument but soft-deletes is disabled"; + return super.softUpdateDocument(term, doc, softDeletes); + } + @Override + public long softUpdateDocuments(Term term, Iterable> docs, Field... softDeletes) throws IOException { + assert softDeleteEnabled : "Call #softUpdateDocuments but soft-deletes is disabled"; + return super.softUpdateDocuments(term, docs, softDeletes); + } + } + + /** + * Returned the last local checkpoint value has been refreshed internally. + */ + final long lastRefreshedCheckpoint() { + return lastRefreshedCheckpointListener.refreshedCheckpoint.get(); + } + + private final class LastRefreshedCheckpointListener implements ReferenceManager.RefreshListener { + final AtomicLong refreshedCheckpoint; + private long pendingCheckpoint; + + LastRefreshedCheckpointListener(long initialLocalCheckpoint) { + this.refreshedCheckpoint = new AtomicLong(initialLocalCheckpoint); + } + + @Override + public void beforeRefresh() { + // all changes until this point should be visible after refresh + pendingCheckpoint = localCheckpointTracker.getCheckpoint(); + } + + @Override + public void afterRefresh(boolean didRefresh) { + if (didRefresh) { + updateRefreshedCheckpoint(pendingCheckpoint); + } + } + + void updateRefreshedCheckpoint(long checkpoint) { + refreshedCheckpoint.updateAndGet(curr -> Math.max(curr, checkpoint)); + assert refreshedCheckpoint.get() >= checkpoint : refreshedCheckpoint.get() + " < " + checkpoint; + } + } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java new file mode 100644 index 00000000000..deebfba9ed4 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -0,0 +1,368 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +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.index.Term; +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.util.ArrayUtil; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.index.fieldvisitor.FieldsVisitor; +import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.VersionFieldMapper; +import org.elasticsearch.index.translog.Translog; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +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; + private long lastSeenSeqNo; + private int skippedOperations; + private final boolean requiredFullRange; + + private final IndexSearcher indexSearcher; + private final MapperService mapperService; + private int docIndex = 0; + private final int totalHits; + private ScoreDoc[] scoreDocs; + private final ParallelArray parallelArray; + private final Closeable onClose; + + /** + * Creates a new "translog" snapshot from Lucene for reading operations whose seq# in the specified range. + * + * @param engineSearcher the internal engine searcher which will be taken over if the snapshot is opened successfully + * @param mapperService the mapper service which will be mainly used to resolve the document's type and uid + * @param searchBatchSize the number of documents should be returned by each search + * @param fromSeqNo the min requesting seq# - inclusive + * @param toSeqNo the maximum requesting seq# - inclusive + * @param requiredFullRange if true, the snapshot will strictly check for the existence of operations between fromSeqNo and toSeqNo + */ + LuceneChangesSnapshot(Engine.Searcher engineSearcher, MapperService mapperService, int searchBatchSize, + long fromSeqNo, long toSeqNo, boolean requiredFullRange) 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.mapperService = mapperService; + this.searchBatchSize = searchBatchSize; + this.fromSeqNo = fromSeqNo; + this.toSeqNo = toSeqNo; + this.lastSeenSeqNo = fromSeqNo - 1; + this.requiredFullRange = requiredFullRange; + this.indexSearcher = new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); + this.indexSearcher.setQueryCache(null); + this.parallelArray = new ParallelArray(searchBatchSize); + final TopDocs topDocs = searchOperations(null); + this.totalHits = Math.toIntExact(topDocs.totalHits); + this.scoreDocs = topDocs.scoreDocs; + fillParallelArray(scoreDocs, parallelArray); + } + + @Override + public void close() throws IOException { + onClose.close(); + } + + @Override + public int totalOperations() { + return totalHits; + } + + @Override + public int skippedOperations() { + return skippedOperations; + } + + @Override + public Translog.Operation next() throws IOException { + Translog.Operation op = null; + for (int idx = nextDocIndex(); idx != -1; idx = nextDocIndex()) { + op = readDocAsOp(idx); + if (op != null) { + break; + } + } + if (requiredFullRange) { + rangeCheck(op); + } + if (op != null) { + lastSeenSeqNo = op.seqNo(); + } + return op; + } + + private void rangeCheck(Translog.Operation op) { + if (op == null) { + if (lastSeenSeqNo < toSeqNo) { + throw new IllegalStateException("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 IllegalStateException("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(prev).scoreDocs; + fillParallelArray(scoreDocs, parallelArray); + docIndex = 0; + } + if (docIndex < scoreDocs.length) { + int idx = docIndex; + docIndex++; + return idx; + } + return -1; + } + + private void fillParallelArray(ScoreDoc[] scoreDocs, ParallelArray parallelArray) throws IOException { + if (scoreDocs.length > 0) { + for (int i = 0; i < scoreDocs.length; i++) { + scoreDocs[i].shardIndex = i; + } + // for better loading performance we sort the array by docID and + // then visit all leaves in order. + ArrayUtil.introSort(scoreDocs, Comparator.comparingInt(i -> i.doc)); + int docBase = -1; + int maxDoc = 0; + List leaves = indexSearcher.getIndexReader().leaves(); + int readerIndex = 0; + CombinedDocValues combinedDocValues = null; + LeafReaderContext leaf = null; + for (int i = 0; i < scoreDocs.length; i++) { + ScoreDoc scoreDoc = scoreDocs[i]; + if (scoreDoc.doc >= docBase + maxDoc) { + do { + leaf = leaves.get(readerIndex++); + docBase = leaf.docBase; + maxDoc = leaf.reader().maxDoc(); + } while (scoreDoc.doc >= docBase + maxDoc); + combinedDocValues = new CombinedDocValues(leaf.reader()); + } + final int segmentDocID = scoreDoc.doc - docBase; + final int index = scoreDoc.shardIndex; + parallelArray.leafReaderContexts[index] = leaf; + parallelArray.seqNo[index] = combinedDocValues.docSeqNo(segmentDocID); + parallelArray.primaryTerm[index] = combinedDocValues.docPrimaryTerm(segmentDocID); + parallelArray.version[index] = combinedDocValues.docVersion(segmentDocID); + parallelArray.isTombStone[index] = combinedDocValues.isTombstone(segmentDocID); + parallelArray.hasRecoverySource[index] = combinedDocValues.hasRecoverySource(segmentDocID); + } + // now sort back based on the shardIndex. we use this to store the previous index + ArrayUtil.introSort(scoreDocs, Comparator.comparingInt(i -> i.shardIndex)); + } + } + + private TopDocs searchOperations(ScoreDoc after) throws IOException { + final Query rangeQuery = LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, lastSeenSeqNo + 1, toSeqNo); + final Sort sortedBySeqNoThenByTerm = new Sort( + new SortField(SeqNoFieldMapper.NAME, SortField.Type.LONG), + new SortField(SeqNoFieldMapper.PRIMARY_TERM_NAME, SortField.Type.LONG, true) + ); + return indexSearcher.searchAfter(after, rangeQuery, searchBatchSize, sortedBySeqNoThenByTerm); + } + + private Translog.Operation readDocAsOp(int docIndex) throws IOException { + final LeafReaderContext leaf = parallelArray.leafReaderContexts[docIndex]; + final int segmentDocID = scoreDocs[docIndex].doc - leaf.docBase; + final long primaryTerm = parallelArray.primaryTerm[docIndex]; + // We don't have to read the nested child documents - those docs don't have primary terms. + if (primaryTerm == -1) { + skippedOperations++; + return null; + } + final long seqNo = parallelArray.seqNo[docIndex]; + // Only pick the first seen seq# + if (seqNo == lastSeenSeqNo) { + skippedOperations++; + return null; + } + final long version = parallelArray.version[docIndex]; + final String sourceField = parallelArray.hasRecoverySource[docIndex] ? SourceFieldMapper.RECOVERY_SOURCE_NAME : + SourceFieldMapper.NAME; + final FieldsVisitor fields = new FieldsVisitor(true, sourceField); + leaf.reader().document(segmentDocID, fields); + fields.postProcess(mapperService); + + final Translog.Operation op; + final boolean isTombstone = parallelArray.isTombStone[docIndex]; + if (isTombstone && fields.uid() == null) { + op = new Translog.NoOp(seqNo, primaryTerm, fields.source().utf8ToString()); + assert version == 1L : "Noop tombstone should have version 1L; actual version [" + version + "]"; + assert assertDocSoftDeleted(leaf.reader(), segmentDocID) : "Noop but soft_deletes field is not set [" + op + "]"; + } else { + final String id = fields.uid().id(); + final String type = fields.uid().type(); + final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); + if (isTombstone) { + op = new Translog.Delete(type, id, uid, seqNo, primaryTerm, version); + assert assertDocSoftDeleted(leaf.reader(), segmentDocID) : "Delete op but soft_deletes field is not set [" + op + "]"; + } else { + final BytesReference source = fields.source(); + if (source == null) { + // 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 IllegalStateException("source not found for seqno=" + seqNo + + " from_seqno=" + fromSeqNo + " to_seqno=" + toSeqNo); + } else { + skippedOperations++; + return null; + } + } + // TODO: pass the latest timestamp from engine. + final long autoGeneratedIdTimestamp = -1; + op = new Translog.Index(type, id, seqNo, primaryTerm, version, + source.toBytesRef().bytes, fields.routing(), autoGeneratedIdTimestamp); + } + } + assert fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo && lastSeenSeqNo < op.seqNo() : "Unexpected operation; " + + "last_seen_seqno [" + lastSeenSeqNo + "], from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "], op [" + op + "]"; + return op; + } + + private 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 long[] version; + final long[] seqNo; + final long[] primaryTerm; + final boolean[] isTombStone; + final boolean[] hasRecoverySource; + + ParallelArray(int size) { + version = new long[size]; + seqNo = new long[size]; + primaryTerm = new long[size]; + isTombStone = new boolean[size]; + hasRecoverySource = new boolean[size]; + leafReaderContexts = new LeafReaderContext[size]; + } + } + + private static final class CombinedDocValues { + private final NumericDocValues versionDV; + private final NumericDocValues seqNoDV; + private final NumericDocValues primaryTermDV; + private final NumericDocValues tombstoneDV; + private final NumericDocValues recoverySource; + + CombinedDocValues(LeafReader leafReader) throws IOException { + this.versionDV = Objects.requireNonNull(leafReader.getNumericDocValues(VersionFieldMapper.NAME), "VersionDV is missing"); + this.seqNoDV = Objects.requireNonNull(leafReader.getNumericDocValues(SeqNoFieldMapper.NAME), "SeqNoDV is missing"); + this.primaryTermDV = Objects.requireNonNull( + leafReader.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME), "PrimaryTermDV is missing"); + this.tombstoneDV = leafReader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); + this.recoverySource = leafReader.getNumericDocValues(SourceFieldMapper.RECOVERY_SOURCE_NAME); + } + + long docVersion(int segmentDocId) throws IOException { + assert versionDV.docID() < segmentDocId; + if (versionDV.advanceExact(segmentDocId) == false) { + throw new IllegalStateException("DocValues for field [" + VersionFieldMapper.NAME + "] is not found"); + } + return versionDV.longValue(); + } + + long docSeqNo(int segmentDocId) throws IOException { + assert seqNoDV.docID() < segmentDocId; + if (seqNoDV.advanceExact(segmentDocId) == false) { + throw new IllegalStateException("DocValues for field [" + SeqNoFieldMapper.NAME + "] is not found"); + } + return seqNoDV.longValue(); + } + + long docPrimaryTerm(int segmentDocId) throws IOException { + if (primaryTermDV == null) { + return -1L; + } + assert primaryTermDV.docID() < segmentDocId; + // Use -1 for docs which don't have primary term. The caller considers those docs as nested docs. + if (primaryTermDV.advanceExact(segmentDocId) == false) { + return -1; + } + return primaryTermDV.longValue(); + } + + boolean isTombstone(int segmentDocId) throws IOException { + if (tombstoneDV == null) { + return false; + } + assert tombstoneDV.docID() < segmentDocId; + return tombstoneDV.advanceExact(segmentDocId) && tombstoneDV.longValue() > 0; + } + + boolean hasRecoverySource(int segmentDocId) throws IOException { + if (recoverySource == null) { + return false; + } + assert recoverySource.docID() < segmentDocId; + return recoverySource.advanceExact(segmentDocId); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java b/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java new file mode 100644 index 00000000000..fde97562de8 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java @@ -0,0 +1,292 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.StoredFieldsReader; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.CodecReader; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FilterCodecReader; +import org.apache.lucene.index.FilterNumericDocValues; +import org.apache.lucene.index.MergePolicy; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.OneMergeWrappingMergePolicy; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.index.StoredFieldVisitor; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.ConjunctionDISI; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.DocValuesFieldExistsQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.apache.lucene.util.BitSet; +import org.apache.lucene.util.BitSetIterator; + +import java.io.IOException; +import java.util.Arrays; +import java.util.function.Supplier; + +final class RecoverySourcePruneMergePolicy extends OneMergeWrappingMergePolicy { + RecoverySourcePruneMergePolicy(String recoverySourceField, Supplier retainSourceQuerySupplier, MergePolicy in) { + super(in, toWrap -> new OneMerge(toWrap.segments) { + @Override + public CodecReader wrapForMerge(CodecReader reader) throws IOException { + CodecReader wrapped = toWrap.wrapForMerge(reader); + return wrapReader(recoverySourceField, wrapped, retainSourceQuerySupplier); + } + }); + } + + // pkg private for testing + static CodecReader wrapReader(String recoverySourceField, CodecReader reader, Supplier retainSourceQuerySupplier) + throws IOException { + NumericDocValues recoverySource = reader.getNumericDocValues(recoverySourceField); + 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. + } + BooleanQuery.Builder builder = new BooleanQuery.Builder(); + builder.add(new DocValuesFieldExistsQuery(recoverySourceField), BooleanClause.Occur.FILTER); + builder.add(retainSourceQuerySupplier.get(), BooleanClause.Occur.FILTER); + IndexSearcher s = new IndexSearcher(reader); + s.setQueryCache(null); + Weight weight = s.createWeight(s.rewrite(builder.build()), false, 1.0f); + Scorer scorer = weight.scorer(reader.getContext()); + if (scorer != null) { + return new SourcePruningFilterCodecReader(recoverySourceField, reader, BitSet.of(scorer.iterator(), reader.maxDoc())); + } else { + return new SourcePruningFilterCodecReader(recoverySourceField, reader, null); + } + } + + private static class SourcePruningFilterCodecReader extends FilterCodecReader { + private final BitSet recoverySourceToKeep; + private final String recoverySourceField; + + SourcePruningFilterCodecReader(String recoverySourceField, CodecReader reader, BitSet recoverySourceToKeep) { + super(reader); + this.recoverySourceField = recoverySourceField; + this.recoverySourceToKeep = recoverySourceToKeep; + } + + @Override + public DocValuesProducer getDocValuesReader() { + DocValuesProducer docValuesReader = super.getDocValuesReader(); + return new FilterDocValuesProducer(docValuesReader) { + @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"; + final DocIdSetIterator intersection; + if (recoverySourceToKeep == null) { + // we can't return null here lucenes DocIdMerger expects an instance + intersection = DocIdSetIterator.empty(); + } else { + intersection = ConjunctionDISI.intersectIterators(Arrays.asList(numeric, + new BitSetIterator(recoverySourceToKeep, recoverySourceToKeep.length()))); + } + return new FilterNumericDocValues(numeric) { + @Override + public int nextDoc() throws IOException { + return intersection.nextDoc(); + } + + @Override + public int advance(int target) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean advanceExact(int target) { + throw new UnsupportedOperationException(); + } + }; + + } + return numeric; + } + }; + } + + @Override + public StoredFieldsReader getFieldsReader() { + StoredFieldsReader fieldsReader = super.getFieldsReader(); + return new FilterStoredFieldsReader(fieldsReader) { + @Override + public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException { + if (recoverySourceToKeep != null && recoverySourceToKeep.get(docID)) { + super.visitDocument(docID, visitor); + } else { + super.visitDocument(docID, new FilterStoredFieldVisitor(visitor) { + @Override + public Status needsField(FieldInfo fieldInfo) throws IOException { + if (recoverySourceField.equals(fieldInfo.name)) { + return Status.NO; + } + return super.needsField(fieldInfo); + } + }); + } + } + }; + } + + @Override + public CacheHelper getCoreCacheHelper() { + return null; + } + + @Override + public CacheHelper getReaderCacheHelper() { + return null; + } + + private static class FilterDocValuesProducer extends DocValuesProducer { + private final DocValuesProducer in; + + FilterDocValuesProducer(DocValuesProducer in) { + this.in = in; + } + + @Override + public NumericDocValues getNumeric(FieldInfo field) throws IOException { + return in.getNumeric(field); + } + + @Override + public BinaryDocValues getBinary(FieldInfo field) throws IOException { + return in.getBinary(field); + } + + @Override + public SortedDocValues getSorted(FieldInfo field) throws IOException { + return in.getSorted(field); + } + + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + return in.getSortedNumeric(field); + } + + @Override + public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { + return in.getSortedSet(field); + } + + @Override + public void checkIntegrity() throws IOException { + in.checkIntegrity(); + } + + @Override + public void close() throws IOException { + in.close(); + } + + @Override + public long ramBytesUsed() { + return in.ramBytesUsed(); + } + } + + private static class FilterStoredFieldsReader extends StoredFieldsReader { + + private final StoredFieldsReader fieldsReader; + + FilterStoredFieldsReader(StoredFieldsReader fieldsReader) { + this.fieldsReader = fieldsReader; + } + + @Override + public long ramBytesUsed() { + return fieldsReader.ramBytesUsed(); + } + + @Override + public void close() throws IOException { + fieldsReader.close(); + } + + @Override + public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException { + fieldsReader.visitDocument(docID, visitor); + } + + @Override + public StoredFieldsReader clone() { + return fieldsReader.clone(); + } + + @Override + public void checkIntegrity() throws IOException { + fieldsReader.checkIntegrity(); + } + } + + private static class FilterStoredFieldVisitor extends StoredFieldVisitor { + private final StoredFieldVisitor visitor; + + FilterStoredFieldVisitor(StoredFieldVisitor visitor) { + this.visitor = visitor; + } + + @Override + public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { + visitor.binaryField(fieldInfo, value); + } + + @Override + public void stringField(FieldInfo fieldInfo, byte[] value) throws IOException { + visitor.stringField(fieldInfo, value); + } + + @Override + public void intField(FieldInfo fieldInfo, int value) throws IOException { + visitor.intField(fieldInfo, value); + } + + @Override + public void longField(FieldInfo fieldInfo, long value) throws IOException { + visitor.longField(fieldInfo, value); + } + + @Override + public void floatField(FieldInfo fieldInfo, float value) throws IOException { + visitor.floatField(fieldInfo, value); + } + + @Override + public void doubleField(FieldInfo fieldInfo, double value) throws IOException { + visitor.doubleField(fieldInfo, value); + } + + @Override + public Status needsField(FieldInfo fieldInfo) throws IOException { + return visitor.needsField(fieldInfo); + } + } + } +} diff --git a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java new file mode 100644 index 00000000000..af2ded8c466 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java @@ -0,0 +1,120 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.search.Query; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.translog.Translog; + +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.LongSupplier; + +/** + * A policy that controls how many soft-deleted documents should be retained for peer-recovery and querying history changes purpose. + */ +final class SoftDeletesPolicy { + private final LongSupplier globalCheckpointSupplier; + private long localCheckpointOfSafeCommit; + // This lock count is used to prevent `minRetainedSeqNo` from advancing. + private int retentionLockCount; + // The extra number of operations before the global checkpoint are retained + private long retentionOperations; + // The min seq_no value that is retained - ops after this seq# should exist in the Lucene index. + private long minRetainedSeqNo; + + SoftDeletesPolicy(LongSupplier globalCheckpointSupplier, long minRetainedSeqNo, long retentionOperations) { + this.globalCheckpointSupplier = globalCheckpointSupplier; + this.retentionOperations = retentionOperations; + this.minRetainedSeqNo = minRetainedSeqNo; + this.localCheckpointOfSafeCommit = SequenceNumbers.NO_OPS_PERFORMED; + this.retentionLockCount = 0; + } + + /** + * Updates the number of soft-deleted documents prior to the global checkpoint to be retained + * See {@link org.elasticsearch.index.IndexSettings#INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING} + */ + synchronized void setRetentionOperations(long retentionOperations) { + this.retentionOperations = retentionOperations; + } + + /** + * Sets the local checkpoint of the current safe commit + */ + synchronized void setLocalCheckpointOfSafeCommit(long newCheckpoint) { + if (newCheckpoint < this.localCheckpointOfSafeCommit) { + throw new IllegalArgumentException("Local checkpoint can't go backwards; " + + "new checkpoint [" + newCheckpoint + "]," + "current checkpoint [" + localCheckpointOfSafeCommit + "]"); + } + this.localCheckpointOfSafeCommit = newCheckpoint; + } + + /** + * Acquires a lock on soft-deleted documents to prevent them from cleaning up in merge processes. This is necessary to + * make sure that all operations that are being retained will be retained until the lock is released. + * This is a analogy to the translog's retention lock; see {@link Translog#acquireRetentionLock()} + */ + synchronized Releasable acquireRetentionLock() { + assert retentionLockCount >= 0 : "Invalid number of retention locks [" + retentionLockCount + "]"; + retentionLockCount++; + final AtomicBoolean released = new AtomicBoolean(); + return () -> { + if (released.compareAndSet(false, true)) { + releaseRetentionLock(); + } + }; + } + + private synchronized void releaseRetentionLock() { + assert retentionLockCount > 0 : "Invalid number of retention locks [" + retentionLockCount + "]"; + retentionLockCount--; + } + + /** + * Returns the min seqno that is retained in the Lucene index. + * Operations whose seq# is least this value should exist in the Lucene index. + */ + synchronized long getMinRetainedSeqNo() { + // Do not advance if the retention lock is held + if (retentionLockCount == 0) { + // This policy retains operations for two purposes: peer-recovery and querying changes history. + // - Peer-recovery is driven by the local checkpoint of the safe commit. In peer-recovery, the primary transfers a safe commit, + // then sends ops after the local checkpoint of that commit. This requires keeping all ops after localCheckpointOfSafeCommit; + // - Changes APIs are driven the combination of the global checkpoint and retention ops. Here we prefer using the global + // checkpoint instead of max_seqno because only operations up to the global checkpoint are exposed in the the changes APIs. + final long minSeqNoForQueryingChanges = globalCheckpointSupplier.getAsLong() - retentionOperations; + final long minSeqNoToRetain = Math.min(minSeqNoForQueryingChanges, localCheckpointOfSafeCommit) + 1; + // This can go backward as the retentionOperations value can be changed in settings. + minRetainedSeqNo = Math.max(minRetainedSeqNo, minSeqNoToRetain); + } + return minRetainedSeqNo; + } + + /** + * Returns a soft-deletes retention query that will be used in {@link org.apache.lucene.index.SoftDeletesRetentionMergePolicy} + * Documents including tombstones are soft-deleted and matched this query will be retained and won't cleaned up by merges. + */ + Query getRetentionQuery() { + return LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, getMinRetainedSeqNo(), Long.MAX_VALUE); + } +} diff --git a/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java b/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java index 4c65635c61b..462f8ce8e68 100644 --- a/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java +++ b/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java @@ -54,13 +54,19 @@ public class FieldsVisitor extends StoredFieldVisitor { RoutingFieldMapper.NAME)); private final boolean loadSource; + private final String sourceFieldName; private final Set requiredFields; protected BytesReference source; protected String type, id; protected Map> fieldsValues; public FieldsVisitor(boolean loadSource) { + this(loadSource, SourceFieldMapper.NAME); + } + + public FieldsVisitor(boolean loadSource, String sourceFieldName) { this.loadSource = loadSource; + this.sourceFieldName = sourceFieldName; requiredFields = new HashSet<>(); reset(); } @@ -103,7 +109,7 @@ public class FieldsVisitor extends StoredFieldVisitor { @Override public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { - if (SourceFieldMapper.NAME.equals(fieldInfo.name)) { + if (sourceFieldName.equals(fieldInfo.name)) { source = new BytesArray(value); } else if (IdFieldMapper.NAME.equals(fieldInfo.name)) { id = Uid.decodeId(value); @@ -175,7 +181,7 @@ public class FieldsVisitor extends StoredFieldVisitor { requiredFields.addAll(BASE_REQUIRED_FIELDS); if (loadSource) { - requiredFields.add(SourceFieldMapper.NAME); + requiredFields.add(sourceFieldName); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index a0640ac68a9..663aa7e6f9e 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -19,11 +19,14 @@ package org.elasticsearch.index.mapper; +import org.apache.lucene.document.StoredField; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchGenerationException; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.text.Text; @@ -39,12 +42,15 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.stream.Stream; public class DocumentMapper implements ToXContentFragment { @@ -121,6 +127,8 @@ public class DocumentMapper implements ToXContentFragment { private final Map objectMappers; private final boolean hasNestedObjects; + private final MetadataFieldMapper[] deleteTombstoneMetadataFieldMappers; + private final MetadataFieldMapper[] noopTombstoneMetadataFieldMappers; public DocumentMapper(MapperService mapperService, Mapping mapping) { this.mapperService = mapperService; @@ -171,6 +179,15 @@ public class DocumentMapper implements ToXContentFragment { } catch (Exception e) { throw new ElasticsearchGenerationException("failed to serialize source for type [" + type + "]", e); } + + final Collection deleteTombstoneMetadataFields = Arrays.asList(VersionFieldMapper.NAME, IdFieldMapper.NAME, + TypeFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME); + this.deleteTombstoneMetadataFieldMappers = Stream.of(mapping.metadataMappers) + .filter(field -> deleteTombstoneMetadataFields.contains(field.name())).toArray(MetadataFieldMapper[]::new); + final Collection noopTombstoneMetadataFields = Arrays.asList( + VersionFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME); + this.noopTombstoneMetadataFieldMappers = Stream.of(mapping.metadataMappers) + .filter(field -> noopTombstoneMetadataFields.contains(field.name())).toArray(MetadataFieldMapper[]::new); } public Mapping mapping() { @@ -242,7 +259,22 @@ public class DocumentMapper implements ToXContentFragment { } public ParsedDocument parse(SourceToParse source) throws MapperParsingException { - return documentParser.parseDocument(source); + return documentParser.parseDocument(source, mapping.metadataMappers); + } + + public ParsedDocument createDeleteTombstoneDoc(String index, String type, String id) throws MapperParsingException { + final SourceToParse emptySource = SourceToParse.source(index, type, id, new BytesArray("{}"), XContentType.JSON); + return documentParser.parseDocument(emptySource, deleteTombstoneMetadataFieldMappers).toTombstone(); + } + + public ParsedDocument createNoopTombstoneDoc(String index, String reason) throws MapperParsingException { + final String id = ""; // _id won't be used. + final SourceToParse sourceToParse = SourceToParse.source(index, type, id, new BytesArray("{}"), XContentType.JSON); + final ParsedDocument parsedDoc = documentParser.parseDocument(sourceToParse, noopTombstoneMetadataFieldMappers).toTombstone(); + // Store the reason of a noop as a raw string in the _source field + final BytesRef byteRef = new BytesRef(reason); + parsedDoc.rootDoc().add(new StoredField(SourceFieldMapper.NAME, byteRef.bytes, byteRef.offset, byteRef.length)); + return parsedDoc; } /** diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java index 0fd156c0905..85123f602ed 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -55,7 +55,7 @@ final class DocumentParser { this.docMapper = docMapper; } - ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { + ParsedDocument parseDocument(SourceToParse source, MetadataFieldMapper[] metadataFieldsMappers) throws MapperParsingException { validateType(source); final Mapping mapping = docMapper.mapping(); @@ -64,9 +64,9 @@ final class DocumentParser { try (XContentParser parser = XContentHelper.createParser(docMapperParser.getXContentRegistry(), LoggingDeprecationHandler.INSTANCE, source.source(), xContentType)) { - context = new ParseContext.InternalParseContext(indexSettings.getSettings(), docMapperParser, docMapper, source, parser); + context = new ParseContext.InternalParseContext(indexSettings, docMapperParser, docMapper, source, parser); validateStart(parser); - internalParseDocument(mapping, context, parser); + internalParseDocument(mapping, metadataFieldsMappers, context, parser); validateEnd(parser); } catch (Exception e) { throw wrapInMapperParsingException(source, e); @@ -81,10 +81,11 @@ final class DocumentParser { return parsedDocument(source, context, createDynamicUpdate(mapping, docMapper, context.getDynamicMappers())); } - private static void internalParseDocument(Mapping mapping, ParseContext.InternalParseContext context, XContentParser parser) throws IOException { + private static void internalParseDocument(Mapping mapping, MetadataFieldMapper[] metadataFieldsMappers, + ParseContext.InternalParseContext context, XContentParser parser) throws IOException { final boolean emptyDoc = isEmptyDoc(mapping, parser); - for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { + for (MetadataFieldMapper metadataMapper : metadataFieldsMappers) { metadataMapper.preParse(context); } @@ -95,7 +96,7 @@ final class DocumentParser { parseObjectOrNested(context, mapping.root); } - for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { + for (MetadataFieldMapper metadataMapper : metadataFieldsMappers) { metadataMapper.postParse(context); } } @@ -495,7 +496,7 @@ final class DocumentParser { if (builder == null) { builder = new ObjectMapper.Builder(currentFieldName).enabled(true); } - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); objectMapper = builder.build(builderContext); context.addDynamicMapper(objectMapper); context.path().add(currentFieldName); @@ -538,7 +539,7 @@ final class DocumentParser { if (builder == null) { parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName); } else { - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); mapper = builder.build(builderContext); assert mapper != null; if (mapper instanceof ArrayValueMapperParser) { @@ -696,13 +697,13 @@ final class DocumentParser { if (parseableAsLong && context.root().numericDetection()) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG); if (builder == null) { - builder = newLongBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); + builder = newLongBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } else if (parseableAsDouble && context.root().numericDetection()) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DOUBLE); if (builder == null) { - builder = newFloatBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); + builder = newFloatBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } else if (parseableAsLong == false && parseableAsDouble == false && context.root().dateDetection()) { @@ -718,7 +719,7 @@ final class DocumentParser { } Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DATE); if (builder == null) { - builder = newDateBuilder(currentFieldName, dateTimeFormatter, Version.indexCreated(context.indexSettings())); + builder = newDateBuilder(currentFieldName, dateTimeFormatter, context.indexSettings().getIndexVersionCreated()); } if (builder instanceof DateFieldMapper.Builder) { DateFieldMapper.Builder dateBuilder = (DateFieldMapper.Builder) builder; @@ -741,7 +742,7 @@ final class DocumentParser { if (numberType == XContentParser.NumberType.INT || numberType == XContentParser.NumberType.LONG) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG); if (builder == null) { - builder = newLongBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); + builder = newLongBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } else if (numberType == XContentParser.NumberType.FLOAT || numberType == XContentParser.NumberType.DOUBLE) { @@ -750,7 +751,7 @@ final class DocumentParser { // no templates are defined, we use float by default instead of double // since this is much more space-efficient and should be enough most of // the time - builder = newFloatBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); + builder = newFloatBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } @@ -785,7 +786,7 @@ final class DocumentParser { return; } final String path = context.path().pathAsText(currentFieldName); - final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); final MappedFieldType existingFieldType = context.mapperService().fullName(path); final Mapper.Builder builder; if (existingFieldType != null) { @@ -883,8 +884,8 @@ final class DocumentParser { if (builder == null) { builder = new ObjectMapper.Builder(paths[i]).enabled(true); } - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); - mapper = (ObjectMapper) builder.build(builderContext); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), + context.path()); mapper = (ObjectMapper) builder.build(builderContext); if (mapper.nested() != ObjectMapper.Nested.NO) { throw new MapperParsingException("It is forbidden to create dynamic nested objects ([" + context.path().pathAsText(paths[i]) + "]) through `copy_to` or dots in field names"); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java index 606777392de..8389a306270 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java @@ -24,7 +24,6 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; import org.apache.lucene.search.Query; import org.elasticsearch.Version; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.lucene.Lucene; @@ -205,12 +204,12 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper { } @Override - public void preParse(ParseContext context) throws IOException { + public void preParse(ParseContext context) { } @Override public void postParse(ParseContext context) throws IOException { - if (context.indexSettings().getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).before(Version.V_6_1_0)) { + if (context.indexSettings().getIndexVersionCreated().before(Version.V_6_1_0)) { super.parse(context); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java index b77ffee05ca..cf8cc4022fd 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -24,9 +24,8 @@ import com.carrotsearch.hppc.ObjectObjectMap; import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexableField; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.IndexSettings; import java.util.ArrayList; import java.util.Collection; @@ -196,7 +195,7 @@ public abstract class ParseContext implements Iterable{ } @Override - public Settings indexSettings() { + public IndexSettings indexSettings() { return in.indexSettings(); } @@ -315,8 +314,7 @@ public abstract class ParseContext implements Iterable{ private final List documents; - @Nullable - private final Settings indexSettings; + private final IndexSettings indexSettings; private final SourceToParse sourceToParse; @@ -334,8 +332,8 @@ public abstract class ParseContext implements Iterable{ private final Set ignoredFields = new HashSet<>(); - public InternalParseContext(@Nullable Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, - SourceToParse source, XContentParser parser) { + public InternalParseContext(IndexSettings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, + SourceToParse source, XContentParser parser) { this.indexSettings = indexSettings; this.docMapper = docMapper; this.docMapperParser = docMapperParser; @@ -347,7 +345,7 @@ public abstract class ParseContext implements Iterable{ this.version = null; this.sourceToParse = source; this.dynamicMappers = new ArrayList<>(); - this.maxAllowedNumNestedDocs = MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING.get(indexSettings); + this.maxAllowedNumNestedDocs = indexSettings.getValue(MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING); this.numNestedDocs = 0L; } @@ -357,8 +355,7 @@ public abstract class ParseContext implements Iterable{ } @Override - @Nullable - public Settings indexSettings() { + public IndexSettings indexSettings() { return this.indexSettings; } @@ -565,8 +562,7 @@ public abstract class ParseContext implements Iterable{ return false; } - @Nullable - public abstract Settings indexSettings(); + public abstract IndexSettings indexSettings(); public abstract SourceToParse sourceToParse(); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java b/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java index 414cb3a98ec..d2cf17ddd35 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java @@ -83,6 +83,17 @@ public class ParsedDocument { this.seqID.primaryTerm.setLongValue(primaryTerm); } + /** + * Makes the processing document as a tombstone document rather than a regular document. + * Tombstone documents are stored in Lucene index to represent delete operations or Noops. + */ + ParsedDocument toTombstone() { + assert docs().size() == 1 : "Tombstone should have a single doc [" + docs() + "]"; + this.seqID.tombstoneField.setLongValue(1); + rootDoc().add(this.seqID.tombstoneField); + return this; + } + public String routing() { return this.routing; } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java index ac3ffe46272..5a0db4163bf 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java @@ -69,26 +69,29 @@ public class SeqNoFieldMapper extends MetadataFieldMapper { public final Field seqNo; public final Field seqNoDocValue; public final Field primaryTerm; + public final Field tombstoneField; - public SequenceIDFields(Field seqNo, Field seqNoDocValue, Field primaryTerm) { + public SequenceIDFields(Field seqNo, Field seqNoDocValue, Field primaryTerm, Field tombstoneField) { Objects.requireNonNull(seqNo, "sequence number field cannot be null"); Objects.requireNonNull(seqNoDocValue, "sequence number dv field cannot be null"); Objects.requireNonNull(primaryTerm, "primary term field cannot be null"); this.seqNo = seqNo; this.seqNoDocValue = seqNoDocValue; this.primaryTerm = primaryTerm; + this.tombstoneField = tombstoneField; } public static SequenceIDFields emptySeqID() { return new SequenceIDFields(new LongPoint(NAME, SequenceNumbers.UNASSIGNED_SEQ_NO), new NumericDocValuesField(NAME, SequenceNumbers.UNASSIGNED_SEQ_NO), - new NumericDocValuesField(PRIMARY_TERM_NAME, 0)); + new NumericDocValuesField(PRIMARY_TERM_NAME, 0), new NumericDocValuesField(TOMBSTONE_NAME, 0)); } } public static final String NAME = "_seq_no"; public static final String CONTENT_TYPE = "_seq_no"; public static final String PRIMARY_TERM_NAME = "_primary_term"; + public static final String TOMBSTONE_NAME = "_tombstone"; public static class SeqNoDefaults { public static final String NAME = SeqNoFieldMapper.NAME; 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 f2090613c09..7bfe793bba4 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.mapper; +import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.StoredField; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; @@ -49,6 +50,7 @@ import java.util.function.Function; public class SourceFieldMapper extends MetadataFieldMapper { public static final String NAME = "_source"; + public static final String RECOVERY_SOURCE_NAME = "_recovery_source"; public static final String CONTENT_TYPE = "_source"; private final Function, Map> filter; @@ -224,7 +226,8 @@ public class SourceFieldMapper extends MetadataFieldMapper { @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { - BytesReference source = context.sourceToParse().source(); + BytesReference originalSource = context.sourceToParse().source(); + BytesReference source = originalSource; if (enabled && fieldType().stored() && source != null) { // Percolate and tv APIs may not set the source and that is ok, because these APIs will not index any data if (filter != null) { @@ -240,8 +243,17 @@ public class SourceFieldMapper extends MetadataFieldMapper { } BytesRef ref = source.toBytesRef(); fields.add(new StoredField(fieldType().name(), ref.bytes, ref.offset, ref.length)); + } else { + source = null; } - } + + if (originalSource != null && source != originalSource && context.indexSettings().isSoftDeleteEnabled()) { + // 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(); + fields.add(new StoredField(RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); + fields.add(new NumericDocValuesField(RECOVERY_SOURCE_NAME, 1)); + } + } @Override protected String contentType() { 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 e030c95b56e..ef5f9ab0ef3 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -92,12 +92,14 @@ import org.elasticsearch.index.fielddata.ShardFieldData; import org.elasticsearch.index.flush.FlushStats; import org.elasticsearch.index.get.GetStats; import org.elasticsearch.index.get.ShardGetService; +import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperForType; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; @@ -1620,25 +1622,33 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl } /** - * Acquires a lock on the translog files, preventing them from being trimmed. + * Acquires a lock on the translog files and Lucene soft-deleted documents to prevent them from being trimmed */ - public Closeable acquireTranslogRetentionLock() { - return getEngine().acquireTranslogRetentionLock(); + public Closeable acquireRetentionLockForPeerRecovery() { + return getEngine().acquireRetentionLockForPeerRecovery(); } /** - * Creates a new translog snapshot for reading translog operations whose seq# at least the provided seq#. - * The caller has to close the returned snapshot after finishing the reading. + * Returns the estimated number of history operations whose seq# at least the provided seq# in this shard. */ - public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { - return getEngine().newTranslogSnapshotFromMinSeqNo(minSeqNo); + public int estimateNumberOfHistoryOperations(String source, long startingSeqNo) throws IOException { + return getEngine().estimateNumberOfHistoryOperations(source, mapperService, startingSeqNo); } /** - * Returns the estimated number of operations in translog whose seq# at least the provided seq#. + * Creates a new history snapshot for reading operations since the provided starting seqno (inclusive). + * The returned snapshot can be retrieved from either Lucene index or translog files. */ - public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { - return getEngine().estimateTranslogOperationsFromMinSeq(minSeqNo); + public Translog.Snapshot getHistoryOperations(String source, long startingSeqNo) throws IOException { + return getEngine().readHistoryOperations(source, mapperService, startingSeqNo); + } + + /** + * Checks if we have a completed history of operations since the given starting seqno (inclusive). + * This method should be called after acquiring the retention lock; See {@link #acquireRetentionLockForPeerRecovery()} + */ + public boolean hasCompleteHistoryOperations(String source, long startingSeqNo) throws IOException { + return getEngine().hasCompleteOperationHistory(source, mapperService, startingSeqNo); } public List segments(boolean verbose) { @@ -2209,7 +2219,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), - indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, () -> operationPrimaryTerm); + indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, () -> operationPrimaryTerm, tombstoneDocSupplier()); } /** @@ -2648,4 +2658,19 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl refreshMetric.inc(System.nanoTime() - currentRefreshStartTime); } } + + private EngineConfig.TombstoneDocSupplier tombstoneDocSupplier() { + final RootObjectMapper.Builder noopRootMapper = new RootObjectMapper.Builder("__noop"); + final DocumentMapper noopDocumentMapper = new DocumentMapper.Builder(noopRootMapper, mapperService).build(mapperService); + return new EngineConfig.TombstoneDocSupplier() { + @Override + public ParsedDocument newDeleteTombstoneDoc(String type, String id) { + return docMapper(type).getDocumentMapper().createDeleteTombstoneDoc(shardId.getIndexName(), type, id); + } + @Override + public ParsedDocument newNoopTombstoneDoc(String reason) { + return noopDocumentMapper.createNoopTombstoneDoc(shardId.getIndexName(), reason); + } + }; + } } 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 1edc0eb5dca..016a8afff69 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java +++ b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java @@ -89,7 +89,7 @@ public class PrimaryReplicaSyncer extends AbstractComponent { // 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.newTranslogSnapshotFromMinSeqNo(startingSeqNo); + snapshot = indexShard.getHistoryOperations("resync", startingSeqNo); final Translog.Snapshot originalSnapshot = snapshot; final Translog.Snapshot wrappedSnapshot = new Translog.Snapshot() { @Override diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index e9acfe3d8b0..ae3f90e63e7 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -156,6 +156,7 @@ final class StoreRecovery { final Directory hardLinkOrCopyTarget = new org.apache.lucene.store.HardlinkCopyDirectoryWrapper(target); IndexWriterConfig iwc = new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) // we don't want merges to happen here - we call maybe merge on the engine // later once we stared it up otherwise we would need to wait for it here diff --git a/server/src/main/java/org/elasticsearch/index/store/Store.java b/server/src/main/java/org/elasticsearch/index/store/Store.java index 001e263ea8f..85975bc68c8 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -1009,7 +1009,6 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref } final String segmentId = IndexFileNames.parseSegmentName(meta.name()); final String extension = IndexFileNames.getExtension(meta.name()); - assert FIELD_INFOS_FILE_EXTENSION.equals(extension) == false || IndexFileNames.stripExtension(IndexFileNames.stripSegmentName(meta.name())).isEmpty() : "FieldInfos are generational but updateable DV are not supported in elasticsearch"; if (IndexFileNames.SEGMENTS.equals(segmentId) || DEL_FILE_EXTENSION.equals(extension) || LIV_FILE_EXTENSION.equals(extension)) { // only treat del files as per-commit files fnm files are generational but only for upgradable DV perCommitStoreFiles.add(meta); @@ -1595,6 +1594,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref throws IOException { assert openMode == IndexWriterConfig.OpenMode.APPEND || commit == null : "can't specify create flag with a commit"; IndexWriterConfig iwc = new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) .setIndexCommit(commit) // we don't want merges to happen here - we call maybe merge on the engine diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index 618aa546e42..f17acac3789 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -1261,6 +1261,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC ", type='" + type + '\'' + ", seqNo=" + seqNo + ", primaryTerm=" + primaryTerm + + ", version=" + version + + ", autoGeneratedIdTimestamp=" + autoGeneratedIdTimestamp + '}'; } @@ -1403,6 +1405,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC "uid=" + uid + ", seqNo=" + seqNo + ", primaryTerm=" + primaryTerm + + ", version=" + version + '}'; } } diff --git a/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java b/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java index f48f2ceb792..e0cfe9eaaff 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -40,6 +40,7 @@ import java.nio.file.Path; import java.nio.file.StandardOpenOption; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.LongSupplier; @@ -192,7 +193,24 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable { new BufferedChecksumStreamInput(data.streamInput(), "assertion")); Translog.Operation prvOp = Translog.readOperation( new BufferedChecksumStreamInput(previous.v1().streamInput(), "assertion")); - if (newOp.equals(prvOp) == false) { + // TODO: We haven't had timestamp for Index operations in Lucene yet, we need to loosen this check without timestamp. + final boolean sameOp; + if (newOp instanceof Translog.Index && prvOp instanceof Translog.Index) { + final Translog.Index o1 = (Translog.Index) prvOp; + final Translog.Index o2 = (Translog.Index) newOp; + sameOp = Objects.equals(o1.id(), o2.id()) && Objects.equals(o1.type(), o2.type()) + && Objects.equals(o1.source(), o2.source()) && Objects.equals(o1.routing(), o2.routing()) + && o1.primaryTerm() == o2.primaryTerm() && o1.seqNo() == o2.seqNo() + && o1.version() == o2.version(); + } else if (newOp instanceof Translog.Delete && prvOp instanceof Translog.Delete) { + final Translog.Delete o1 = (Translog.Delete) newOp; + final Translog.Delete o2 = (Translog.Delete) prvOp; + sameOp = Objects.equals(o1.id(), o2.id()) && Objects.equals(o1.type(), o2.type()) + && o1.primaryTerm() == o2.primaryTerm() && o1.seqNo() == o2.seqNo() && o1.version() == o2.version(); + } else { + sameOp = false; + } + if (sameOp == false) { throw new AssertionError( "seqNo [" + seqNo + "] was processed twice in generation [" + generation + "], with different data. " + "prvOp [" + prvOp + "], newOp [" + newOp + "]", previous.v2()); diff --git a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java index 86995ae7c5a..a90f8af0af4 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java @@ -32,6 +32,7 @@ import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.Lock; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.store.NativeFSLockFactory; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cli.EnvironmentAwareCommand; @@ -177,6 +178,7 @@ public class TruncateTranslogCommand extends EnvironmentAwareCommand { terminal.println("Marking index with the new history uuid"); // commit the new histroy id IndexWriterConfig iwc = new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) // we don't want merges to happen here - we call maybe merge on the engine // later once we stared it up otherwise we would need to wait for it here 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 352f07d5764..10f796e5e15 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -146,11 +146,11 @@ public class RecoverySourceHandler { assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; }, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger); - try (Closeable ignored = shard.acquireTranslogRetentionLock()) { + try (Closeable ignored = shard.acquireRetentionLockForPeerRecovery()) { final long startingSeqNo; final long requiredSeqNoRangeStart; final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && - isTargetSameHistory() && isTranslogReadyForSequenceNumberBasedRecovery(); + isTargetSameHistory() && shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo()); if (isSequenceNumberBasedRecovery) { logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); startingSeqNo = request.startingSeqNo(); @@ -162,14 +162,16 @@ public class RecoverySourceHandler { } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e); } - // we set this to 0 to create a translog roughly according to the retention policy - // on the target. Note that it will still filter out legacy operations with no sequence numbers - startingSeqNo = 0; - // but we must have everything above the local checkpoint in the commit + // We must have everything above the local checkpoint in the commit requiredSeqNoRangeStart = Long.parseLong(phase1Snapshot.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1; + // If soft-deletes enabled, we need to transfer only operations after the local_checkpoint of the commit to have + // the same history on the target. However, with translog, we need to set this to 0 to create a translog roughly + // according to the retention policy on the target. Note that it will still filter out legacy operations without seqNo. + startingSeqNo = shard.indexSettings().isSoftDeleteEnabled() ? requiredSeqNoRangeStart : 0; try { - phase1(phase1Snapshot.getIndexCommit(), () -> shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); + final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo); + phase1(phase1Snapshot.getIndexCommit(), () -> estimateNumOps); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e); } finally { @@ -186,7 +188,8 @@ public class RecoverySourceHandler { try { // For a sequence based recovery, the target can keep its local translog - prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); + prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, + shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); } @@ -207,11 +210,13 @@ public class RecoverySourceHandler { */ cancellableThreads.execute(() -> shard.waitForOpsToComplete(endingSeqNo)); - logger.trace("all operations up to [{}] completed, which will be used as an ending sequence number", endingSeqNo); - - logger.trace("snapshot translog for recovery; current size is [{}]", shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); + if (logger.isTraceEnabled()) { + logger.trace("all operations up to [{}] completed, which will be used as an ending sequence number", endingSeqNo); + logger.trace("snapshot translog for recovery; current size is [{}]", + shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); + } final long targetLocalCheckpoint; - try(Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) { + try (Translog.Snapshot snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo)) { targetLocalCheckpoint = phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot); } catch (Exception e) { throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); @@ -268,36 +273,6 @@ public class RecoverySourceHandler { }); } - /** - * Determines if the source translog is ready for a sequence-number-based peer recovery. The main condition here is that the source - * translog contains all operations above the local checkpoint on the target. We already know the that translog contains or will contain - * all ops above the source local checkpoint, so we can stop check there. - * - * @return {@code true} if the source is ready for a sequence-number-based recovery - * @throws IOException if an I/O exception occurred reading the translog snapshot - */ - boolean isTranslogReadyForSequenceNumberBasedRecovery() throws IOException { - final long startingSeqNo = request.startingSeqNo(); - assert startingSeqNo >= 0; - final long localCheckpoint = shard.getLocalCheckpoint(); - logger.trace("testing sequence numbers in range: [{}, {}]", startingSeqNo, localCheckpoint); - // the start recovery request is initialized with the starting sequence number set to the target shard's local checkpoint plus one - if (startingSeqNo - 1 <= localCheckpoint) { - final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1); - try (Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) { - Translog.Operation operation; - while ((operation = snapshot.next()) != null) { - if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - tracker.markSeqNoAsCompleted(operation.seqNo()); - } - } - } - return tracker.getCheckpoint() >= localCheckpoint; - } else { - return false; - } - } - /** * Perform phase1 of the recovery operations. Once this {@link IndexCommit} * snapshot has been performed no commit operations (files being fsync'd) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index a4d6518e9af..9469f657c96 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1492,6 +1492,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp // empty shard would cause exceptions to be thrown. Since there is no data to restore from an empty // shard anyway, we just create the empty shard here and then exit. IndexWriter writer = new IndexWriter(store.directory(), new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setOpenMode(IndexWriterConfig.OpenMode.CREATE) .setCommitOnClose(true)); writer.close(); diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 702d63d0d94..6acdbad2cce 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -64,6 +64,7 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.repositories.IndexId; @@ -120,7 +121,8 @@ public class RestoreService extends AbstractComponent implements ClusterStateApp SETTING_NUMBER_OF_SHARDS, SETTING_VERSION_CREATED, SETTING_INDEX_UUID, - SETTING_CREATION_DATE)); + SETTING_CREATION_DATE, + IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey())); // It's OK to change some settings, but we shouldn't allow simply removing them private static final Set UNREMOVABLE_SETTINGS; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 90173455c3b..9786c0eaf52 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -392,6 +392,7 @@ public class PrimaryAllocationIT extends ESIntegTestCase { assertThat(shard.getLocalCheckpoint(), equalTo(numDocs + moreDocs)); } }, 30, TimeUnit.SECONDS); + internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); } } diff --git a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java index 753aedea01e..890f6ef163b 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java @@ -33,18 +33,23 @@ import org.apache.lucene.index.NoDeletionPolicy; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.Weight; import org.apache.lucene.store.Directory; import org.apache.lucene.store.MMapDirectory; import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.util.Bits; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.test.ESTestCase; import java.io.IOException; +import java.io.StringReader; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -53,6 +58,8 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import static org.hamcrest.Matchers.equalTo; + public class LuceneTests extends ESTestCase { public void testWaitForIndex() throws Exception { final MockDirectoryWrapper dir = newMockDirectory(); @@ -406,4 +413,88 @@ public class LuceneTests extends ESTestCase { // add assume's here if needed for certain platforms, but we should know if it does not work. assertTrue("MMapDirectory does not support unmapping: " + MMapDirectory.UNMAP_NOT_SUPPORTED_REASON, MMapDirectory.UNMAP_SUPPORTED); } + + public void testWrapAllDocsLive() throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) + .setMergePolicy(new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, MatchAllDocsQuery::new, newMergePolicy())); + IndexWriter writer = new IndexWriter(dir, config); + int numDocs = between(1, 10); + Set liveDocs = new HashSet<>(); + for (int i = 0; i < numDocs; i++) { + String id = Integer.toString(i); + Document doc = new Document(); + doc.add(new StringField("id", id, Store.YES)); + writer.addDocument(doc); + liveDocs.add(id); + } + for (int i = 0; i < numDocs; i++) { + if (randomBoolean()) { + String id = Integer.toString(i); + Document doc = new Document(); + doc.add(new StringField("id", "v2-" + id, Store.YES)); + if (randomBoolean()) { + doc.add(Lucene.newSoftDeletesField()); + } + writer.softUpdateDocument(new Term("id", id), doc, Lucene.newSoftDeletesField()); + liveDocs.add("v2-" + id); + } + } + try (DirectoryReader unwrapped = DirectoryReader.open(writer)) { + DirectoryReader reader = Lucene.wrapAllDocsLive(unwrapped); + assertThat(reader.numDocs(), equalTo(liveDocs.size())); + IndexSearcher searcher = new IndexSearcher(reader); + Set actualDocs = new HashSet<>(); + TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), Integer.MAX_VALUE); + for (ScoreDoc scoreDoc : topDocs.scoreDocs) { + actualDocs.add(reader.document(scoreDoc.doc).get("id")); + } + assertThat(actualDocs, equalTo(liveDocs)); + } + IOUtils.close(writer, dir); + } + + public void testWrapLiveDocsNotExposeAbortedDocuments() throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) + .setMergePolicy(new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, MatchAllDocsQuery::new, newMergePolicy())); + IndexWriter writer = new IndexWriter(dir, config); + int numDocs = between(1, 10); + List liveDocs = new ArrayList<>(); + for (int i = 0; i < numDocs; i++) { + String id = Integer.toString(i); + Document doc = new Document(); + doc.add(new StringField("id", id, Store.YES)); + if (randomBoolean()) { + doc.add(Lucene.newSoftDeletesField()); + } + writer.addDocument(doc); + liveDocs.add(id); + } + int abortedDocs = between(1, 10); + for (int i = 0; i < abortedDocs; i++) { + try { + Document doc = new Document(); + doc.add(new StringField("id", "aborted-" + i, Store.YES)); + StringReader reader = new StringReader(""); + doc.add(new TextField("other", reader)); + reader.close(); // mark the indexing hit non-aborting error + writer.addDocument(doc); + fail("index should have failed"); + } catch (Exception ignored) { } + } + try (DirectoryReader unwrapped = DirectoryReader.open(writer)) { + DirectoryReader reader = Lucene.wrapAllDocsLive(unwrapped); + assertThat(reader.maxDoc(), equalTo(numDocs + abortedDocs)); + assertThat(reader.numDocs(), equalTo(liveDocs.size())); + IndexSearcher searcher = new IndexSearcher(reader); + List actualDocs = new ArrayList<>(); + TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), Integer.MAX_VALUE); + for (ScoreDoc scoreDoc : topDocs.scoreDocs) { + actualDocs.add(reader.document(scoreDoc.doc).get("id")); + } + assertThat(actualDocs, equalTo(liveDocs)); + } + IOUtils.close(writer, dir); + } } diff --git a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java index 6bdd8ea3f2e..ac2f2b0d4f3 100644 --- a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java +++ b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java @@ -109,6 +109,7 @@ public abstract class AbstractDisruptionTestCase extends ESIntegTestCase { protected void beforeIndexDeletion() throws Exception { if (disableBeforeIndexDeletion == false) { super.beforeIndexDeletion(); + internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); assertSeqNos(); } } diff --git a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index d098c4918a7..b0b6c35f92a 100644 --- a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -40,6 +40,7 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; +import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -397,7 +398,8 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { .get(); logger.info("--> indexing docs"); - for (int i = 0; i < randomIntBetween(1, 1024); i++) { + int numDocs = randomIntBetween(1, 1024); + for (int i = 0; i < numDocs; i++) { client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); } @@ -419,12 +421,15 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { } logger.info("--> restart replica node"); + boolean softDeleteEnabled = internalCluster().getInstance(IndicesService.class, primaryNode) + .indexServiceSafe(resolveIndex("test")).getShard(0).indexSettings().isSoftDeleteEnabled(); + int moreDocs = randomIntBetween(1, 1024); internalCluster().restartNode(replicaNode, new RestartCallback() { @Override public Settings onNodeStopped(String nodeName) throws Exception { // index some more documents; we expect to reuse the files that already exist on the replica - for (int i = 0; i < randomIntBetween(1, 1024); i++) { + for (int i = 0; i < moreDocs; i++) { client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); } @@ -432,8 +437,12 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { client(primaryNode).admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder() .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) ).get(); client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get(); + if (softDeleteEnabled) { // We need an extra flush to advance the min_retained_seqno of the SoftDeletesPolicy + client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get(); + } return super.onNodeStopped(nodeName); } }); diff --git a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java index 28fa440d96a..b0b4ec3930a 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -306,7 +307,7 @@ public class IndexServiceTests extends ESSingleNodeTestCase { .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), -1)) .get(); IndexShard shard = indexService.getShard(0); - assertBusy(() -> assertThat(shard.estimateTranslogOperationsFromMinSeq(0L), equalTo(0))); + assertBusy(() -> assertThat(IndexShardTestCase.getTranslog(shard).totalOperations(), equalTo(0))); } public void testIllegalFsyncInterval() { diff --git a/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java b/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java index b7da5add2ac..64a2fa69bcb 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java @@ -553,4 +553,12 @@ public class IndexSettingsTests extends ESTestCase { ); assertThat(index.getDefaultFields(), equalTo(Arrays.asList("body", "title"))); } + + public void testUpdateSoftDeletesFails() { + IndexScopedSettings settings = new IndexScopedSettings(Settings.EMPTY, IndexScopedSettings.BUILT_IN_INDEX_SETTINGS); + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, () -> + settings.updateSettings(Settings.builder().put("index.soft_deletes.enabled", randomBoolean()).build(), + Settings.builder(), Settings.builder(), "index")); + assertThat(error.getMessage(), equalTo("final index setting [index.soft_deletes.enabled], not updateable")); + } } diff --git a/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index ea7de50b7b3..3f9fc9a0429 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -51,20 +51,24 @@ public class CombinedDeletionPolicyTests extends ESTestCase { public void testKeepCommitsAfterGlobalCheckpoint() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); + final int extraRetainedOps = between(0, 100); + final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, extraRetainedOps); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); final LongArrayList maxSeqNoList = new LongArrayList(); final LongArrayList translogGenList = new LongArrayList(); final List commitList = new ArrayList<>(); int totalCommits = between(2, 20); long lastMaxSeqNo = 0; + long lastCheckpoint = lastMaxSeqNo; long lastTranslogGen = 0; final UUID translogUUID = UUID.randomUUID(); for (int i = 0; i < totalCommits; i++) { lastMaxSeqNo += between(1, 10000); + lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); lastTranslogGen += between(1, 100); - commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); maxSeqNoList.add(lastMaxSeqNo); translogGenList.add(lastTranslogGen); } @@ -85,14 +89,19 @@ public class CombinedDeletionPolicyTests extends ESTestCase { } assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(translogGenList.get(keptIndex))); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), + equalTo(Math.min(getLocalCheckpoint(commitList.get(keptIndex)) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } public void testAcquireIndexCommit() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); + final int extraRetainedOps = between(0, 100); + final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, extraRetainedOps); final UUID translogUUID = UUID.randomUUID(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); long lastMaxSeqNo = between(1, 1000); + long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); long lastTranslogGen = between(1, 20); int safeIndex = 0; List commitList = new ArrayList<>(); @@ -102,8 +111,9 @@ public class CombinedDeletionPolicyTests extends ESTestCase { int newCommits = between(1, 10); for (int n = 0; n < newCommits; n++) { lastMaxSeqNo += between(1, 1000); + lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); lastTranslogGen += between(1, 20); - commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); } // Advance the global checkpoint to between [safeIndex, safeIndex + 1) safeIndex = randomIntBetween(safeIndex, commitList.size() - 1); @@ -114,6 +124,9 @@ public class CombinedDeletionPolicyTests extends ESTestCase { globalCheckpoint.set(randomLongBetween(lower, upper)); commitList.forEach(this::resetDeletion); indexPolicy.onCommit(commitList); + IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), + equalTo(Math.min(getLocalCheckpoint(safeCommit) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); // Captures and releases some commits int captures = between(0, 5); for (int n = 0; n < captures; n++) { @@ -132,7 +145,7 @@ public class CombinedDeletionPolicyTests extends ESTestCase { snapshottingCommits.remove(snapshot); final long pendingSnapshots = snapshottingCommits.stream().filter(snapshot::equals).count(); final IndexCommit lastCommit = commitList.get(commitList.size() - 1); - final IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); + safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); assertThat(indexPolicy.releaseCommit(snapshot), equalTo(pendingSnapshots == 0 && snapshot.equals(lastCommit) == false && snapshot.equals(safeCommit) == false)); } @@ -143,6 +156,8 @@ public class CombinedDeletionPolicyTests extends ESTestCase { equalTo(Long.parseLong(commitList.get(safeIndex).getUserData().get(Translog.TRANSLOG_GENERATION_KEY)))); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(Long.parseLong(commitList.get(commitList.size() - 1).getUserData().get(Translog.TRANSLOG_GENERATION_KEY)))); + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), + equalTo(Math.min(getLocalCheckpoint(commitList.get(safeIndex)) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } snapshottingCommits.forEach(indexPolicy::releaseCommit); globalCheckpoint.set(randomLongBetween(lastMaxSeqNo, Long.MAX_VALUE)); @@ -154,25 +169,27 @@ public class CombinedDeletionPolicyTests extends ESTestCase { assertThat(commitList.get(commitList.size() - 1).isDeleted(), equalTo(false)); assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(lastTranslogGen)); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); + IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), + equalTo(Math.min(getLocalCheckpoint(safeCommit) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } public void testLegacyIndex() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); + final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); final UUID translogUUID = UUID.randomUUID(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); long legacyTranslogGen = randomNonNegativeLong(); IndexCommit legacyCommit = mockLegacyIndexCommit(translogUUID, legacyTranslogGen); - indexPolicy.onCommit(singletonList(legacyCommit)); - verify(legacyCommit, never()).delete(); - assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(legacyTranslogGen)); - assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(legacyTranslogGen)); + assertThat(CombinedDeletionPolicy.findSafeCommitPoint(singletonList(legacyCommit), globalCheckpoint.get()), + equalTo(legacyCommit)); long safeTranslogGen = randomLongBetween(legacyTranslogGen, Long.MAX_VALUE); long maxSeqNo = randomLongBetween(1, Long.MAX_VALUE); - final IndexCommit freshCommit = mockIndexCommit(maxSeqNo, translogUUID, safeTranslogGen); + final IndexCommit freshCommit = mockIndexCommit(randomLongBetween(-1, maxSeqNo), maxSeqNo, translogUUID, safeTranslogGen); globalCheckpoint.set(randomLongBetween(0, maxSeqNo - 1)); indexPolicy.onCommit(Arrays.asList(legacyCommit, freshCommit)); @@ -189,25 +206,32 @@ public class CombinedDeletionPolicyTests extends ESTestCase { verify(freshCommit, times(0)).delete(); assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(safeTranslogGen)); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(safeTranslogGen)); + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), equalTo(getLocalCheckpoint(freshCommit) + 1)); } public void testDeleteInvalidCommits() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(randomNonNegativeLong()); + final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); final int invalidCommits = between(1, 10); final List commitList = new ArrayList<>(); for (int i = 0; i < invalidCommits; i++) { - commitList.add(mockIndexCommit(randomNonNegativeLong(), UUID.randomUUID(), randomNonNegativeLong())); + long maxSeqNo = randomNonNegativeLong(); + commitList.add(mockIndexCommit(randomLongBetween(-1, maxSeqNo), maxSeqNo, UUID.randomUUID(), randomNonNegativeLong())); } final UUID expectedTranslogUUID = UUID.randomUUID(); long lastTranslogGen = 0; final int validCommits = between(1, 10); + long lastMaxSeqNo = between(1, 1000); + long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); for (int i = 0; i < validCommits; i++) { lastTranslogGen += between(1, 1000); - commitList.add(mockIndexCommit(randomNonNegativeLong(), expectedTranslogUUID, lastTranslogGen)); + lastMaxSeqNo += between(1, 1000); + lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, expectedTranslogUUID, lastTranslogGen)); } // We should never keep invalid commits regardless of the value of the global checkpoint. @@ -215,21 +239,26 @@ public class CombinedDeletionPolicyTests extends ESTestCase { for (int i = 0; i < invalidCommits - 1; i++) { verify(commitList.get(i), times(1)).delete(); } + assertThat(softDeletesPolicy.getMinRetainedSeqNo(), + equalTo(getLocalCheckpoint(CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get())) + 1)); } public void testCheckUnreferencedCommits() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); final UUID translogUUID = UUID.randomUUID(); final TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); final List commitList = new ArrayList<>(); int totalCommits = between(2, 20); long lastMaxSeqNo = between(1, 1000); + long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); long lastTranslogGen = between(1, 50); for (int i = 0; i < totalCommits; i++) { lastMaxSeqNo += between(1, 10000); lastTranslogGen += between(1, 100); - commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); + lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); } IndexCommit safeCommit = randomFrom(commitList); globalCheckpoint.set(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO))); @@ -256,8 +285,9 @@ public class CombinedDeletionPolicyTests extends ESTestCase { } } - IndexCommit mockIndexCommit(long maxSeqNo, UUID translogUUID, long translogGen) throws IOException { + IndexCommit mockIndexCommit(long localCheckpoint, long maxSeqNo, UUID translogUUID, long translogGen) throws IOException { final Map userData = new HashMap<>(); + userData.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(localCheckpoint)); userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); userData.put(Translog.TRANSLOG_UUID_KEY, translogUUID.toString()); userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(translogGen)); @@ -278,6 +308,10 @@ public class CombinedDeletionPolicyTests extends ESTestCase { }).when(commit).delete(); } + private long getLocalCheckpoint(IndexCommit commit) throws IOException { + return Long.parseLong(commit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); + } + IndexCommit mockLegacyIndexCommit(UUID translogUUID, long translogGen) throws IOException { final Map userData = new HashMap<>(); userData.put(Translog.TRANSLOG_UUID_KEY, translogUUID.toString()); @@ -287,4 +321,5 @@ public class CombinedDeletionPolicyTests extends ESTestCase { resetDeletion(commit); return commit; } + } 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 76e05ba1e0b..d3aead9e44e 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.engine; +import java.io.Closeable; import java.io.IOException; import java.io.UncheckedIOException; import java.nio.charset.Charset; @@ -77,10 +78,12 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.LogByteSizeMergePolicy; import org.apache.lucene.index.LogDocMergePolicy; +import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PointValues; import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.index.TieredMergePolicy; import org.apache.lucene.search.IndexSearcher; @@ -114,6 +117,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; @@ -133,6 +137,7 @@ import org.elasticsearch.index.fieldvisitor.FieldsVisitor; import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.Mapper.BuilderContext; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.ParseContext; @@ -172,8 +177,10 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.isIn; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @@ -247,8 +254,13 @@ public class InternalEngineTests extends EngineTestCase { } public void testSegments() throws Exception { + Settings settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings( + IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); try (Store store = createStore(); - InternalEngine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { + InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null))) { List segments = engine.segments(false); assertThat(segments.isEmpty(), equalTo(true)); assertThat(engine.segmentsStats(false).getCount(), equalTo(0L)); @@ -1311,9 +1323,13 @@ public class InternalEngineTests extends EngineTestCase { assertThat(indexResult.getVersion(), equalTo(1L)); } - public void testForceMerge() throws IOException { + public void testForceMergeWithoutSoftDeletes() throws IOException { + Settings settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); + IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); try (Store store = createStore(); - Engine engine = createEngine(config(defaultSettings, store, createTempDir(), + Engine engine = createEngine(config(IndexSettingsModule.newIndexSettings(indexMetaData), store, createTempDir(), new LogByteSizeMergePolicy(), null))) { // use log MP here we test some behavior in ESMP int numDocs = randomIntBetween(10, 100); for (int i = 0; i < numDocs; i++) { @@ -1354,6 +1370,165 @@ public class InternalEngineTests extends EngineTestCase { } } + public void testForceMergeWithSoftDeletesRetention() throws Exception { + final long retainedExtraOps = randomLongBetween(0, 10); + Settings.Builder settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), retainedExtraOps); + final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + final MapperService mapperService = createMapperService("test"); + final Set liveDocs = new HashSet<>(); + try (Store store = createStore(); + InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get))) { + int numDocs = scaledRandomIntBetween(10, 100); + for (int i = 0; i < numDocs; i++) { + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null); + engine.index(indexForDoc(doc)); + liveDocs.add(doc.id()); + } + for (int i = 0; i < numDocs; i++) { + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null); + if (randomBoolean()) { + engine.delete(new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get())); + liveDocs.remove(doc.id()); + } + if (randomBoolean()) { + engine.index(indexForDoc(doc)); + liveDocs.add(doc.id()); + } + if (randomBoolean()) { + engine.flush(randomBoolean(), true); + } + } + engine.flush(); + + long localCheckpoint = engine.getLocalCheckpoint(); + globalCheckpoint.set(randomLongBetween(0, localCheckpoint)); + engine.syncTranslog(); + final long safeCommitCheckpoint; + try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { + safeCommitCheckpoint = Long.parseLong(safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); + } + engine.forceMerge(true, 1, false, false, false); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + Map ops = readAllOperationsInLucene(engine, mapperService) + .stream().collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); + for (long seqno = 0; seqno <= localCheckpoint; seqno++) { + long minSeqNoToRetain = Math.min(globalCheckpoint.get() + 1 - retainedExtraOps, safeCommitCheckpoint + 1); + String msg = "seq# [" + seqno + "], global checkpoint [" + globalCheckpoint + "], retained-ops [" + retainedExtraOps + "]"; + if (seqno < minSeqNoToRetain) { + Translog.Operation op = ops.get(seqno); + if (op != null) { + assertThat(op, instanceOf(Translog.Index.class)); + assertThat(msg, ((Translog.Index) op).id(), isIn(liveDocs)); + assertEquals(msg, ((Translog.Index) op).source(), B_1); + } + } else { + assertThat(msg, ops.get(seqno), notNullValue()); + } + } + settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0); + indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); + engine.onSettingsChanged(); + globalCheckpoint.set(localCheckpoint); + engine.syncTranslog(); + + engine.forceMerge(true, 1, false, false, false); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + assertThat(readAllOperationsInLucene(engine, mapperService), hasSize(liveDocs.size())); + } + } + + public void testForceMergeWithSoftDeletesRetentionAndRecoverySource() throws Exception { + final long retainedExtraOps = randomLongBetween(0, 10); + Settings.Builder settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), retainedExtraOps); + final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + final MapperService mapperService = createMapperService("test"); + final boolean omitSourceAllTheTime = randomBoolean(); + final Set liveDocs = new HashSet<>(); + final Set liveDocsWithSource = new HashSet<>(); + try (Store store = createStore(); + InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, + globalCheckpoint::get))) { + int numDocs = scaledRandomIntBetween(10, 100); + for (int i = 0; i < numDocs; i++) { + boolean useRecoverySource = randomBoolean() || omitSourceAllTheTime; + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null, useRecoverySource); + engine.index(indexForDoc(doc)); + liveDocs.add(doc.id()); + if (useRecoverySource == false) { + liveDocsWithSource.add(Integer.toString(i)); + } + } + for (int i = 0; i < numDocs; i++) { + boolean useRecoverySource = randomBoolean() || omitSourceAllTheTime; + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null, useRecoverySource); + if (randomBoolean()) { + engine.delete(new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get())); + liveDocs.remove(doc.id()); + liveDocsWithSource.remove(doc.id()); + } + if (randomBoolean()) { + engine.index(indexForDoc(doc)); + liveDocs.add(doc.id()); + if (useRecoverySource == false) { + liveDocsWithSource.add(doc.id()); + } else { + liveDocsWithSource.remove(doc.id()); + } + } + if (randomBoolean()) { + engine.flush(randomBoolean(), true); + } + } + engine.flush(); + globalCheckpoint.set(randomLongBetween(0, engine.getLocalCheckpoint())); + engine.syncTranslog(); + final long minSeqNoToRetain; + try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { + long safeCommitLocalCheckpoint = Long.parseLong( + safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); + minSeqNoToRetain = Math.min(globalCheckpoint.get() + 1 - retainedExtraOps, safeCommitLocalCheckpoint + 1); + } + engine.forceMerge(true, 1, false, false, false); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + Map ops = readAllOperationsInLucene(engine, mapperService) + .stream().collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); + for (long seqno = 0; seqno <= engine.getLocalCheckpoint(); seqno++) { + String msg = "seq# [" + seqno + "], global checkpoint [" + globalCheckpoint + "], retained-ops [" + retainedExtraOps + "]"; + if (seqno < minSeqNoToRetain) { + Translog.Operation op = ops.get(seqno); + if (op != null) { + assertThat(op, instanceOf(Translog.Index.class)); + assertThat(msg, ((Translog.Index) op).id(), isIn(liveDocs)); + } + } else { + Translog.Operation op = ops.get(seqno); + assertThat(msg, op, notNullValue()); + if (op instanceof Translog.Index) { + assertEquals(msg, ((Translog.Index) op).source(), B_1); + } + } + } + settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0); + indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); + engine.onSettingsChanged(); + globalCheckpoint.set(engine.getLocalCheckpoint()); + engine.syncTranslog(); + engine.forceMerge(true, 1, false, false, false); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + assertThat(readAllOperationsInLucene(engine, mapperService), hasSize(liveDocsWithSource.size())); + } + } + public void testForceMergeAndClose() throws IOException, InterruptedException { int numIters = randomIntBetween(2, 10); for (int j = 0; j < numIters; j++) { @@ -1422,126 +1597,10 @@ public class InternalEngineTests extends EngineTestCase { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } - protected List generateSingleDocHistory(boolean forReplica, VersionType versionType, - long primaryTerm, - int minOpCount, int maxOpCount, String docId) { - final int numOfOps = randomIntBetween(minOpCount, maxOpCount); - final List ops = new ArrayList<>(); - final Term id = newUid(docId); - final int startWithSeqNo = 0; - final String valuePrefix = (forReplica ? "r_" : "p_" ) + docId + "_"; - final boolean incrementTermWhenIntroducingSeqNo = randomBoolean(); - for (int i = 0; i < numOfOps; i++) { - final Engine.Operation op; - final long version; - switch (versionType) { - case INTERNAL: - version = forReplica ? i : Versions.MATCH_ANY; - break; - case EXTERNAL: - version = i; - break; - case EXTERNAL_GTE: - version = randomBoolean() ? Math.max(i - 1, 0) : i; - break; - case FORCE: - version = randomNonNegativeLong(); - break; - default: - throw new UnsupportedOperationException("unknown version type: " + versionType); - } - if (randomBoolean()) { - op = new Engine.Index(id, testParsedDocument(docId, null, testDocumentWithTextField(valuePrefix + i), B_1, null), - forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, - forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, - version, - forReplica ? null : versionType, - forReplica ? REPLICA : PRIMARY, - System.currentTimeMillis(), -1, false - ); - } else { - op = new Engine.Delete("test", docId, id, - forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, - forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, - version, - forReplica ? null : versionType, - forReplica ? REPLICA : PRIMARY, - System.currentTimeMillis()); - } - ops.add(op); - } - return ops; - } - public void testOutOfOrderDocsOnReplica() throws IOException { final List ops = generateSingleDocHistory(true, randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE, VersionType.FORCE), 2, 2, 20, "1"); - assertOpsOnReplica(ops, replicaEngine, true); - } - - private void assertOpsOnReplica(List ops, InternalEngine replicaEngine, boolean shuffleOps) throws IOException { - final Engine.Operation lastOp = ops.get(ops.size() - 1); - final String lastFieldValue; - if (lastOp instanceof Engine.Index) { - Engine.Index index = (Engine.Index) lastOp; - lastFieldValue = index.docs().get(0).get("value"); - } else { - // delete - lastFieldValue = null; - } - if (shuffleOps) { - int firstOpWithSeqNo = 0; - while (firstOpWithSeqNo < ops.size() && ops.get(firstOpWithSeqNo).seqNo() < 0) { - firstOpWithSeqNo++; - } - // shuffle ops but make sure legacy ops are first - shuffle(ops.subList(0, firstOpWithSeqNo), random()); - shuffle(ops.subList(firstOpWithSeqNo, ops.size()), random()); - } - boolean firstOp = true; - for (Engine.Operation op : ops) { - logger.info("performing [{}], v [{}], seq# [{}], term [{}]", - op.operationType().name().charAt(0), op.version(), op.seqNo(), op.primaryTerm()); - if (op instanceof Engine.Index) { - Engine.IndexResult result = replicaEngine.index((Engine.Index) op); - // replicas don't really care to about creation status of documents - // this allows to ignore the case where a document was found in the live version maps in - // a delete state and return false for the created flag in favor of code simplicity - // as deleted or not. This check is just signal regression so a decision can be made if it's - // intentional - assertThat(result.isCreated(), equalTo(firstOp)); - assertThat(result.getVersion(), equalTo(op.version())); - assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); - - } else { - Engine.DeleteResult result = replicaEngine.delete((Engine.Delete) op); - // Replicas don't really care to about found status of documents - // this allows to ignore the case where a document was found in the live version maps in - // a delete state and return true for the found flag in favor of code simplicity - // his check is just signal regression so a decision can be made if it's - // intentional - assertThat(result.isFound(), equalTo(firstOp == false)); - assertThat(result.getVersion(), equalTo(op.version())); - assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); - } - if (randomBoolean()) { - engine.refresh("test"); - } - if (randomBoolean()) { - engine.flush(); - engine.refresh("test"); - } - firstOp = false; - } - - assertVisibleCount(replicaEngine, lastFieldValue == null ? 0 : 1); - if (lastFieldValue != null) { - try (Searcher searcher = replicaEngine.acquireSearcher("test")) { - final TotalHitCountCollector collector = new TotalHitCountCollector(); - searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); - assertThat(collector.getTotalHits(), equalTo(1)); - } - } + assertOpsOnReplica(ops, replicaEngine, true, logger); } public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, InterruptedException { @@ -1569,11 +1628,12 @@ public class InternalEngineTests extends EngineTestCase { } // randomly interleave final AtomicLong seqNoGenerator = new AtomicLong(); - Function seqNoUpdater = operation -> { - final long newSeqNo = seqNoGenerator.getAndIncrement(); + BiFunction seqNoUpdater = (operation, newSeqNo) -> { if (operation instanceof Engine.Index) { Engine.Index index = (Engine.Index) operation; - return new Engine.Index(index.uid(), index.parsedDoc(), newSeqNo, index.primaryTerm(), index.version(), + Document doc = testDocumentWithTextField(index.docs().get(0).get("value")); + ParsedDocument parsedDocument = testParsedDocument(index.id(), index.routing(), doc, index.source(), null); + return new Engine.Index(index.uid(), parsedDocument, newSeqNo, index.primaryTerm(), index.version(), index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry()); } else { Engine.Delete delete = (Engine.Delete) operation; @@ -1586,12 +1646,12 @@ public class InternalEngineTests extends EngineTestCase { Iterator iter2 = opsDoc2.iterator(); while (iter1.hasNext() && iter2.hasNext()) { final Engine.Operation next = randomBoolean() ? iter1.next() : iter2.next(); - allOps.add(seqNoUpdater.apply(next)); + allOps.add(seqNoUpdater.apply(next, seqNoGenerator.getAndIncrement())); } - iter1.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o))); - iter2.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o))); + iter1.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o, seqNoGenerator.getAndIncrement()))); + iter2.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o, seqNoGenerator.getAndIncrement()))); // insert some duplicates - allOps.addAll(randomSubsetOf(allOps)); + randomSubsetOf(allOps).forEach(op -> allOps.add(seqNoUpdater.apply(op, op.seqNo()))); shuffle(allOps, random()); concurrentlyApplyOps(allOps, engine); @@ -1623,42 +1683,6 @@ public class InternalEngineTests extends EngineTestCase { assertVisibleCount(engine, totalExpectedOps); } - private void concurrentlyApplyOps(List ops, InternalEngine engine) throws InterruptedException { - Thread[] thread = new Thread[randomIntBetween(3, 5)]; - CountDownLatch startGun = new CountDownLatch(thread.length); - AtomicInteger offset = new AtomicInteger(-1); - for (int i = 0; i < thread.length; i++) { - thread[i] = new Thread(() -> { - startGun.countDown(); - try { - startGun.await(); - } catch (InterruptedException e) { - throw new AssertionError(e); - } - int docOffset; - while ((docOffset = offset.incrementAndGet()) < ops.size()) { - try { - final Engine.Operation op = ops.get(docOffset); - if (op instanceof Engine.Index) { - engine.index((Engine.Index) op); - } else { - engine.delete((Engine.Delete) op); - } - if ((docOffset + 1) % 4 == 0) { - engine.refresh("test"); - } - } catch (IOException e) { - throw new AssertionError(e); - } - } - }); - thread[i].start(); - } - for (int i = 0; i < thread.length; i++) { - thread[i].join(); - } - } - public void testInternalVersioningOnPrimary() throws IOException { final List ops = generateSingleDocHistory(false, VersionType.INTERNAL, 2, 2, 20, "1"); assertOpsOnPrimary(ops, Versions.NOT_FOUND, true, engine); @@ -1869,7 +1893,7 @@ public class InternalEngineTests extends EngineTestCase { final boolean deletedOnReplica = lastReplicaOp instanceof Engine.Delete; final long finalReplicaVersion = lastReplicaOp.version(); final long finalReplicaSeqNo = lastReplicaOp.seqNo(); - assertOpsOnReplica(replicaOps, replicaEngine, true); + assertOpsOnReplica(replicaOps, replicaEngine, true, logger); final int opsOnPrimary = assertOpsOnPrimary(primaryOps, finalReplicaVersion, deletedOnReplica, replicaEngine); final long currentSeqNo = getSequenceID(replicaEngine, new Engine.Get(false, false, "type", lastReplicaOp.uid().text(), lastReplicaOp.uid())).v1(); @@ -2674,14 +2698,16 @@ public class InternalEngineTests extends EngineTestCase { Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } + EngineConfig config = engine.config(); assertVisibleCount(engine, numDocs); engine.close(); - trimUnsafeCommits(engine.config()); - engine = new InternalEngine(engine.config()); - engine.skipTranslogRecovery(); - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(0L)); + trimUnsafeCommits(config); + try (InternalEngine engine = new InternalEngine(config)) { + engine.skipTranslogRecovery(); + try (Engine.Searcher searcher = engine.acquireSearcher("test")) { + TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); + assertThat(topDocs.totalHits, equalTo(0L)); + } } } @@ -2811,7 +2837,7 @@ public class InternalEngineTests extends EngineTestCase { new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm::get); + new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm::get, tombstoneDocSupplier()); try { InternalEngine internalEngine = new InternalEngine(brokenConfig); fail("translog belongs to a different engine"); @@ -2940,6 +2966,12 @@ public class InternalEngineTests extends EngineTestCase { } } + @Override + public long softUpdateDocument(Term term, Iterable doc, Field... softDeletes) throws IOException { + maybeThrowFailure(); + return super.softUpdateDocument(term, doc, softDeletes); + } + @Override public long deleteDocuments(Term... terms) throws IOException { maybeThrowFailure(); @@ -3140,10 +3172,10 @@ public class InternalEngineTests extends EngineTestCase { } public void testDoubleDeliveryReplicaAppendingOnly() throws IOException { - final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), + final Supplier doc = () -> testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); - Engine.Index operation = appendOnlyReplica(doc, false, 1, randomIntBetween(0, 5)); - Engine.Index retry = appendOnlyReplica(doc, true, 1, randomIntBetween(0, 5)); + Engine.Index operation = appendOnlyReplica(doc.get(), false, 1, randomIntBetween(0, 5)); + Engine.Index retry = appendOnlyReplica(doc.get(), true, 1, randomIntBetween(0, 5)); // operations with a seq# equal or lower to the local checkpoint are not indexed to lucene // and the version lookup is skipped final boolean belowLckp = operation.seqNo() == 0 && retry.seqNo() == 0; @@ -3182,8 +3214,8 @@ public class InternalEngineTests extends EngineTestCase { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); assertEquals(1, topDocs.totalHits); } - operation = randomAppendOnly(doc, false, 1); - retry = randomAppendOnly(doc, true, 1); + operation = randomAppendOnly(doc.get(), false, 1); + retry = randomAppendOnly(doc.get(), true, 1); if (randomBoolean()) { Engine.IndexResult indexResult = engine.index(operation); assertNotNull(indexResult.getTranslogLocation()); @@ -3248,6 +3280,8 @@ public class InternalEngineTests extends EngineTestCase { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); assertEquals(1, topDocs.totalHits); } + List ops = readAllOperationsInLucene(engine, createMapperService("test")); + assertThat(ops.stream().map(o -> o.seqNo()).collect(Collectors.toList()), hasItem(20L)); } public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOException { @@ -3716,20 +3750,22 @@ public class InternalEngineTests extends EngineTestCase { final List operations = new ArrayList<>(); final int numberOfOperations = randomIntBetween(16, 32); - final Document document = testDocumentWithTextField(); final AtomicLong sequenceNumber = new AtomicLong(); final Engine.Operation.Origin origin = randomFrom(LOCAL_TRANSLOG_RECOVERY, PEER_RECOVERY, PRIMARY, REPLICA); final LongSupplier sequenceNumberSupplier = origin == PRIMARY ? () -> SequenceNumbers.UNASSIGNED_SEQ_NO : sequenceNumber::getAndIncrement; - document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); - final ParsedDocument doc = testParsedDocument("1", null, document, B_1, null); - final Term uid = newUid(doc); + final Supplier doc = () -> { + final Document document = testDocumentWithTextField(); + document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); + return testParsedDocument("1", null, document, B_1, null); + }; + final Term uid = newUid("1"); final BiFunction searcherFactory = engine::acquireSearcher; for (int i = 0; i < numberOfOperations; i++) { if (randomBoolean()) { final Engine.Index index = new Engine.Index( uid, - doc, + doc.get(), sequenceNumberSupplier.getAsLong(), 1, i, @@ -3805,7 +3841,9 @@ public class InternalEngineTests extends EngineTestCase { maxSeqNo, localCheckpoint); trimUnsafeCommits(engine.config()); - noOpEngine = new InternalEngine(engine.config(), supplier) { + EngineConfig noopEngineConfig = copy(engine.config(), new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, + () -> new MatchAllDocsQuery(), engine.config().getMergePolicy())); + noOpEngine = new InternalEngine(noopEngineConfig, supplier) { @Override protected long doGenerateSeqNoForOperation(Operation operation) { throw new UnsupportedOperationException(); @@ -3813,7 +3851,7 @@ public class InternalEngineTests extends EngineTestCase { }; noOpEngine.recoverFromTranslog(Long.MAX_VALUE); final int gapsFilled = noOpEngine.fillSeqNoGaps(primaryTerm.get()); - final String reason = randomAlphaOfLength(16); + final String reason = "filling gaps"; noOpEngine.noOp(new Engine.NoOp(maxSeqNo + 1, primaryTerm.get(), LOCAL_TRANSLOG_RECOVERY, System.nanoTime(), reason)); assertThat(noOpEngine.getLocalCheckpoint(), equalTo((long) (maxSeqNo + 1))); assertThat(noOpEngine.getTranslog().stats().getUncommittedOperations(), equalTo(gapsFilled)); @@ -3835,11 +3873,77 @@ public class InternalEngineTests extends EngineTestCase { assertThat(noOp.seqNo(), equalTo((long) (maxSeqNo + 2))); assertThat(noOp.primaryTerm(), equalTo(primaryTerm.get())); assertThat(noOp.reason(), equalTo(reason)); + if (engine.engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + MapperService mapperService = createMapperService("test"); + List operationsFromLucene = readAllOperationsInLucene(noOpEngine, mapperService); + assertThat(operationsFromLucene, hasSize(maxSeqNo + 2 - localCheckpoint)); // fills n gap and 2 manual noop. + for (int i = 0; i < operationsFromLucene.size(); i++) { + assertThat(operationsFromLucene.get(i), equalTo(new Translog.NoOp(localCheckpoint + 1 + i, primaryTerm.get(), "filling gaps"))); + } + assertConsistentHistoryBetweenTranslogAndLuceneIndex(noOpEngine, mapperService); + } } finally { IOUtils.close(noOpEngine); } } + /** + * Verifies that a segment containing only no-ops can be used to look up _version and _seqno. + */ + public void testSegmentContainsOnlyNoOps() throws Exception { + Engine.NoOpResult noOpResult = engine.noOp(new Engine.NoOp(1, primaryTerm.get(), + randomFrom(Engine.Operation.Origin.values()), randomNonNegativeLong(), "test")); + assertThat(noOpResult.getFailure(), nullValue()); + engine.refresh("test"); + Engine.DeleteResult deleteResult = engine.delete(replicaDeleteForDoc("id", 1, 2, randomNonNegativeLong())); + assertThat(deleteResult.getFailure(), nullValue()); + engine.refresh("test"); + } + + /** + * A simple test to check that random combination of operations can coexist in segments and be lookup. + * This is needed as some fields in Lucene may not exist if a segment misses operation types and this code is to check for that. + * For example, a segment containing only no-ops does not have neither _uid or _version. + */ + public void testRandomOperations() throws Exception { + int numOps = between(10, 100); + for (int i = 0; i < numOps; i++) { + String id = Integer.toString(randomIntBetween(1, 10)); + ParsedDocument doc = createParsedDoc(id, null); + Engine.Operation.TYPE type = randomFrom(Engine.Operation.TYPE.values()); + switch (type) { + case INDEX: + Engine.IndexResult index = engine.index(replicaIndexForDoc(doc, between(1, 100), i, randomBoolean())); + assertThat(index.getFailure(), nullValue()); + break; + case DELETE: + Engine.DeleteResult delete = engine.delete(replicaDeleteForDoc(doc.id(), between(1, 100), i, randomNonNegativeLong())); + assertThat(delete.getFailure(), nullValue()); + break; + case NO_OP: + Engine.NoOpResult noOp = engine.noOp(new Engine.NoOp(i, primaryTerm.get(), + randomFrom(Engine.Operation.Origin.values()), randomNonNegativeLong(), "")); + assertThat(noOp.getFailure(), nullValue()); + break; + default: + throw new IllegalStateException("Invalid op [" + type + "]"); + } + if (randomBoolean()) { + engine.refresh("test"); + } + if (randomBoolean()) { + engine.flush(); + } + if (randomBoolean()) { + engine.forceMerge(randomBoolean(), between(1, 10), randomBoolean(), false, false); + } + } + if (engine.engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + List operations = readAllOperationsInLucene(engine, createMapperService("test")); + assertThat(operations, hasSize(numOps)); + } + } + public void testMinGenerationForSeqNo() throws IOException, BrokenBarrierException, InterruptedException { engine.close(); final int numberOfTriplets = randomIntBetween(1, 32); @@ -4405,7 +4509,7 @@ public class InternalEngineTests extends EngineTestCase { globalCheckpoint.set(randomLongBetween(engine.getLocalCheckpoint(), Long.MAX_VALUE)); engine.syncTranslog(); assertThat(DirectoryReader.listCommits(store.directory()), contains(commits.get(commits.size() - 1))); - assertThat(engine.estimateTranslogOperationsFromMinSeq(0L), equalTo(0)); + assertThat(engine.getTranslog().totalOperations(), equalTo(0)); } } @@ -4768,6 +4872,154 @@ public class InternalEngineTests extends EngineTestCase { } } + public void testLuceneHistoryOnPrimary() throws Exception { + final List operations = generateSingleDocHistory(false, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "1"); + assertOperationHistoryInLucene(operations); + } + + public void testLuceneHistoryOnReplica() throws Exception { + final List operations = generateSingleDocHistory(true, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); + Randomness.shuffle(operations); + assertOperationHistoryInLucene(operations); + } + + private void assertOperationHistoryInLucene(List operations) throws IOException { + final MergePolicy keepSoftDeleteDocsMP = new SoftDeletesRetentionMergePolicy( + Lucene.SOFT_DELETES_FIELD, () -> new MatchAllDocsQuery(), engine.config().getMergePolicy()); + Settings.Builder settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); + final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); + Set expectedSeqNos = new HashSet<>(); + try (Store store = createStore(); + Engine engine = createEngine(config(indexSettings, store, createTempDir(), keepSoftDeleteDocsMP, null))) { + for (Engine.Operation op : operations) { + if (op instanceof Engine.Index) { + Engine.IndexResult indexResult = engine.index((Engine.Index) op); + assertThat(indexResult.getFailure(), nullValue()); + expectedSeqNos.add(indexResult.getSeqNo()); + } else { + Engine.DeleteResult deleteResult = engine.delete((Engine.Delete) op); + assertThat(deleteResult.getFailure(), nullValue()); + expectedSeqNos.add(deleteResult.getSeqNo()); + } + if (rarely()) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(); + } + if (rarely()) { + engine.forceMerge(true); + } + } + MapperService mapperService = createMapperService("test"); + List actualOps = readAllOperationsInLucene(engine, mapperService); + assertThat(actualOps.stream().map(o -> o.seqNo()).collect(Collectors.toList()), containsInAnyOrder(expectedSeqNos.toArray())); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + } + } + + public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException { + IOUtils.close(engine, store); + Settings.Builder settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); + final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + final List operations = generateSingleDocHistory(true, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); + Randomness.shuffle(operations); + Set existingSeqNos = new HashSet<>(); + store = createStore(); + engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get)); + assertThat(engine.getMinRetainedSeqNo(), equalTo(0L)); + long lastMinRetainedSeqNo = engine.getMinRetainedSeqNo(); + for (Engine.Operation op : operations) { + final Engine.Result result; + if (op instanceof Engine.Index) { + result = engine.index((Engine.Index) op); + } else { + result = engine.delete((Engine.Delete) op); + } + existingSeqNos.add(result.getSeqNo()); + if (randomBoolean()) { + globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getCheckpoint())); + } + if (rarely()) { + settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); + indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); + engine.onSettingsChanged(); + } + if (rarely()) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(true, true); + assertThat(Long.parseLong(engine.getLastCommittedSegmentInfos().userData.get(Engine.MIN_RETAINED_SEQNO)), + equalTo(engine.getMinRetainedSeqNo())); + } + if (rarely()) { + engine.forceMerge(randomBoolean()); + } + try (Closeable ignored = engine.acquireRetentionLockForPeerRecovery()) { + long minRetainSeqNos = engine.getMinRetainedSeqNo(); + assertThat(minRetainSeqNos, lessThanOrEqualTo(globalCheckpoint.get() + 1)); + Long[] expectedOps = existingSeqNos.stream().filter(seqno -> seqno >= minRetainSeqNos).toArray(Long[]::new); + Set actualOps = readAllOperationsInLucene(engine, createMapperService("test")).stream() + .map(Translog.Operation::seqNo).collect(Collectors.toSet()); + assertThat(actualOps, containsInAnyOrder(expectedOps)); + } + try (Engine.IndexCommitRef commitRef = engine.acquireSafeIndexCommit()) { + IndexCommit safeCommit = commitRef.getIndexCommit(); + if (safeCommit.getUserData().containsKey(Engine.MIN_RETAINED_SEQNO)) { + lastMinRetainedSeqNo = Long.parseLong(safeCommit.getUserData().get(Engine.MIN_RETAINED_SEQNO)); + } + } + } + if (randomBoolean()) { + engine.close(); + } else { + engine.flushAndClose(); + } + trimUnsafeCommits(engine.config()); + try (InternalEngine recoveringEngine = new InternalEngine(engine.config())) { + assertThat(recoveringEngine.getMinRetainedSeqNo(), equalTo(lastMinRetainedSeqNo)); + } + } + + public void testLastRefreshCheckpoint() throws Exception { + AtomicBoolean done = new AtomicBoolean(); + Thread[] refreshThreads = new Thread[between(1, 8)]; + CountDownLatch latch = new CountDownLatch(refreshThreads.length); + for (int i = 0; i < refreshThreads.length; i++) { + latch.countDown(); + refreshThreads[i] = new Thread(() -> { + while (done.get() == false) { + long checkPointBeforeRefresh = engine.getLocalCheckpoint(); + engine.refresh("test", randomFrom(Engine.SearcherScope.values())); + assertThat(engine.lastRefreshedCheckpoint(), greaterThanOrEqualTo(checkPointBeforeRefresh)); + } + }); + refreshThreads[i].start(); + } + latch.await(); + List ops = generateSingleDocHistory(true, VersionType.EXTERNAL, 1, 10, 1000, "1"); + concurrentlyApplyOps(ops, engine); + done.set(true); + for (Thread thread : refreshThreads) { + thread.join(); + } + engine.refresh("test"); + assertThat(engine.lastRefreshedCheckpoint(), equalTo(engine.getLocalCheckpoint())); + } + private static void trimUnsafeCommits(EngineConfig config) throws IOException { final Store store = config.getStore(); final TranslogConfig translogConfig = config.getTranslogConfig(); diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java new file mode 100644 index 00000000000..2d097366a27 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -0,0 +1,289 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.SnapshotMatchers; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.test.IndexSettingsModule; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class LuceneChangesSnapshotTests extends EngineTestCase { + private MapperService mapperService; + + @Before + public void createMapper() throws Exception { + mapperService = createMapperService("test"); + } + + @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", mapperService, fromSeqNo, toSeqNo, true)) { + 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", mapperService, fromSeqNo, toSeqNo, false)) { + 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.type(), doc.id(), newUid(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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, false)) { + 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { + 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, false)) { + 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { + 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, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { + 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", mapperService, fromSeqNo, toSeqNo, randomBoolean())) { + assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); + } + } + + public void testDedupByPrimaryTerm() throws Exception { + Map latestOperations = new HashMap<>(); + List terms = Arrays.asList(between(1, 1000), between(1000, 2000)); + int totalOps = 0; + for (long term : terms) { + final List ops = generateSingleDocHistory(true, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE), term, 2, 20, "1"); + primaryTerm.set(Math.max(primaryTerm.get(), term)); + engine.rollTranslogGeneration(); + for (Engine.Operation op : ops) { + // We need to simulate a rollback here as only ops after local checkpoint get into the engine + if (op.seqNo() <= engine.getLocalCheckpointTracker().getCheckpoint()) { + engine.getLocalCheckpointTracker().resetCheckpoint(randomLongBetween(-1, op.seqNo() - 1)); + engine.rollTranslogGeneration(); + } + if (op instanceof Engine.Index) { + engine.index((Engine.Index) op); + } else if (op instanceof Engine.Delete) { + engine.delete((Engine.Delete) op); + } + latestOperations.put(op.seqNo(), op.primaryTerm()); + if (rarely()) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(); + } + totalOps++; + } + } + long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); + try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", mapperService, 0, maxSeqNo, false)) { + Translog.Operation op; + while ((op = snapshot.next()) != null) { + assertThat(op.toString(), op.primaryTerm(), equalTo(latestOperations.get(op.seqNo()))); + } + assertThat(snapshot.skippedOperations(), equalTo(totalOps - latestOperations.size())); + } + } + + 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 = scaledRandomIntBetween(1, 1000); + for (int i = 0; i < numOps; i++) { + String id = Integer.toString(randomIntBetween(1, 10)); + 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.type(), doc.id(), newUid(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(); + concurrentlyApplyOps(operations, engine); + assertThat(engine.getLocalCheckpointTracker().getCheckpoint(), equalTo(operations.size() - 1L)); + isDone.set(true); + for (Follower follower : followers) { + follower.join(); + } + } + + class Follower extends Thread { + private final Engine leader; + private final TranslogHandler translogHandler; + private final AtomicBoolean isDone; + private final CountDownLatch readLatch; + + Follower(Engine leader, AtomicBoolean isDone, CountDownLatch readLatch) { + this.leader = leader; + this.isDone = isDone; + this.readLatch = readLatch; + this.translogHandler = new TranslogHandler(xContentRegistry(), IndexSettingsModule.newIndexSettings(shardId.getIndexName(), + engine.engineConfig.getIndexSettings().getSettings())); + } + + void pullOperations(Engine follower) throws IOException { + long leaderCheckpoint = leader.getLocalCheckpoint(); + long followerCheckpoint = follower.getLocalCheckpoint(); + 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", mapperService, fromSeqNo, toSeqNo, true)) { + translogHandler.run(follower, snapshot); + } + } + } + + @Override + public void run() { + try (Store store = createStore(); + InternalEngine follower = createEngine(store, createTempDir())) { + readLatch.countDown(); + readLatch.await(); + while (isDone.get() == false || + follower.getLocalCheckpointTracker().getCheckpoint() < leader.getLocalCheckpoint()) { + pullOperations(follower); + } + assertConsistentHistoryBetweenTranslogAndLuceneIndex(follower, mapperService); + assertThat(getDocIds(follower, true), equalTo(getDocIds(leader, true))); + } 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.error("Reading [{}]", op); + assert operations.stream().allMatch(o -> o.seqNo() < newOp.seqNo()) : "Operations [" + operations + "], op [" + op + "]"; + operations.add(newOp); + } + return operations; + } +} diff --git a/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java new file mode 100644 index 00000000000..c46b47b87d0 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java @@ -0,0 +1,161 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.CodecReader; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.MergePolicy; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SegmentCommitInfo; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.StandardDirectoryReader; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.NullInfoStream; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.Collections; +import java.util.Set; +import java.util.stream.Collectors; + +public class RecoverySourcePruneMergePolicyTests extends ESTestCase { + + public void testPruneAll() throws IOException { + try (Directory dir = newDirectory()) { + IndexWriterConfig iwc = newIndexWriterConfig(); + RecoverySourcePruneMergePolicy mp = new RecoverySourcePruneMergePolicy("extra_source", MatchNoDocsQuery::new, + newLogMergePolicy()); + iwc.setMergePolicy(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("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); + assertEquals(1, document.getFields().size()); + assertEquals("source", document.getFields().get(0).name()); + } + 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 && reader instanceof StandardDirectoryReader) { + CodecReader codecReader = (CodecReader) leafReader; + StandardDirectoryReader sdr = (StandardDirectoryReader) reader; + 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 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)); + } + } + } + } + } + + + public void testPruneSome() throws IOException { + try (Directory dir = newDirectory()) { + IndexWriterConfig iwc = newIndexWriterConfig(); + iwc.setMergePolicy(new RecoverySourcePruneMergePolicy("extra_source", + () -> 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 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() == 3) { + assertTrue(collect.contains("extra_source")); + assertEquals("true", document.getField("even").stringValue()); + assertEquals(i, extra_source.nextDoc()); + } else { + assertEquals(2, document.getFields().size()); + } + } + assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); + } + } + } + } +} diff --git a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java new file mode 100644 index 00000000000..f3590100382 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java @@ -0,0 +1,75 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + +import static org.hamcrest.Matchers.equalTo; + +public class SoftDeletesPolicyTests extends ESTestCase { + /** + * Makes sure we won't advance the retained seq# if the retention lock is held + */ + public void testSoftDeletesRetentionLock() { + long retainedOps = between(0, 10000); + AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + long safeCommitCheckpoint = globalCheckpoint.get(); + SoftDeletesPolicy policy = new SoftDeletesPolicy(globalCheckpoint::get, between(1, 10000), retainedOps); + long minRetainedSeqNo = policy.getMinRetainedSeqNo(); + List locks = new ArrayList<>(); + int iters = scaledRandomIntBetween(10, 1000); + for (int i = 0; i < iters; i++) { + if (randomBoolean()) { + locks.add(policy.acquireRetentionLock()); + } + // Advances the global checkpoint and the local checkpoint of a safe commit + globalCheckpoint.addAndGet(between(0, 1000)); + safeCommitCheckpoint = randomLongBetween(safeCommitCheckpoint, globalCheckpoint.get()); + policy.setLocalCheckpointOfSafeCommit(safeCommitCheckpoint); + if (rarely()) { + retainedOps = between(0, 10000); + policy.setRetentionOperations(retainedOps); + } + // Release some locks + List releasingLocks = randomSubsetOf(locks); + locks.removeAll(releasingLocks); + releasingLocks.forEach(Releasable::close); + + // We only expose the seqno to the merge policy if the retention lock is not held. + policy.getRetentionQuery(); + if (locks.isEmpty()) { + long retainedSeqNo = Math.min(safeCommitCheckpoint, globalCheckpoint.get() - retainedOps) + 1; + minRetainedSeqNo = Math.max(minRetainedSeqNo, retainedSeqNo); + } + assertThat(policy.getMinRetainedSeqNo(), equalTo(minRetainedSeqNo)); + } + + locks.forEach(Releasable::close); + long retainedSeqNo = Math.min(safeCommitCheckpoint, globalCheckpoint.get() - retainedOps) + 1; + minRetainedSeqNo = Math.max(minRetainedSeqNo, retainedSeqNo); + assertThat(policy.getMinRetainedSeqNo(), equalTo(minRetainedSeqNo)); + } +} diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java index 76ca6aa7ea8..5a46b9a889f 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -311,15 +312,18 @@ public class DocumentParserTests extends ESSingleNodeTestCase { // creates an object mapper, which is about 100x harder than it should be.... ObjectMapper createObjectMapper(MapperService mapperService, String name) throws Exception { - ParseContext context = new ParseContext.InternalParseContext( - Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(), + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + ParseContext context = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), mapperService.documentMapper("type"), null, null); String[] nameParts = name.split("\\."); for (int i = 0; i < nameParts.length - 1; ++i) { context.path().add(nameParts[i]); } Mapper.Builder builder = new ObjectMapper.Builder(nameParts[nameParts.length - 1]).enabled(true); - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); return (ObjectMapper)builder.build(builderContext); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java index cb2ed785699..b11e4876f9e 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.BooleanFieldMapper.BooleanFieldType; import org.elasticsearch.index.mapper.DateFieldMapper.DateFieldType; import org.elasticsearch.index.mapper.NumberFieldMapper.NumberFieldType; @@ -215,7 +216,10 @@ public class DynamicMappingTests extends ESSingleNodeTestCase { } private Mapper parse(DocumentMapper mapper, DocumentMapperParser parser, XContentBuilder builder) throws Exception { - Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); + IndexMetaData build = IndexMetaData.builder("") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); + IndexSettings settings = new IndexSettings(build, Settings.EMPTY); SourceToParse source = SourceToParse.source("test", mapper.type(), "some_id", BytesReference.bytes(builder), builder.contentType()); try (XContentParser xContentParser = createParser(JsonXContent.jsonXContent, source.source())) { ParseContext.InternalParseContext ctx = new ParseContext.InternalParseContext(settings, parser, mapper, source, xContentParser); 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 1d1e423afc1..fba71dd1e52 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.index.replication; +import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; @@ -41,6 +42,7 @@ import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.engine.InternalEngineTests; import org.elasticsearch.index.engine.SegmentsStats; import org.elasticsearch.index.engine.VersionConflictEngineException; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -140,7 +142,9 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase } public void testInheritMaxValidAutoIDTimestampOnRecovery() throws Exception { - try (ReplicationGroup shards = createGroup(0)) { + //TODO: Enables this test with soft-deletes once we have timestamp + Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); + try (ReplicationGroup shards = createGroup(0, settings)) { shards.startAll(); final IndexRequest indexRequest = new IndexRequest(index.getName(), "type").source("{}", XContentType.JSON); indexRequest.onRetry(); // force an update of the timestamp @@ -346,7 +350,13 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase final AtomicBoolean throwAfterIndexedOneDoc = new AtomicBoolean(); // need one document to trigger delete in IW. @Override public long addDocument(Iterable doc) throws IOException { - if (throwAfterIndexedOneDoc.getAndSet(true)) { + boolean isTombstone = false; + for (IndexableField field : doc) { + if (SeqNoFieldMapper.TOMBSTONE_NAME.equals(field.name())) { + isTombstone = true; + } + } + if (isTombstone == false && throwAfterIndexedOneDoc.getAndSet(true)) { throw indexException; } else { return super.addDocument(doc); @@ -356,6 +366,10 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase public long deleteDocuments(Term... terms) throws IOException { throw deleteException; } + @Override + public long softUpdateDocument(Term term, Iterable doc, Field...fields) throws IOException { + throw deleteException; // a delete uses softUpdateDocument API if soft-deletes enabled + } }, null, null, config); try (ReplicationGroup shards = new ReplicationGroup(buildIndexMetaData(0)) { @Override @@ -390,6 +404,9 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase try (Translog.Snapshot snapshot = getTranslog(shard).newSnapshot()) { assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); } + try (Translog.Snapshot snapshot = shard.getHistoryOperations("test", 0)) { + assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); + } } // unlike previous failures, these two failures replicated directly from the replication channel. indexResp = shards.index(new IndexRequest(index.getName(), "type", "any").source("{}", XContentType.JSON)); @@ -404,6 +421,9 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase try (Translog.Snapshot snapshot = getTranslog(shard).newSnapshot()) { assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); } + try (Translog.Snapshot snapshot = shard.getHistoryOperations("test", 0)) { + assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); + } } shards.assertAllEqual(1); } @@ -501,8 +521,9 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase recoverReplica(replica3, replica2, true); try (Translog.Snapshot snapshot = getTranslog(replica3).newSnapshot()) { assertThat(snapshot.totalOperations(), equalTo(initDocs + 1)); - assertThat(snapshot.next(), equalTo(op2)); - assertThat("Remaining of snapshot should contain init operations", snapshot, containsOperationsInAnyOrder(initOperations)); + final List expectedOps = new ArrayList<>(initOperations); + expectedOps.add(op2); + assertThat(snapshot, containsOperationsInAnyOrder(expectedOps)); assertThat("Peer-recovery should not send overridden operations", snapshot.skippedOperations(), equalTo(0)); } // TODO: We should assert the content of shards in the ReplicationGroup. diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index 2d198c32ba7..28122665e9b 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -98,7 +98,8 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC } public void testRecoveryOfDisconnectedReplica() throws Exception { - try (ReplicationGroup shards = createGroup(1)) { + Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); + try (ReplicationGroup shards = createGroup(1, settings)) { shards.startAll(); int docs = shards.indexDocs(randomInt(50)); shards.flush(); @@ -266,6 +267,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC builder.settings(Settings.builder().put(newPrimary.indexSettings().getSettings()) .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) ); newPrimary.indexSettings().updateIndexMetaData(builder.build()); newPrimary.onSettingsChanged(); @@ -275,7 +277,12 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC shards.syncGlobalCheckpoint(); assertThat(newPrimary.getLastSyncedGlobalCheckpoint(), equalTo(newPrimary.seqNoStats().getMaxSeqNo())); }); - newPrimary.flush(new FlushRequest()); + newPrimary.flush(new FlushRequest().force(true)); + if (replica.indexSettings().isSoftDeleteEnabled()) { + // We need an extra flush to advance the min_retained_seqno on the new primary so ops-based won't happen. + // The min_retained_seqno only advances when a merge asks for the retention query. + newPrimary.flush(new FlushRequest().force(true)); + } uncommittedOpsOnPrimary = shards.indexDocs(randomIntBetween(0, 10)); totalDocs += uncommittedOpsOnPrimary; } 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 2228e1b017f..50f95bf4d47 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.TermQuery; @@ -30,6 +31,7 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; +import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Constants; import org.elasticsearch.Assertions; import org.elasticsearch.Version; @@ -89,8 +91,13 @@ import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.ParseContext; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; @@ -160,6 +167,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.hasToString; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.lessThan; @@ -237,7 +245,8 @@ public class IndexShardTests extends IndexShardTestCase { assertNotNull(shardPath); // fail shard shard.failShard("test shard fail", new CorruptIndexException("", "")); - closeShards(shard); + shard.close("do not assert history", false); + shard.store().close(); // check state file still exists ShardStateMetaData shardStateMetaData = load(logger, shardPath.getShardStatePath()); assertEquals(shardStateMetaData, getShardStateMetadata(shard)); @@ -2394,7 +2403,8 @@ public class IndexShardTests extends IndexShardTestCase { public void testDocStats() throws IOException, InterruptedException { IndexShard indexShard = null; try { - indexShard = newStartedShard(); + indexShard = newStartedShard( + Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0).build()); final long numDocs = randomIntBetween(2, 32); // at least two documents so we have docs to delete final long numDocsToDelete = randomLongBetween(1, numDocs); for (int i = 0; i < numDocs; i++) { @@ -2424,7 +2434,16 @@ public class IndexShardTests extends IndexShardTestCase { deleteDoc(indexShard, "_doc", id); indexDoc(indexShard, "_doc", id); } - + // Need to update and sync the global checkpoint as the soft-deletes retention MergePolicy depends on it. + if (indexShard.indexSettings.isSoftDeleteEnabled()) { + if (indexShard.routingEntry().primary()) { + indexShard.updateGlobalCheckpointForShard(indexShard.routingEntry().allocationId().getId(), + indexShard.getLocalCheckpoint()); + } else { + indexShard.updateGlobalCheckpointOnReplica(indexShard.getLocalCheckpoint(), "test"); + } + indexShard.sync(); + } // flush the buffered deletes final FlushRequest flushRequest = new FlushRequest(); flushRequest.force(false); @@ -2962,6 +2981,7 @@ public class IndexShardTests extends IndexShardTestCase { assertThat(breaker.getUsed(), greaterThan(preRefreshBytes)); indexDoc(primary, "_doc", "4", "{\"foo\": \"potato\"}"); + indexDoc(primary, "_doc", "5", "{\"foo\": \"potato\"}"); // Forces a refresh with the INTERNAL scope ((InternalEngine) primary.getEngine()).writeIndexingBuffer(); @@ -2973,6 +2993,13 @@ public class IndexShardTests extends IndexShardTestCase { // Deleting a doc causes its memory to be freed from the breaker deleteDoc(primary, "_doc", "0"); + // Here we are testing that a fully deleted segment should be dropped and its memory usage is freed. + // In order to instruct the merge policy not to keep a fully deleted segment, + // we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything. + if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { + primary.sync(); + flushShard(primary); + } primary.refresh("force refresh"); ss = primary.segmentStats(randomBoolean()); @@ -3064,6 +3091,7 @@ public class IndexShardTests extends IndexShardTestCase { // Close remaining searchers IOUtils.close(searchers); + primary.refresh("test"); SegmentsStats ss = primary.segmentStats(randomBoolean()); CircuitBreaker breaker = primary.circuitBreakerService.getBreaker(CircuitBreaker.ACCOUNTING); @@ -3181,4 +3209,28 @@ public class IndexShardTests extends IndexShardTestCase { } + public void testSupplyTombstoneDoc() throws Exception { + IndexShard shard = newStartedShard(); + String id = randomRealisticUnicodeOfLengthBetween(1, 10); + ParsedDocument deleteTombstone = shard.getEngine().config().getTombstoneDocSupplier().newDeleteTombstoneDoc("doc", id); + assertThat(deleteTombstone.docs(), hasSize(1)); + ParseContext.Document deleteDoc = deleteTombstone.docs().get(0); + assertThat(deleteDoc.getFields().stream().map(IndexableField::name).collect(Collectors.toList()), + containsInAnyOrder(IdFieldMapper.NAME, VersionFieldMapper.NAME, + SeqNoFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME)); + assertThat(deleteDoc.getField(IdFieldMapper.NAME).binaryValue(), equalTo(Uid.encodeId(id))); + assertThat(deleteDoc.getField(SeqNoFieldMapper.TOMBSTONE_NAME).numericValue().longValue(), equalTo(1L)); + + final String reason = randomUnicodeOfLength(200); + ParsedDocument noopTombstone = shard.getEngine().config().getTombstoneDocSupplier().newNoopTombstoneDoc(reason); + assertThat(noopTombstone.docs(), hasSize(1)); + ParseContext.Document noopDoc = noopTombstone.docs().get(0); + assertThat(noopDoc.getFields().stream().map(IndexableField::name).collect(Collectors.toList()), + containsInAnyOrder(VersionFieldMapper.NAME, SourceFieldMapper.NAME, SeqNoFieldMapper.TOMBSTONE_NAME, + SeqNoFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME)); + assertThat(noopDoc.getField(SeqNoFieldMapper.TOMBSTONE_NAME).numericValue().longValue(), equalTo(1L)); + assertThat(noopDoc.getField(SourceFieldMapper.NAME).binaryValue(), equalTo(new BytesRef(reason))); + + closeShards(shard); + } } diff --git a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java index ae2cc84e487..29b16ca28f4 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java @@ -106,17 +106,22 @@ public class PrimaryReplicaSyncerTests extends IndexShardTestCase { .isPresent(), is(false)); } - - assertEquals(globalCheckPoint == numDocs - 1 ? 0 : numDocs, resyncTask.getTotalOperations()); if (syncNeeded && globalCheckPoint < numDocs - 1) { - long skippedOps = globalCheckPoint + 1; // everything up to global checkpoint included - assertEquals(skippedOps, resyncTask.getSkippedOperations()); - assertEquals(numDocs - skippedOps, resyncTask.getResyncedOperations()); + if (shard.indexSettings.isSoftDeleteEnabled()) { + assertThat(resyncTask.getSkippedOperations(), equalTo(0)); + assertThat(resyncTask.getResyncedOperations(), equalTo(resyncTask.getTotalOperations())); + assertThat(resyncTask.getTotalOperations(), equalTo(Math.toIntExact(numDocs - 1 - globalCheckPoint))); + } else { + int skippedOps = Math.toIntExact(globalCheckPoint + 1); // everything up to global checkpoint included + assertThat(resyncTask.getSkippedOperations(), equalTo(skippedOps)); + assertThat(resyncTask.getResyncedOperations(), equalTo(numDocs - skippedOps)); + assertThat(resyncTask.getTotalOperations(), equalTo(globalCheckPoint == numDocs - 1 ? 0 : numDocs)); + } } else { - assertEquals(0, resyncTask.getSkippedOperations()); - assertEquals(0, resyncTask.getResyncedOperations()); + assertThat(resyncTask.getSkippedOperations(), equalTo(0)); + assertThat(resyncTask.getResyncedOperations(), equalTo(0)); + assertThat(resyncTask.getTotalOperations(), equalTo(0)); } - closeShards(shard); } 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 774b272121a..b93f170174c 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; +import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; import org.elasticsearch.index.mapper.IdFieldMapper; @@ -130,7 +131,8 @@ public class RefreshListenersTests extends ESTestCase { indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), Collections.singletonList(listeners), Collections.emptyList(), null, - (e, s) -> 0, new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, () -> primaryTerm); + (e, s) -> 0, new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, () -> primaryTerm, + EngineTestCase.tombstoneDocSupplier()); engine = new InternalEngine(config); engine.recoverFromTranslog(Long.MAX_VALUE); listeners.setCurrentRefreshLocationSupplier(engine::getTranslogLastWriteLocation); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index 89a8813e3e0..81afab4bb8f 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -67,6 +67,7 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; +import org.junit.After; import java.io.IOException; import java.util.ArrayList; @@ -110,6 +111,11 @@ public class IndexRecoveryIT extends ESIntegTestCase { RecoverySettingsChunkSizePlugin.class); } + @After + public void assertConsistentHistoryInLuceneIndex() throws Exception { + internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); + } + private void assertRecoveryStateWithoutStage(RecoveryState state, int shardId, RecoverySource recoverySource, boolean primary, String sourceNode, String targetNode) { assertThat(state.getShardId().getId(), equalTo(shardId)); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java index 4b1419375e6..b6f5a7b6451 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -25,6 +25,7 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.NoMergePolicy; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; @@ -91,6 +92,7 @@ public class PeerRecoveryTargetServiceTests extends IndexShardTestCase { replica.close("test", false); final List commits = DirectoryReader.listCommits(replica.store().directory()); IndexWriterConfig iwc = new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) .setMergePolicy(NoMergePolicy.INSTANCE) .setOpenMode(IndexWriterConfig.OpenMode.APPEND); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index f0644b029c3..0351111c305 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -411,12 +411,6 @@ public class RecoverySourceHandlerTests extends ESTestCase { recoverySettings.getChunkSize().bytesAsInt(), Settings.EMPTY) { - - @Override - boolean isTranslogReadyForSequenceNumberBasedRecovery() throws IOException { - return randomBoolean(); - } - @Override public void phase1(final IndexCommit snapshot, final Supplier translogOps) { phase1Called.set(true); 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 5547a629ab2..45535e19672 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.SourceToParse; @@ -63,13 +64,13 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { int docs = shards.indexDocs(10); getTranslog(shards.getPrimary()).rollGeneration(); shards.flush(); - if (randomBoolean()) { - docs += shards.indexDocs(10); - } + int moreDocs = shards.indexDocs(randomInt(10)); shards.addReplica(); shards.startAll(); final IndexShard replica = shards.getReplicas().get(0); - assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(docs)); + boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); + assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? moreDocs : docs + moreDocs)); + shards.assertAllEqual(docs + moreDocs); } } @@ -101,12 +102,12 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { // rolling/flushing is async assertBusy(() -> { assertThat(replica.getLastSyncedGlobalCheckpoint(), equalTo(19L)); - assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(0)); + assertThat(getTranslog(replica).totalOperations(), equalTo(0)); }); } } - public void testRecoveryWithOutOfOrderDelete() throws Exception { + public void testRecoveryWithOutOfOrderDeleteWithTranslog() throws Exception { /* * The flow of this test: * - delete #1 @@ -118,7 +119,8 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { * - index #5 * - If flush and the translog retention disabled, delete #1 will be removed while index #0 is still retained and replayed. */ - try (ReplicationGroup shards = createGroup(1)) { + Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); + try (ReplicationGroup shards = createGroup(1, settings)) { shards.startAll(); // create out of order delete and index op on replica final IndexShard orgReplica = shards.getReplicas().get(0); @@ -170,7 +172,63 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(newReplica); shards.assertAllEqual(3); - assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(translogOps)); + assertThat(getTranslog(newReplica).totalOperations(), equalTo(translogOps)); + } + } + + public void testRecoveryWithOutOfOrderDeleteWithSoftDeletes() throws Exception { + Settings settings = Settings.builder() + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 10) + // If soft-deletes is enabled, delete#1 will be reclaimed because its segment (segment_1) is fully deleted + // index#0 will be retained if merge is disabled; otherwise it will be reclaimed because gcp=3 and retained_ops=0 + .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false).build(); + try (ReplicationGroup shards = createGroup(1, settings)) { + shards.startAll(); + // create out of order delete and index op on replica + final IndexShard orgReplica = shards.getReplicas().get(0); + final String indexName = orgReplica.shardId().getIndexName(); + + // delete #1 + orgReplica.applyDeleteOperationOnReplica(1, 2, "type", "id"); + orgReplica.flush(new FlushRequest().force(true)); // isolate delete#1 in its own translog generation and lucene segment + // index #0 + orgReplica.applyIndexOperationOnReplica(0, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(indexName, "type", "id", new BytesArray("{}"), XContentType.JSON)); + // index #3 + orgReplica.applyIndexOperationOnReplica(3, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(indexName, "type", "id-3", new BytesArray("{}"), XContentType.JSON)); + // Flushing a new commit with local checkpoint=1 allows to delete the translog gen #1. + orgReplica.flush(new FlushRequest().force(true).waitIfOngoing(true)); + // index #2 + orgReplica.applyIndexOperationOnReplica(2, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(indexName, "type", "id-2", new BytesArray("{}"), XContentType.JSON)); + orgReplica.updateGlobalCheckpointOnReplica(3L, "test"); + // index #5 -> force NoOp #4. + orgReplica.applyIndexOperationOnReplica(5, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(indexName, "type", "id-5", new BytesArray("{}"), XContentType.JSON)); + + if (randomBoolean()) { + if (randomBoolean()) { + logger.info("--> flushing shard (translog/soft-deletes will be trimmed)"); + IndexMetaData.Builder builder = IndexMetaData.builder(orgReplica.indexSettings().getIndexMetaData()); + builder.settings(Settings.builder().put(orgReplica.indexSettings().getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0)); + orgReplica.indexSettings().updateIndexMetaData(builder.build()); + orgReplica.onSettingsChanged(); + } + flushShard(orgReplica); + } + + final IndexShard orgPrimary = shards.getPrimary(); + shards.promoteReplicaToPrimary(orgReplica).get(); // wait for primary/replica sync to make sure seq# gap is closed. + + IndexShard newReplica = shards.addReplicaWithExistingPath(orgPrimary.shardPath(), orgPrimary.routingEntry().currentNodeId()); + shards.recoverReplica(newReplica); + shards.assertAllEqual(3); + try (Translog.Snapshot snapshot = newReplica.getHistoryOperations("test", 0)) { + assertThat(snapshot, SnapshotMatchers.size(6)); + } } } @@ -222,7 +280,8 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(newReplica); // file based recovery should be made assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty())); - assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); + boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); + assertThat(getTranslog(newReplica).totalOperations(), equalTo(softDeletesEnabled ? nonFlushedDocs : numDocs)); // history uuid was restored assertThat(newReplica.getHistoryUUID(), equalTo(historyUUID)); @@ -326,7 +385,8 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(replica); // Make sure the flushing will eventually be completed (eg. `shouldPeriodicallyFlush` is false) assertBusy(() -> assertThat(getEngine(replica).shouldPeriodicallyFlush(), equalTo(false))); - assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); + boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); + assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? 0 : numDocs)); shards.assertAllEqual(numDocs); } } diff --git a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java index fa591411bba..ce162b9600c 100644 --- a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java @@ -43,6 +43,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexModule; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.MergeSchedulerConfig; @@ -50,6 +51,7 @@ import org.elasticsearch.index.VersionType; import org.elasticsearch.index.cache.query.QueryCacheStats; import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesQueryCache; import org.elasticsearch.indices.IndicesRequestCache; @@ -69,6 +71,7 @@ import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.Random; +import java.util.Set; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; @@ -115,6 +118,7 @@ public class IndexStatsIT extends ESIntegTestCase { return Settings.builder().put(super.indexSettings()) .put(IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.getKey(), true) .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) .build(); } @@ -1006,10 +1010,15 @@ public class IndexStatsIT extends ESIntegTestCase { @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/32506") public void testFilterCacheStats() throws Exception { - assertAcked(prepareCreate("index").setSettings(Settings.builder().put(indexSettings()).put("number_of_replicas", 0).build()).get()); - indexRandom(true, + Settings settings = Settings.builder().put(indexSettings()).put("number_of_replicas", 0).build(); + assertAcked(prepareCreate("index").setSettings(settings).get()); + indexRandom(false, true, client().prepareIndex("index", "type", "1").setSource("foo", "bar"), client().prepareIndex("index", "type", "2").setSource("foo", "baz")); + if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { + persistGlobalCheckpoint("index"); // Need to persist the global checkpoint for the soft-deletes retention MP. + } + refresh(); ensureGreen(); IndicesStatsResponse response = client().admin().indices().prepareStats("index").setQueryCache(true).get(); @@ -1040,6 +1049,13 @@ public class IndexStatsIT extends ESIntegTestCase { assertEquals(DocWriteResponse.Result.DELETED, client().prepareDelete("index", "type", "1").get().getResult()); assertEquals(DocWriteResponse.Result.DELETED, client().prepareDelete("index", "type", "2").get().getResult()); + // Here we are testing that a fully deleted segment should be dropped and its cached is evicted. + // In order to instruct the merge policy not to keep a fully deleted segment, + // we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything. + if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { + persistGlobalCheckpoint("index"); + flush("index"); + } refresh(); response = client().admin().indices().prepareStats("index").setQueryCache(true).get(); assertCumulativeQueryCacheStats(response); @@ -1173,4 +1189,21 @@ public class IndexStatsIT extends ESIntegTestCase { assertThat(executionFailures.get(), emptyCollectionOf(Exception.class)); } + + /** + * Persist the global checkpoint on all shards of the given index into disk. + * This makes sure that the persisted global checkpoint on those shards will equal to the in-memory value. + */ + private void persistGlobalCheckpoint(String index) throws Exception { + final Set nodes = internalCluster().nodesInclude(index); + for (String node : nodes) { + final IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node); + for (IndexService indexService : indexServices) { + for (IndexShard indexShard : indexService) { + indexShard.sync(); + assertThat(indexShard.getLastSyncedGlobalCheckpoint(), equalTo(indexShard.getGlobalCheckpoint())); + } + } + } + } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java b/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java index 23c56688e00..c25cad61e07 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -27,6 +27,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.snapshots.mockstore.MockRepository; import org.elasticsearch.test.ESIntegTestCase; +import org.junit.After; import java.io.IOException; import java.nio.file.FileVisitResult; @@ -58,6 +59,11 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase { return Arrays.asList(MockRepository.Plugin.class); } + @After + public void assertConsistentHistoryInLuceneIndex() throws Exception { + internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); + } + public static long getFailureCount(String repository) { long failureCount = 0; for (RepositoriesService repositoriesService : diff --git a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index 1230d594b98..632a1ecbee1 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -122,6 +122,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF import static org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.IndexSettings.INDEX_REFRESH_INTERVAL_SETTING; +import static org.elasticsearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; import static org.elasticsearch.index.query.QueryBuilders.matchQuery; import static org.elasticsearch.index.shard.IndexShardTests.getEngineFromShard; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -2048,7 +2049,9 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES))); // only one shard - assertAcked(prepareCreate("test").setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1))); + final Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).build(); + assertAcked(prepareCreate("test").setSettings(indexSettings)); ensureGreen(); logger.info("--> indexing"); @@ -2094,7 +2097,13 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas SnapshotStatus snapshotStatus = client.admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("test-2").get().getSnapshots().get(0); List shards = snapshotStatus.getShards(); for (SnapshotIndexShardStatus status : shards) { - assertThat(status.getStats().getProcessedFileCount(), equalTo(2)); // we flush before the snapshot such that we have to process the segments_N files plus the .del file + // we flush before the snapshot such that we have to process the segments_N files plus the .del file + if (INDEX_SOFT_DELETES_SETTING.get(indexSettings)) { + // soft-delete generates DV files. + assertThat(status.getStats().getProcessedFileCount(), greaterThan(2)); + } else { + assertThat(status.getStats().getProcessedFileCount(), equalTo(2)); + } } } } diff --git a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java index caf4f725fa4..588118db4ae 100644 --- a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java +++ b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.VersionType; @@ -785,4 +786,26 @@ public class SimpleVersioningIT extends ESIntegTestCase { .getVersion(), equalTo(-1L)); } + + public void testSpecialVersioning() { + internalCluster().ensureAtLeastNumDataNodes(2); + createIndex("test", Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).build()); + IndexResponse doc1 = client().prepareIndex("test", "type", "1").setSource("field", "value1") + .setVersion(0).setVersionType(VersionType.EXTERNAL).execute().actionGet(); + assertThat(doc1.getVersion(), equalTo(0L)); + IndexResponse doc2 = client().prepareIndex("test", "type", "1").setSource("field", "value2") + .setVersion(Versions.MATCH_ANY).setVersionType(VersionType.INTERNAL).execute().actionGet(); + assertThat(doc2.getVersion(), equalTo(1L)); + client().prepareDelete("test", "type", "1").get(); //v2 + IndexResponse doc3 = client().prepareIndex("test", "type", "1").setSource("field", "value3") + .setVersion(Versions.MATCH_DELETED).setVersionType(VersionType.INTERNAL).execute().actionGet(); + assertThat(doc3.getVersion(), equalTo(3L)); + IndexResponse doc4 = client().prepareIndex("test", "type", "1").setSource("field", "value4") + .setVersion(4L).setVersionType(VersionType.EXTERNAL_GTE).execute().actionGet(); + assertThat(doc4.getVersion(), equalTo(4L)); + // Make sure that these versions are replicated correctly + client().admin().indices().prepareUpdateSettings("test") + .setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)).get(); + ensureGreen("test"); + } } 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 b5ba5f18b39..b558cd1ba90 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 @@ -19,14 +19,18 @@ package org.elasticsearch.index.engine; +import org.apache.logging.log4j.Logger; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.StoredField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.Term; @@ -34,32 +38,41 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.search.Sort; +import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TotalHitCountCollector; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.MapperTestUtils; +import org.elasticsearch.index.VersionType; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -80,17 +93,30 @@ import org.junit.Before; import java.io.IOException; import java.nio.charset.Charset; import java.nio.file.Path; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; +import java.util.function.Function; import java.util.function.LongSupplier; import java.util.function.ToLongBiFunction; +import java.util.stream.Collectors; import static java.util.Collections.emptyList; +import static java.util.Collections.shuffle; +import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; +import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; public abstract class EngineTestCase extends ESTestCase { @@ -128,6 +154,20 @@ public abstract class EngineTestCase extends ESTestCase { } } + protected Settings indexSettings() { + // TODO randomize more settings + return Settings.builder() + .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), "1h") // make sure this doesn't kick in on us + .put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName) + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), + between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), + randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) + .build(); + } + @Override @Before public void setUp() throws Exception { @@ -142,13 +182,7 @@ public abstract class EngineTestCase extends ESTestCase { } else { codecName = "default"; } - defaultSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder() - .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), "1h") // make sure this doesn't kick in on us - .put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName) - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), - between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) - .build()); // TODO randomize more settings + defaultSettings = IndexSettingsModule.newIndexSettings("test", indexSettings()); threadPool = new TestThreadPool(getClass().getName()); store = createStore(); storeReplica = createStore(); @@ -180,7 +214,7 @@ public abstract class EngineTestCase extends ESTestCase { new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService(), globalCheckpointSupplier, config.getPrimaryTermSupplier()); + config.getCircuitBreakerService(), globalCheckpointSupplier, config.getPrimaryTermSupplier(), tombstoneDocSupplier()); } public EngineConfig copy(EngineConfig config, Analyzer analyzer) { @@ -189,7 +223,18 @@ public abstract class EngineTestCase extends ESTestCase { new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier()); + config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier(), + config.getTombstoneDocSupplier()); + } + + public EngineConfig copy(EngineConfig config, MergePolicy mergePolicy) { + return new EngineConfig(config.getShardId(), config.getAllocationId(), config.getThreadPool(), config.getIndexSettings(), + config.getWarmer(), config.getStore(), mergePolicy, config.getAnalyzer(), config.getSimilarity(), + new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), + config.getTranslogConfig(), config.getFlushMergesAfter(), + config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), + config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier(), + config.getTombstoneDocSupplier()); } @Override @@ -198,9 +243,11 @@ public abstract class EngineTestCase extends ESTestCase { super.tearDown(); if (engine != null && engine.isClosed.get() == false) { engine.getTranslog().getDeletionPolicy().assertNoOpenTranslogRefs(); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, createMapperService("test")); } if (replicaEngine != null && replicaEngine.isClosed.get() == false) { replicaEngine.getTranslog().getDeletionPolicy().assertNoOpenTranslogRefs(); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(replicaEngine, createMapperService("test")); } IOUtils.close( replicaEngine, storeReplica, @@ -228,8 +275,18 @@ public abstract class EngineTestCase extends ESTestCase { return testParsedDocument(id, routing, testDocumentWithTextField(), new BytesArray("{ \"value\" : \"test\" }"), null); } + public static ParsedDocument createParsedDoc(String id, String routing, boolean recoverySource) { + return testParsedDocument(id, routing, testDocumentWithTextField(), new BytesArray("{ \"value\" : \"test\" }"), null, + recoverySource); + } + protected static ParsedDocument testParsedDocument( String id, String routing, ParseContext.Document document, BytesReference source, Mapping mappingUpdate) { + return testParsedDocument(id, routing, document, source, mappingUpdate, false); + } + protected static ParsedDocument testParsedDocument( + String id, String routing, ParseContext.Document document, BytesReference source, Mapping mappingUpdate, + boolean recoverySource) { Field uidField = new Field("_id", Uid.encodeId(id), IdFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); @@ -239,11 +296,57 @@ public abstract class EngineTestCase extends ESTestCase { document.add(seqID.seqNoDocValue); document.add(seqID.primaryTerm); BytesRef ref = source.toBytesRef(); - document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length)); + if (recoverySource) { + document.add(new StoredField(SourceFieldMapper.RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); + document.add(new NumericDocValuesField(SourceFieldMapper.RECOVERY_SOURCE_NAME, 1)); + } else { + document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length)); + } return new ParsedDocument(versionField, seqID, id, "test", routing, Arrays.asList(document), source, XContentType.JSON, mappingUpdate); } + /** + * Creates a tombstone document that only includes uid, seq#, term and version fields. + */ + public static EngineConfig.TombstoneDocSupplier tombstoneDocSupplier(){ + return new EngineConfig.TombstoneDocSupplier() { + @Override + public ParsedDocument newDeleteTombstoneDoc(String type, String id) { + final ParseContext.Document doc = new ParseContext.Document(); + Field uidField = new Field(IdFieldMapper.NAME, Uid.encodeId(id), IdFieldMapper.Defaults.FIELD_TYPE); + doc.add(uidField); + Field versionField = new NumericDocValuesField(VersionFieldMapper.NAME, 0); + doc.add(versionField); + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); + doc.add(seqID.seqNo); + doc.add(seqID.seqNoDocValue); + doc.add(seqID.primaryTerm); + seqID.tombstoneField.setLongValue(1); + doc.add(seqID.tombstoneField); + return new ParsedDocument(versionField, seqID, id, type, null, + Collections.singletonList(doc), new BytesArray("{}"), XContentType.JSON, null); + } + + @Override + public ParsedDocument newNoopTombstoneDoc(String reason) { + final ParseContext.Document doc = new ParseContext.Document(); + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); + doc.add(seqID.seqNo); + doc.add(seqID.seqNoDocValue); + doc.add(seqID.primaryTerm); + seqID.tombstoneField.setLongValue(1); + doc.add(seqID.tombstoneField); + Field versionField = new NumericDocValuesField(VersionFieldMapper.NAME, 0); + doc.add(versionField); + BytesRef byteRef = new BytesRef(reason); + doc.add(new StoredField(SourceFieldMapper.NAME, byteRef.bytes, byteRef.offset, byteRef.length)); + return new ParsedDocument(versionField, seqID, null, null, null, + Collections.singletonList(doc), null, XContentType.JSON, null); + } + }; + } + protected Store createStore() throws IOException { return createStore(newDirectory()); } @@ -461,7 +564,7 @@ public abstract class EngineTestCase extends ESTestCase { new NoneCircuitBreakerService(), globalCheckpointSupplier == null ? new ReplicationTracker(shardId, allocationId.getId(), indexSettings, SequenceNumbers.NO_OPS_PERFORMED, update -> {}) : - globalCheckpointSupplier, primaryTerm::get); + globalCheckpointSupplier, primaryTerm::get, tombstoneDocSupplier()); return config; } @@ -474,7 +577,7 @@ public abstract class EngineTestCase extends ESTestCase { return new BytesArray(string.getBytes(Charset.defaultCharset())); } - protected Term newUid(String id) { + protected static Term newUid(String id) { return new Term("_id", Uid.encodeId(id)); } @@ -499,6 +602,279 @@ public abstract class EngineTestCase extends ESTestCase { protected Engine.Delete replicaDeleteForDoc(String id, long version, long seqNo, long startTime) { return new Engine.Delete("test", id, newUid(id), seqNo, 1, version, null, Engine.Operation.Origin.REPLICA, startTime); } + protected static void assertVisibleCount(InternalEngine engine, int numDocs) throws IOException { + assertVisibleCount(engine, numDocs, true); + } + + protected static void assertVisibleCount(InternalEngine engine, int numDocs, boolean refresh) throws IOException { + if (refresh) { + engine.refresh("test"); + } + try (Engine.Searcher searcher = engine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new MatchAllDocsQuery(), collector); + assertThat(collector.getTotalHits(), equalTo(numDocs)); + } + } + + public static List generateSingleDocHistory(boolean forReplica, VersionType versionType, + long primaryTerm, int minOpCount, int maxOpCount, String docId) { + final int numOfOps = randomIntBetween(minOpCount, maxOpCount); + final List ops = new ArrayList<>(); + final Term id = newUid(docId); + final int startWithSeqNo = 0; + final String valuePrefix = (forReplica ? "r_" : "p_" ) + docId + "_"; + final boolean incrementTermWhenIntroducingSeqNo = randomBoolean(); + for (int i = 0; i < numOfOps; i++) { + final Engine.Operation op; + final long version; + switch (versionType) { + case INTERNAL: + version = forReplica ? i : Versions.MATCH_ANY; + break; + case EXTERNAL: + version = i; + break; + case EXTERNAL_GTE: + version = randomBoolean() ? Math.max(i - 1, 0) : i; + break; + case FORCE: + version = randomNonNegativeLong(); + break; + default: + throw new UnsupportedOperationException("unknown version type: " + versionType); + } + if (randomBoolean()) { + op = new Engine.Index(id, testParsedDocument(docId, null, testDocumentWithTextField(valuePrefix + i), B_1, null), + forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, + version, + forReplica ? null : versionType, + forReplica ? REPLICA : PRIMARY, + System.currentTimeMillis(), -1, false + ); + } else { + op = new Engine.Delete("test", docId, id, + forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, + version, + forReplica ? null : versionType, + forReplica ? REPLICA : PRIMARY, + System.currentTimeMillis()); + } + ops.add(op); + } + return ops; + } + + public static void assertOpsOnReplica( + final List ops, + final InternalEngine replicaEngine, + boolean shuffleOps, + final Logger logger) throws IOException { + final Engine.Operation lastOp = ops.get(ops.size() - 1); + final String lastFieldValue; + if (lastOp instanceof Engine.Index) { + Engine.Index index = (Engine.Index) lastOp; + lastFieldValue = index.docs().get(0).get("value"); + } else { + // delete + lastFieldValue = null; + } + if (shuffleOps) { + int firstOpWithSeqNo = 0; + while (firstOpWithSeqNo < ops.size() && ops.get(firstOpWithSeqNo).seqNo() < 0) { + firstOpWithSeqNo++; + } + // shuffle ops but make sure legacy ops are first + shuffle(ops.subList(0, firstOpWithSeqNo), random()); + shuffle(ops.subList(firstOpWithSeqNo, ops.size()), random()); + } + boolean firstOp = true; + for (Engine.Operation op : ops) { + logger.info("performing [{}], v [{}], seq# [{}], term [{}]", + op.operationType().name().charAt(0), op.version(), op.seqNo(), op.primaryTerm()); + if (op instanceof Engine.Index) { + Engine.IndexResult result = replicaEngine.index((Engine.Index) op); + // replicas don't really care to about creation status of documents + // this allows to ignore the case where a document was found in the live version maps in + // a delete state and return false for the created flag in favor of code simplicity + // as deleted or not. This check is just signal regression so a decision can be made if it's + // intentional + assertThat(result.isCreated(), equalTo(firstOp)); + assertThat(result.getVersion(), equalTo(op.version())); + assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); + + } else { + Engine.DeleteResult result = replicaEngine.delete((Engine.Delete) op); + // Replicas don't really care to about found status of documents + // this allows to ignore the case where a document was found in the live version maps in + // a delete state and return true for the found flag in favor of code simplicity + // his check is just signal regression so a decision can be made if it's + // intentional + assertThat(result.isFound(), equalTo(firstOp == false)); + assertThat(result.getVersion(), equalTo(op.version())); + assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); + } + if (randomBoolean()) { + replicaEngine.refresh("test"); + } + if (randomBoolean()) { + replicaEngine.flush(); + replicaEngine.refresh("test"); + } + firstOp = false; + } + + assertVisibleCount(replicaEngine, lastFieldValue == null ? 0 : 1); + if (lastFieldValue != null) { + try (Engine.Searcher searcher = replicaEngine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); + assertThat(collector.getTotalHits(), equalTo(1)); + } + } + } + + protected void concurrentlyApplyOps(List ops, InternalEngine engine) throws InterruptedException { + Thread[] thread = new Thread[randomIntBetween(3, 5)]; + CountDownLatch startGun = new CountDownLatch(thread.length); + AtomicInteger offset = new AtomicInteger(-1); + for (int i = 0; i < thread.length; i++) { + thread[i] = new Thread(() -> { + startGun.countDown(); + try { + startGun.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + int docOffset; + while ((docOffset = offset.incrementAndGet()) < ops.size()) { + try { + final Engine.Operation op = ops.get(docOffset); + if (op instanceof Engine.Index) { + engine.index((Engine.Index) op); + } else if (op instanceof Engine.Delete){ + engine.delete((Engine.Delete) op); + } else { + engine.noOp((Engine.NoOp) op); + } + if ((docOffset + 1) % 4 == 0) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(); + } + } catch (IOException e) { + throw new AssertionError(e); + } + } + }); + thread[i].start(); + } + for (int i = 0; i < thread.length; i++) { + thread[i].join(); + } + } + + /** + * Gets all docId from the given engine. + */ + public static Set getDocIds(Engine engine, boolean refresh) throws IOException { + if (refresh) { + engine.refresh("test_get_doc_ids"); + } + try (Engine.Searcher searcher = engine.acquireSearcher("test_get_doc_ids")) { + Set ids = new HashSet<>(); + for (LeafReaderContext leafContext : searcher.reader().leaves()) { + LeafReader reader = leafContext.reader(); + Bits liveDocs = reader.getLiveDocs(); + for (int i = 0; i < reader.maxDoc(); i++) { + if (liveDocs == null || liveDocs.get(i)) { + Document uuid = reader.document(i, Collections.singleton(IdFieldMapper.NAME)); + BytesRef binaryID = uuid.getBinaryValue(IdFieldMapper.NAME); + ids.add(Uid.decodeId(Arrays.copyOfRange(binaryID.bytes, binaryID.offset, binaryID.offset + binaryID.length))); + } + } + } + return ids; + } + } + + /** + * Reads all engine operations that have been processed by the engine from Lucene index. + * The returned operations are sorted and de-duplicated, thus each sequence number will be have at most one operation. + */ + public static List readAllOperationsInLucene(Engine engine, MapperService mapper) throws IOException { + final List operations = new ArrayList<>(); + long maxSeqNo = Math.max(0, ((InternalEngine)engine).getLocalCheckpointTracker().getMaxSeqNo()); + try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", mapper, 0, maxSeqNo, false)) { + Translog.Operation op; + while ((op = snapshot.next()) != null){ + operations.add(op); + } + } + return operations; + } + + /** + * Asserts the provided engine has a consistent document history between translog and Lucene index. + */ + public static void assertConsistentHistoryBetweenTranslogAndLuceneIndex(Engine engine, MapperService mapper) throws IOException { + if (mapper.documentMapper() == null || engine.config().getIndexSettings().isSoftDeleteEnabled() == false) { + return; + } + final long maxSeqNo = ((InternalEngine) engine).getLocalCheckpointTracker().getMaxSeqNo(); + if (maxSeqNo < 0) { + return; // nothing to check + } + final Map translogOps = new HashMap<>(); + try (Translog.Snapshot snapshot = EngineTestCase.getTranslog(engine).newSnapshot()) { + Translog.Operation op; + while ((op = snapshot.next()) != null) { + translogOps.put(op.seqNo(), op); + } + } + final Map luceneOps = readAllOperationsInLucene(engine, mapper).stream() + .collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); + final long globalCheckpoint = EngineTestCase.getTranslog(engine).getLastSyncedGlobalCheckpoint(); + final long retainedOps = engine.config().getIndexSettings().getSoftDeleteRetentionOperations(); + final long seqNoForRecovery; + try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { + seqNoForRecovery = Long.parseLong(safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1; + } + final long minSeqNoToRetain = Math.min(seqNoForRecovery, globalCheckpoint + 1 - retainedOps); + for (Translog.Operation translogOp : translogOps.values()) { + final Translog.Operation luceneOp = luceneOps.get(translogOp.seqNo()); + if (luceneOp == null) { + if (minSeqNoToRetain <= translogOp.seqNo() && translogOp.seqNo() <= maxSeqNo) { + fail("Operation not found seq# [" + translogOp.seqNo() + "], global checkpoint [" + globalCheckpoint + "], " + + "retention policy [" + retainedOps + "], maxSeqNo [" + maxSeqNo + "], translog op [" + translogOp + "]"); + } else { + continue; + } + } + assertThat(luceneOp, notNullValue()); + assertThat(luceneOp.toString(), luceneOp.primaryTerm(), equalTo(translogOp.primaryTerm())); + assertThat(luceneOp.opType(), equalTo(translogOp.opType())); + if (luceneOp.opType() == Translog.Operation.Type.INDEX) { + assertThat(luceneOp.getSource().source, equalTo(translogOp.getSource().source)); + } + } + } + + protected MapperService createMapperService(String type) throws IOException { + IndexMetaData indexMetaData = IndexMetaData.builder("test") + .settings(Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)) + .putMapping(type, "{\"properties\": {}}") + .build(); + MapperService mapperService = MapperTestUtils.newMapperService(new NamedXContentRegistry(ClusterModule.getNamedXWriteables()), + createTempDir(), Settings.EMPTY, "test"); + mapperService.merge(indexMetaData, MapperService.MergeReason.MAPPING_UPDATE); + return mapperService; + } /** * Exposes a translog associated with the given engine for testing purpose. diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 3f1f5daf514..f2afdff9c3a 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -60,6 +60,7 @@ import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; @@ -99,10 +100,14 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase protected final Index index = new Index("test", "uuid"); private final ShardId shardId = new ShardId(index, 0); - private final Map indexMapping = Collections.singletonMap("type", "{ \"type\": {} }"); + protected final Map indexMapping = Collections.singletonMap("type", "{ \"type\": {} }"); protected ReplicationGroup createGroup(int replicas) throws IOException { - IndexMetaData metaData = buildIndexMetaData(replicas); + return createGroup(replicas, Settings.EMPTY); + } + + protected ReplicationGroup createGroup(int replicas, Settings settings) throws IOException { + IndexMetaData metaData = buildIndexMetaData(replicas, settings, indexMapping); return new ReplicationGroup(metaData); } @@ -111,9 +116,17 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } protected IndexMetaData buildIndexMetaData(int replicas, Map mappings) throws IOException { + return buildIndexMetaData(replicas, Settings.EMPTY, mappings); + } + + protected IndexMetaData buildIndexMetaData(int replicas, Settings indexSettings, Map mappings) throws IOException { Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, replicas) .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), + randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) + .put(indexSettings) .build(); IndexMetaData.Builder metaData = IndexMetaData.builder(index.getName()) .settings(settings) @@ -146,7 +159,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } }); - ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { + protected ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { final ShardRouting primaryRouting = this.createShardRouting("s0", true); primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting), () -> {}); replicas = new CopyOnWriteArrayList<>(); @@ -448,7 +461,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } } - abstract class ReplicationAction, + protected abstract class ReplicationAction, ReplicaRequest extends ReplicationRequest, Response extends ReplicationResponse> { private final Request request; @@ -456,7 +469,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase private final ReplicationGroup replicationGroup; private final String opType; - ReplicationAction(Request request, ActionListener listener, ReplicationGroup group, String opType) { + protected ReplicationAction(Request request, ActionListener listener, ReplicationGroup group, String opType) { this.request = request; this.listener = listener; this.replicationGroup = group; @@ -582,11 +595,11 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } } - class PrimaryResult implements ReplicationOperation.PrimaryResult { + protected class PrimaryResult implements ReplicationOperation.PrimaryResult { final ReplicaRequest replicaRequest; final Response finalResponse; - PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) { + public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) { this.replicaRequest = replicaRequest; this.finalResponse = finalResponse; } diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index d2a84589669..2f4a3dfd6c1 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -18,13 +18,8 @@ */ package org.elasticsearch.index.shard; -import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexNotFoundException; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.store.Directory; -import org.apache.lucene.util.Bits; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.index.IndexRequest; @@ -57,10 +52,8 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngineFactory; -import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.SourceToParse; -import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.similarity.SimilarityService; @@ -180,37 +173,63 @@ public abstract class IndexShardTestCase extends ESTestCase { } /** - * creates a new initializing shard. The shard will have its own unique data path. + * Creates a new initializing shard. The shard will have its own unique data path. * - * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica - * (ready to recover from another shard) + * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica (ready to recover from + * another shard) */ protected IndexShard newShard(boolean primary) throws IOException { - ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), primary, - ShardRoutingState.INITIALIZING, - primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); - return newShard(shardRouting); + return newShard(primary, Settings.EMPTY, new InternalEngineFactory()); } /** - * creates a new initializing shard. The shard will have its own unique data path. + * Creates a new initializing shard. The shard will have its own unique data path. * - * @param shardRouting the {@link ShardRouting} to use for this shard - * @param listeners an optional set of listeners to add to the shard + * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica (ready to recover from + * another shard) + * @param settings the settings to use for this shard + * @param engineFactory the engine factory to use for this shard + */ + protected IndexShard newShard(boolean primary, Settings settings, EngineFactory engineFactory) throws IOException { + final RecoverySource recoverySource = + primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE; + final ShardRouting shardRouting = + TestShardRouting.newShardRouting( + new ShardId("index", "_na_", 0), randomAlphaOfLength(10), primary, ShardRoutingState.INITIALIZING, recoverySource); + return newShard(shardRouting, settings, engineFactory); + } + + protected IndexShard newShard(ShardRouting shardRouting, final IndexingOperationListener... listeners) throws IOException { + return newShard(shardRouting, Settings.EMPTY, new InternalEngineFactory(), listeners); + } + + /** + * Creates a new initializing shard. The shard will have its own unique data path. + * + * @param shardRouting the {@link ShardRouting} to use for this shard + * @param settings the settings to use for this shard + * @param engineFactory the engine factory to use for this shard + * @param listeners an optional set of listeners to add to the shard */ protected IndexShard newShard( final ShardRouting shardRouting, + final Settings settings, + final EngineFactory engineFactory, final IndexingOperationListener... listeners) throws IOException { assert shardRouting.initializing() : shardRouting; - Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .build(); + Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), + randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) + .put(settings) + .build(); IndexMetaData.Builder metaData = IndexMetaData.builder(shardRouting.getIndexName()) - .settings(settings) + .settings(indexSettings) .primaryTerm(0, primaryTerm) .putMapping("_doc", "{ \"properties\": {} }"); - return newShard(shardRouting, metaData.build(), listeners); + return newShard(shardRouting, metaData.build(), engineFactory, listeners); } /** @@ -225,7 +244,7 @@ public abstract class IndexShardTestCase extends ESTestCase { ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, randomAlphaOfLength(5), primary, ShardRoutingState.INITIALIZING, primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); - return newShard(shardRouting, listeners); + return newShard(shardRouting, Settings.EMPTY, new InternalEngineFactory(), listeners); } /** @@ -265,9 +284,10 @@ public abstract class IndexShardTestCase extends ESTestCase { * @param indexMetaData indexMetaData for the shard, including any mapping * @param listeners an optional set of listeners to add to the shard */ - protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, IndexingOperationListener... listeners) + protected IndexShard newShard( + ShardRouting routing, IndexMetaData indexMetaData, EngineFactory engineFactory, IndexingOperationListener... listeners) throws IOException { - return newShard(routing, indexMetaData, null, new InternalEngineFactory(), () -> {}, listeners); + return newShard(routing, indexMetaData, null, engineFactory, () -> {}, listeners); } /** @@ -372,19 +392,39 @@ public abstract class IndexShardTestCase extends ESTestCase { } /** - * creates a new empyu shard and starts it. The shard will be either a replica or a primary. + * Creates a new empty shard and starts it. The shard will randomly be a replica or a primary. */ protected IndexShard newStartedShard() throws IOException { return newStartedShard(randomBoolean()); } /** - * creates a new empty shard and starts it. + * Creates a new empty shard and starts it + * @param settings the settings to use for this shard + */ + protected IndexShard newStartedShard(Settings settings) throws IOException { + return newStartedShard(randomBoolean(), settings, new InternalEngineFactory()); + } + + /** + * Creates a new empty shard and starts it. * * @param primary controls whether the shard will be a primary or a replica. */ - protected IndexShard newStartedShard(boolean primary) throws IOException { - IndexShard shard = newShard(primary); + protected IndexShard newStartedShard(final boolean primary) throws IOException { + return newStartedShard(primary, Settings.EMPTY, new InternalEngineFactory()); + } + + /** + * Creates a new empty shard with the specified settings and engine factory and starts it. + * + * @param primary controls whether the shard will be a primary or a replica. + * @param settings the settings to use for this shard + * @param engineFactory the engine factory to use for this shard + */ + protected IndexShard newStartedShard( + final boolean primary, final Settings settings, final EngineFactory engineFactory) throws IOException { + IndexShard shard = newShard(primary, settings, engineFactory); if (primary) { recoverShardFromStore(shard); } else { @@ -401,6 +441,7 @@ public abstract class IndexShardTestCase extends ESTestCase { for (IndexShard shard : shards) { if (shard != null) { try { + assertConsistentHistoryBetweenTranslogAndLucene(shard); shard.close("test", false); } finally { IOUtils.close(shard.store()); @@ -582,22 +623,7 @@ public abstract class IndexShardTestCase extends ESTestCase { } protected Set getShardDocUIDs(final IndexShard shard) throws IOException { - shard.refresh("get_uids"); - try (Engine.Searcher searcher = shard.acquireSearcher("test")) { - Set ids = new HashSet<>(); - for (LeafReaderContext leafContext : searcher.reader().leaves()) { - LeafReader reader = leafContext.reader(); - Bits liveDocs = reader.getLiveDocs(); - for (int i = 0; i < reader.maxDoc(); i++) { - if (liveDocs == null || liveDocs.get(i)) { - Document uuid = reader.document(i, Collections.singleton(IdFieldMapper.NAME)); - BytesRef binaryID = uuid.getBinaryValue(IdFieldMapper.NAME); - ids.add(Uid.decodeId(Arrays.copyOfRange(binaryID.bytes, binaryID.offset, binaryID.offset + binaryID.length))); - } - } - } - return ids; - } + return EngineTestCase.getDocIds(shard.getEngine(), true); } protected void assertDocCount(IndexShard shard, int docDount) throws IOException { @@ -610,6 +636,12 @@ public abstract class IndexShardTestCase extends ESTestCase { assertThat(shardDocUIDs, hasSize(ids.length)); } + public static void assertConsistentHistoryBetweenTranslogAndLucene(IndexShard shard) throws IOException { + final Engine engine = shard.getEngineOrNull(); + if (engine != null) { + EngineTestCase.assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, shard.mapperService()); + } + } protected Engine.IndexResult indexDoc(IndexShard shard, String type, String id) throws IOException { return indexDoc(shard, type, id, "{}"); @@ -653,11 +685,14 @@ public abstract class IndexShardTestCase extends ESTestCase { } protected Engine.DeleteResult deleteDoc(IndexShard shard, String type, String id) throws IOException { + final Engine.DeleteResult result; if (shard.routingEntry().primary()) { - return shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, type, id, VersionType.INTERNAL); + result = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, type, id, VersionType.INTERNAL); + shard.updateLocalCheckpointForShard(shard.routingEntry().allocationId().getId(), shard.getEngine().getLocalCheckpoint()); } else { - return shard.applyDeleteOperationOnReplica(shard.seqNoStats().getMaxSeqNo() + 1, 0L, type, id); + result = shard.applyDeleteOperationOnReplica(shard.seqNoStats().getMaxSeqNo() + 1, 0L, type, id); } + return result; } protected void flushShard(IndexShard shard) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 322e2a128c9..be9e40ab420 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -723,6 +723,10 @@ public abstract class ESIntegTestCase extends ESTestCase { } // always default delayed allocation to 0 to make sure we have tests are not delayed builder.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0); + builder.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()); + if (randomBoolean()) { + builder.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), between(0, 1000)); + } return builder.build(); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java index 9633f56dea9..19290f8cf11 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java @@ -41,6 +41,7 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.node.MockNode; @@ -87,6 +88,14 @@ public abstract class ESSingleNodeTestCase extends ESTestCase { .setOrder(0) .setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)).get(); + client().admin().indices() + .preparePutTemplate("random-soft-deletes-template") + .setPatterns(Collections.singletonList("*")) + .setOrder(0) + .setSettings(Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), + randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) + ).get(); } private static void stopNode() throws IOException { diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 306f79e5e16..4c813372fae 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -1163,6 +1163,26 @@ public final class InternalTestCluster extends TestCluster { }); } + /** + * Asserts that the document history in Lucene index is consistent with Translog's on every index shard of the cluster. + * This assertion might be expensive, thus we prefer not to execute on every test but only interesting tests. + */ + public void assertConsistentHistoryBetweenTranslogAndLuceneIndex() throws IOException { + final Collection nodesAndClients = nodes.values(); + for (NodeAndClient nodeAndClient : nodesAndClients) { + IndicesService indexServices = getInstance(IndicesService.class, nodeAndClient.name); + for (IndexService indexService : indexServices) { + for (IndexShard indexShard : indexService) { + try { + IndexShardTestCase.assertConsistentHistoryBetweenTranslogAndLucene(indexShard); + } catch (AlreadyClosedException ignored) { + // shard is closed + } + } + } + } + } + private void randomlyResetClients() throws IOException { // only reset the clients on nightly tests, it causes heavy load... if (RandomizedTest.isNightly() && rarely(random)) { From 44ed5f6306038c7f80cbd6ffe403104248145d1e Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Fri, 31 Aug 2018 09:31:55 +0300 Subject: [PATCH 08/16] Enable forbiddenapis server java9 (#33245) --- .../groovy/org/elasticsearch/gradle/BuildPlugin.groovy | 3 ++- .../gradle/precommit/PrecommitTasks.groovy | 2 +- libs/core/build.gradle | 9 +++++---- server/build.gradle | 10 +++++----- 4 files changed, 13 insertions(+), 11 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy index 4c4a8cbe881..6a9d4076eef 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy @@ -38,7 +38,6 @@ import org.gradle.api.artifacts.ModuleDependency import org.gradle.api.artifacts.ModuleVersionIdentifier import org.gradle.api.artifacts.ProjectDependency import org.gradle.api.artifacts.ResolvedArtifact -import org.gradle.api.artifacts.SelfResolvingDependency import org.gradle.api.artifacts.dsl.RepositoryHandler import org.gradle.api.execution.TaskExecutionGraph import org.gradle.api.plugins.JavaPlugin @@ -212,6 +211,7 @@ class BuildPlugin implements Plugin { project.rootProject.ext.minimumRuntimeVersion = minimumRuntimeVersion project.rootProject.ext.inFipsJvm = inFipsJvm project.rootProject.ext.gradleJavaVersion = JavaVersion.toVersion(gradleJavaVersion) + project.rootProject.ext.java9Home = findJavaHome("9") } project.targetCompatibility = project.rootProject.ext.minimumRuntimeVersion @@ -225,6 +225,7 @@ class BuildPlugin implements Plugin { project.ext.javaVersions = project.rootProject.ext.javaVersions project.ext.inFipsJvm = project.rootProject.ext.inFipsJvm project.ext.gradleJavaVersion = project.rootProject.ext.gradleJavaVersion + project.ext.java9Home = project.rootProject.ext.java9Home } private static String getPaddedMajorVersion(JavaVersion compilerJavaVersionEnum) { diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy index be7561853bb..06557d4ccfd 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy @@ -100,7 +100,7 @@ class PrecommitTasks { private static Task configureForbiddenApisCli(Project project) { Task forbiddenApisCli = project.tasks.create('forbiddenApis') - project.sourceSets.forEach { sourceSet -> + project.sourceSets.all { sourceSet -> forbiddenApisCli.dependsOn( project.tasks.create(sourceSet.getTaskName('forbiddenApis', null), ForbiddenApisCliTask) { ExportElasticsearchBuildResourcesTask buildResources = project.tasks.getByName('buildResources') diff --git a/libs/core/build.gradle b/libs/core/build.gradle index cc5c1e20fc1..9c90837bd80 100644 --- a/libs/core/build.gradle +++ b/libs/core/build.gradle @@ -46,12 +46,13 @@ if (!isEclipse && !isIdea) { targetCompatibility = 9 } - /* Enable this when forbiddenapis was updated to 2.6. - * See: https://github.com/elastic/elasticsearch/issues/29292 forbiddenApisJava9 { - targetCompatibility = 9 + if (project.runtimeJavaVersion < JavaVersion.VERSION_1_9) { + targetCompatibility = JavaVersion.VERSION_1_9 + javaHome = project.java9Home + } + replaceSignatureFiles 'jdk-signatures' } - */ jar { metaInf { diff --git a/server/build.gradle b/server/build.gradle index edc3f427dfd..c01fb92b050 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -58,13 +58,13 @@ if (!isEclipse && !isIdea) { sourceCompatibility = 9 targetCompatibility = 9 } - - /* Enable this when forbiddenapis was updated to 2.6. - * See: https://github.com/elastic/elasticsearch/issues/29292 + forbiddenApisJava9 { - targetCompatibility = 9 + if (project.runtimeJavaVersion < JavaVersion.VERSION_1_9) { + targetCompatibility = JavaVersion.VERSION_1_9 + javaHome = project.java9Home + } } - */ jar { metaInf { From c6cfa08a615f5b433cf64bef63220f4a22727115 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 31 Aug 2018 08:40:27 +0200 Subject: [PATCH 09/16] MINOR: Remove Dead Code from PathTrie (#33280) * The array size checks are redundant since the array sizes are checked earlier in those methods too * The removed methods are just not used anywhere --- .../elasticsearch/common/path/PathTrie.java | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/path/PathTrie.java b/server/src/main/java/org/elasticsearch/common/path/PathTrie.java index 5243809c64a..08787cea9df 100644 --- a/server/src/main/java/org/elasticsearch/common/path/PathTrie.java +++ b/server/src/main/java/org/elasticsearch/common/path/PathTrie.java @@ -104,24 +104,12 @@ public class PathTrie { namedWildcard = key.substring(key.indexOf('{') + 1, key.indexOf('}')); } - public boolean isWildcard() { - return isWildcard; - } - - public synchronized void addChild(TrieNode child) { - addInnerChild(child.key, child); - } - private void addInnerChild(String key, TrieNode child) { Map newChildren = new HashMap<>(children); newChildren.put(key, child); children = unmodifiableMap(newChildren); } - public TrieNode getChild(String key) { - return children.get(key); - } - public synchronized void insert(String[] path, int index, T value) { if (index >= path.length) return; @@ -302,7 +290,7 @@ public class PathTrie { } int index = 0; // Supports initial delimiter. - if (strings.length > 0 && strings[0].isEmpty()) { + if (strings[0].isEmpty()) { index = 1; } root.insert(strings, index, value); @@ -327,7 +315,7 @@ public class PathTrie { } int index = 0; // Supports initial delimiter. - if (strings.length > 0 && strings[0].isEmpty()) { + if (strings[0].isEmpty()) { index = 1; } root.insertOrUpdate(strings, index, value, updater); @@ -352,7 +340,7 @@ public class PathTrie { int index = 0; // Supports initial delimiter. - if (strings.length > 0 && strings[0].isEmpty()) { + if (strings[0].isEmpty()) { index = 1; } From 73eb4cbbbe1c38508f6fc303ca300c508952b507 Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Fri, 31 Aug 2018 10:45:25 +0300 Subject: [PATCH 10/16] SQL: Support multi-index format as table identifier (#33278) Extend tableIdentifier to support multi-index format; not just * but also enumeration and exclusion Fix #33162 --- .../sql/analysis/index/IndexResolver.java | 3 +- .../xpack/sql/execution/search/Querier.java | 3 +- .../xpack/sql/parser/IdentifierBuilder.java | 14 ------- .../sql/parser/IdentifierBuilderTests.java | 38 ------------------- .../sql/src/main/resources/command.csv-spec | 24 ++++++++++++ 5 files changed, 28 insertions(+), 54 deletions(-) delete mode 100644 x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/IdentifierBuilderTests.java diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java index 10586c991b1..b11542d40ed 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java @@ -19,6 +19,7 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.AliasMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.xpack.sql.type.EsField; @@ -300,7 +301,7 @@ public class IndexResolver { private static GetIndexRequest createGetIndexRequest(String index) { return new GetIndexRequest() .local(true) - .indices(index) + .indices(Strings.commaDelimitedListToStringArray(index)) .features(Feature.MAPPINGS) //lenient because we throw our own errors looking at the response e.g. if something was not resolved //also because this way security doesn't throw authorization exceptions but rather honours ignore_unavailable diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/Querier.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/Querier.java index 055e34758cc..d0bff77a648 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/Querier.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/Querier.java @@ -12,6 +12,7 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.client.Client; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CollectionUtils; @@ -92,7 +93,7 @@ public class Querier { log.trace("About to execute query {} on {}", StringUtils.toString(sourceBuilder), index); } - SearchRequest search = prepareRequest(client, sourceBuilder, timeout, index); + SearchRequest search = prepareRequest(client, sourceBuilder, timeout, Strings.commaDelimitedListToStringArray(index)); ActionListener l; if (query.isAggsOnly()) { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/IdentifierBuilder.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/IdentifierBuilder.java index 8c79ae1ef05..f09f543c6ff 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/IdentifierBuilder.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/IdentifierBuilder.java @@ -21,23 +21,9 @@ abstract class IdentifierBuilder extends AbstractBuilder { ParseTree tree = ctx.name != null ? ctx.name : ctx.TABLE_IDENTIFIER(); String index = tree.getText(); - validateIndex(index, source); return new TableIdentifier(source, visitIdentifier(ctx.catalog), index); } - // see https://github.com/elastic/elasticsearch/issues/6736 - static void validateIndex(String index, Location source) { - for (int i = 0; i < index.length(); i++) { - char c = index.charAt(i); - if (Character.isUpperCase(c)) { - throw new ParsingException(source, "Invalid index name (needs to be lowercase) {}", index); - } - if (c == '\\' || c == '/' || c == '<' || c == '>' || c == '|' || c == ',' || c == ' ') { - throw new ParsingException(source, "Invalid index name (illegal character {}) {}", c, index); - } - } - } - @Override public String visitIdentifier(IdentifierContext ctx) { return ctx == null ? null : ctx.getText(); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/IdentifierBuilderTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/IdentifierBuilderTests.java deleted file mode 100644 index ec8b8abc51f..00000000000 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/IdentifierBuilderTests.java +++ /dev/null @@ -1,38 +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; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.sql.parser; - -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.sql.tree.Location; - -import static org.hamcrest.Matchers.is; - -public class IdentifierBuilderTests extends ESTestCase { - - private static Location L = new Location(1, 10); - - public void testTypicalIndex() throws Exception { - IdentifierBuilder.validateIndex("some-index", L); - } - - public void testInternalIndex() throws Exception { - IdentifierBuilder.validateIndex(".some-internal-index-2020-02-02", L); - } - - public void testIndexPattern() throws Exception { - IdentifierBuilder.validateIndex(".some-*", L); - } - - public void testInvalidIndex() throws Exception { - ParsingException pe = expectThrows(ParsingException.class, () -> IdentifierBuilder.validateIndex("some,index", L)); - assertThat(pe.getMessage(), is("line 1:12: Invalid index name (illegal character ,) some,index")); - } - - public void testUpperCasedIndex() throws Exception { - ParsingException pe = expectThrows(ParsingException.class, () -> IdentifierBuilder.validateIndex("thisIsAnIndex", L)); - assertThat(pe.getMessage(), is("line 1:12: Invalid index name (needs to be lowercase) thisIsAnIndex")); - } -} diff --git a/x-pack/qa/sql/src/main/resources/command.csv-spec b/x-pack/qa/sql/src/main/resources/command.csv-spec index 89e86e887e1..a8f23e27ffa 100644 --- a/x-pack/qa/sql/src/main/resources/command.csv-spec +++ b/x-pack/qa/sql/src/main/resources/command.csv-spec @@ -162,3 +162,27 @@ last_name | VARCHAR last_name.keyword | VARCHAR salary | INTEGER ; + + +describeIncludeExclude +DESCRIBE "test_emp*,-test_alias*"; + +column:s | type:s +birth_date | TIMESTAMP +dep | STRUCT +dep.dep_id | VARCHAR +dep.dep_name | VARCHAR +dep.dep_name.keyword | VARCHAR +dep.from_date | TIMESTAMP +dep.to_date | TIMESTAMP +emp_no | INTEGER +first_name | VARCHAR +first_name.keyword | VARCHAR +gender | VARCHAR +hire_date | TIMESTAMP +languages | TINYINT +last_name | VARCHAR +last_name.keyword | VARCHAR +salary | INTEGER +; + From 7345878d33a00ce95eae929f79b2da0b326d182d Mon Sep 17 00:00:00 2001 From: David Roberts Date: Fri, 31 Aug 2018 08:48:45 +0100 Subject: [PATCH 11/16] [ML] Refactor delimited file structure detection (#33233) 1. Use the term "delimited" rather than "separated values" 2. Use a single factory class with arguments to specify the delimiter and identification constraints This change makes it easier to add support for other delimiter characters. --- .../CsvLogStructureFinderFactory.java | 35 ----- ....java => DelimitedLogStructureFinder.java} | 16 +-- .../DelimitedLogStructureFinderFactory.java | 57 ++++++++ .../ml/logstructurefinder/LogStructure.java | 124 ++++-------------- .../LogStructureFinderManager.java | 8 +- .../logstructurefinder/LogStructureUtils.java | 10 +- ...aratedValuesLogStructureFinderFactory.java | 38 ------ ...aratedValuesLogStructureFinderFactory.java | 37 ------ .../TsvLogStructureFinderFactory.java | 4 +- .../CsvLogStructureFinderFactoryTests.java | 38 ------ ...limitedLogStructureFinderFactoryTests.java | 93 +++++++++++++ ... => DelimitedLogStructureFinderTests.java} | 122 ++++++++--------- .../JsonLogStructureFinderFactoryTests.java | 8 +- .../JsonLogStructureFinderTests.java | 2 +- .../LogStructureFinderManagerTests.java | 2 +- .../LogStructureTestCase.java | 4 +- .../logstructurefinder/LogStructureTests.java | 8 +- ...dValuesLogStructureFinderFactoryTests.java | 23 ---- ...dValuesLogStructureFinderFactoryTests.java | 28 ---- .../TextLogStructureFinderFactoryTests.java | 4 +- .../TextLogStructureFinderTests.java | 2 +- .../TsvLogStructureFinderFactoryTests.java | 33 ----- .../XmlLogStructureFinderFactoryTests.java | 8 +- .../XmlLogStructureFinderTests.java | 2 +- 24 files changed, 277 insertions(+), 429 deletions(-) delete mode 100644 x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/CsvLogStructureFinderFactory.java rename x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/{SeparatedValuesLogStructureFinder.java => DelimitedLogStructureFinder.java} (97%) create mode 100644 x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinderFactory.java delete mode 100644 x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/PipeSeparatedValuesLogStructureFinderFactory.java delete mode 100644 x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/SemiColonSeparatedValuesLogStructureFinderFactory.java delete mode 100644 x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/CsvLogStructureFinderFactoryTests.java create mode 100644 x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinderFactoryTests.java rename x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/{SeparatedValuesLogStructureFinderTests.java => DelimitedLogStructureFinderTests.java} (65%) delete mode 100644 x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/PipeSeparatedValuesLogStructureFinderFactoryTests.java delete mode 100644 x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/SemiColonSeparatedValuesLogStructureFinderFactoryTests.java delete mode 100644 x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TsvLogStructureFinderFactoryTests.java diff --git a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/CsvLogStructureFinderFactory.java b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/CsvLogStructureFinderFactory.java deleted file mode 100644 index cb9e6537252..00000000000 --- a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/CsvLogStructureFinderFactory.java +++ /dev/null @@ -1,35 +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; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.ml.logstructurefinder; - -import org.supercsv.prefs.CsvPreference; - -import java.io.IOException; -import java.util.List; - -public class CsvLogStructureFinderFactory implements LogStructureFinderFactory { - - /** - * Rules are: - * - The file must be valid CSV - * - It must contain at least two complete records - * - There must be at least two fields per record (otherwise files with no commas could be treated as CSV!) - * - Every CSV record except the last must have the same number of fields - * The reason the last record is allowed to have fewer fields than the others is that - * it could have been truncated when the file was sampled. - */ - @Override - public boolean canCreateFromSample(List explanation, String sample) { - return SeparatedValuesLogStructureFinder.canCreateFromSample(explanation, sample, 2, CsvPreference.EXCEL_PREFERENCE, "CSV"); - } - - @Override - public LogStructureFinder createFromSample(List explanation, String sample, String charsetName, Boolean hasByteOrderMarker) - throws IOException { - return SeparatedValuesLogStructureFinder.makeSeparatedValuesLogStructureFinder(explanation, sample, charsetName, hasByteOrderMarker, - CsvPreference.EXCEL_PREFERENCE, false); - } -} diff --git a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/SeparatedValuesLogStructureFinder.java b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinder.java similarity index 97% rename from x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/SeparatedValuesLogStructureFinder.java rename to x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinder.java index fd9d34096b2..2f7bb41d0ba 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/SeparatedValuesLogStructureFinder.java +++ b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinder.java @@ -29,17 +29,16 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.IntStream; -public class SeparatedValuesLogStructureFinder implements LogStructureFinder { +public class DelimitedLogStructureFinder implements LogStructureFinder { private static final int MAX_LEVENSHTEIN_COMPARISONS = 100; private final List sampleMessages; private final LogStructure structure; - static SeparatedValuesLogStructureFinder makeSeparatedValuesLogStructureFinder(List explanation, String sample, - String charsetName, Boolean hasByteOrderMarker, - CsvPreference csvPreference, boolean trimFields) - throws IOException { + static DelimitedLogStructureFinder makeDelimitedLogStructureFinder(List explanation, String sample, String charsetName, + Boolean hasByteOrderMarker, CsvPreference csvPreference, + boolean trimFields) throws IOException { Tuple>, List> parsed = readRows(sample, csvPreference); List> rows = parsed.v1(); @@ -73,13 +72,14 @@ public class SeparatedValuesLogStructureFinder implements LogStructureFinder { String preamble = Pattern.compile("\n").splitAsStream(sample).limit(lineNumbers.get(1)).collect(Collectors.joining("\n", "", "\n")); char delimiter = (char) csvPreference.getDelimiterChar(); - LogStructure.Builder structureBuilder = new LogStructure.Builder(LogStructure.Format.fromSeparator(delimiter)) + LogStructure.Builder structureBuilder = new LogStructure.Builder(LogStructure.Format.DELIMITED) .setCharset(charsetName) .setHasByteOrderMarker(hasByteOrderMarker) .setSampleStart(preamble) .setNumLinesAnalyzed(lineNumbers.get(lineNumbers.size() - 1)) .setNumMessagesAnalyzed(sampleRecords.size()) .setHasHeaderRow(isHeaderInFile) + .setDelimiter(delimiter) .setInputFields(Arrays.stream(headerWithNamedBlanks).collect(Collectors.toList())); if (trimFields) { @@ -131,10 +131,10 @@ public class SeparatedValuesLogStructureFinder implements LogStructureFinder { .setExplanation(explanation) .build(); - return new SeparatedValuesLogStructureFinder(sampleMessages, structure); + return new DelimitedLogStructureFinder(sampleMessages, structure); } - private SeparatedValuesLogStructureFinder(List sampleMessages, LogStructure structure) { + private DelimitedLogStructureFinder(List sampleMessages, LogStructure structure) { this.sampleMessages = Collections.unmodifiableList(sampleMessages); this.structure = structure; } diff --git a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinderFactory.java b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinderFactory.java new file mode 100644 index 00000000000..3e4c3ea225c --- /dev/null +++ b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinderFactory.java @@ -0,0 +1,57 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.logstructurefinder; + +import org.supercsv.prefs.CsvPreference; + +import java.io.IOException; +import java.util.List; +import java.util.Locale; + +public class DelimitedLogStructureFinderFactory implements LogStructureFinderFactory { + + private final CsvPreference csvPreference; + private final int minFieldsPerRow; + private final boolean trimFields; + + DelimitedLogStructureFinderFactory(char delimiter, int minFieldsPerRow, boolean trimFields) { + csvPreference = new CsvPreference.Builder('"', delimiter, "\n").build(); + this.minFieldsPerRow = minFieldsPerRow; + this.trimFields = trimFields; + } + + /** + * Rules are: + * - It must contain at least two complete records + * - There must be a minimum number of fields per record (otherwise files with no commas could be treated as CSV!) + * - Every record except the last must have the same number of fields + * The reason the last record is allowed to have fewer fields than the others is that + * it could have been truncated when the file was sampled. + */ + @Override + public boolean canCreateFromSample(List explanation, String sample) { + String formatName; + switch ((char) csvPreference.getDelimiterChar()) { + case ',': + formatName = "CSV"; + break; + case '\t': + formatName = "TSV"; + break; + default: + formatName = Character.getName(csvPreference.getDelimiterChar()).toLowerCase(Locale.ROOT) + " delimited values"; + break; + } + return DelimitedLogStructureFinder.canCreateFromSample(explanation, sample, minFieldsPerRow, csvPreference, formatName); + } + + @Override + public LogStructureFinder createFromSample(List explanation, String sample, String charsetName, Boolean hasByteOrderMarker) + throws IOException { + return DelimitedLogStructureFinder.makeDelimitedLogStructureFinder(explanation, sample, charsetName, hasByteOrderMarker, + csvPreference, trimFields); + } +} diff --git a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructure.java b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructure.java index 64a00d20899..ea8fe37e62f 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructure.java +++ b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructure.java @@ -27,37 +27,14 @@ public class LogStructure implements ToXContentObject { public enum Format { - JSON, XML, CSV, TSV, SEMI_COLON_SEPARATED_VALUES, PIPE_SEPARATED_VALUES, SEMI_STRUCTURED_TEXT; - - public Character separator() { - switch (this) { - case JSON: - case XML: - return null; - case CSV: - return ','; - case TSV: - return '\t'; - case SEMI_COLON_SEPARATED_VALUES: - return ';'; - case PIPE_SEPARATED_VALUES: - return '|'; - case SEMI_STRUCTURED_TEXT: - return null; - default: - throw new IllegalStateException("enum value [" + this + "] missing from switch."); - } - } + JSON, XML, DELIMITED, SEMI_STRUCTURED_TEXT; public boolean supportsNesting() { switch (this) { case JSON: case XML: return true; - case CSV: - case TSV: - case SEMI_COLON_SEPARATED_VALUES: - case PIPE_SEPARATED_VALUES: + case DELIMITED: case SEMI_STRUCTURED_TEXT: return false; default: @@ -69,10 +46,7 @@ public class LogStructure implements ToXContentObject { switch (this) { case JSON: case XML: - case CSV: - case TSV: - case SEMI_COLON_SEPARATED_VALUES: - case PIPE_SEPARATED_VALUES: + case DELIMITED: return true; case SEMI_STRUCTURED_TEXT: return false; @@ -85,10 +59,7 @@ public class LogStructure implements ToXContentObject { switch (this) { case JSON: case XML: - case CSV: - case TSV: - case SEMI_COLON_SEPARATED_VALUES: - case PIPE_SEPARATED_VALUES: + case DELIMITED: return false; case SEMI_STRUCTURED_TEXT: return true; @@ -97,38 +68,6 @@ public class LogStructure implements ToXContentObject { } } - public boolean isSeparatedValues() { - switch (this) { - case JSON: - case XML: - return false; - case CSV: - case TSV: - case SEMI_COLON_SEPARATED_VALUES: - case PIPE_SEPARATED_VALUES: - return true; - case SEMI_STRUCTURED_TEXT: - return false; - default: - throw new IllegalStateException("enum value [" + this + "] missing from switch."); - } - } - - public static Format fromSeparator(char separator) { - switch (separator) { - case ',': - return CSV; - case '\t': - return TSV; - case ';': - return SEMI_COLON_SEPARATED_VALUES; - case '|': - return PIPE_SEPARATED_VALUES; - default: - throw new IllegalArgumentException("No known format has separator [" + separator + "]"); - } - } - public static Format fromString(String name) { return valueOf(name.trim().toUpperCase(Locale.ROOT)); } @@ -149,7 +88,7 @@ public class LogStructure implements ToXContentObject { static final ParseField EXCLUDE_LINES_PATTERN = new ParseField("exclude_lines_pattern"); static final ParseField INPUT_FIELDS = new ParseField("input_fields"); static final ParseField HAS_HEADER_ROW = new ParseField("has_header_row"); - static final ParseField SEPARATOR = new ParseField("separator"); + static final ParseField DELIMITER = new ParseField("delimiter"); static final ParseField SHOULD_TRIM_FIELDS = new ParseField("should_trim_fields"); static final ParseField GROK_PATTERN = new ParseField("grok_pattern"); static final ParseField TIMESTAMP_FIELD = new ParseField("timestamp_field"); @@ -171,7 +110,7 @@ public class LogStructure implements ToXContentObject { PARSER.declareString(Builder::setExcludeLinesPattern, EXCLUDE_LINES_PATTERN); PARSER.declareStringArray(Builder::setInputFields, INPUT_FIELDS); PARSER.declareBoolean(Builder::setHasHeaderRow, HAS_HEADER_ROW); - PARSER.declareString((p, c) -> p.setSeparator(c.charAt(0)), SEPARATOR); + PARSER.declareString((p, c) -> p.setDelimiter(c.charAt(0)), DELIMITER); PARSER.declareBoolean(Builder::setShouldTrimFields, SHOULD_TRIM_FIELDS); PARSER.declareString(Builder::setGrokPattern, GROK_PATTERN); PARSER.declareString(Builder::setTimestampField, TIMESTAMP_FIELD); @@ -191,7 +130,7 @@ public class LogStructure implements ToXContentObject { private final String excludeLinesPattern; private final List inputFields; private final Boolean hasHeaderRow; - private final Character separator; + private final Character delimiter; private final Boolean shouldTrimFields; private final String grokPattern; private final List timestampFormats; @@ -202,7 +141,7 @@ public class LogStructure implements ToXContentObject { public LogStructure(int numLinesAnalyzed, int numMessagesAnalyzed, String sampleStart, String charset, Boolean hasByteOrderMarker, Format format, String multilineStartPattern, String excludeLinesPattern, List inputFields, - Boolean hasHeaderRow, Character separator, Boolean shouldTrimFields, String grokPattern, String timestampField, + Boolean hasHeaderRow, Character delimiter, Boolean shouldTrimFields, String grokPattern, String timestampField, List timestampFormats, boolean needClientTimezone, Map mappings, List explanation) { @@ -216,7 +155,7 @@ public class LogStructure implements ToXContentObject { this.excludeLinesPattern = excludeLinesPattern; this.inputFields = (inputFields == null) ? null : Collections.unmodifiableList(new ArrayList<>(inputFields)); this.hasHeaderRow = hasHeaderRow; - this.separator = separator; + this.delimiter = delimiter; this.shouldTrimFields = shouldTrimFields; this.grokPattern = grokPattern; this.timestampField = timestampField; @@ -266,8 +205,8 @@ public class LogStructure implements ToXContentObject { return hasHeaderRow; } - public Character getSeparator() { - return separator; + public Character getDelimiter() { + return delimiter; } public Boolean getShouldTrimFields() { @@ -322,8 +261,8 @@ public class LogStructure implements ToXContentObject { if (hasHeaderRow != null) { builder.field(HAS_HEADER_ROW.getPreferredName(), hasHeaderRow.booleanValue()); } - if (separator != null) { - builder.field(SEPARATOR.getPreferredName(), String.valueOf(separator)); + if (delimiter != null) { + builder.field(DELIMITER.getPreferredName(), String.valueOf(delimiter)); } if (shouldTrimFields != null) { builder.field(SHOULD_TRIM_FIELDS.getPreferredName(), shouldTrimFields.booleanValue()); @@ -349,7 +288,7 @@ public class LogStructure implements ToXContentObject { public int hashCode() { return Objects.hash(numLinesAnalyzed, numMessagesAnalyzed, sampleStart, charset, hasByteOrderMarker, format, - multilineStartPattern, excludeLinesPattern, inputFields, hasHeaderRow, separator, shouldTrimFields, grokPattern, timestampField, + multilineStartPattern, excludeLinesPattern, inputFields, hasHeaderRow, delimiter, shouldTrimFields, grokPattern, timestampField, timestampFormats, needClientTimezone, mappings, explanation); } @@ -376,7 +315,7 @@ public class LogStructure implements ToXContentObject { Objects.equals(this.excludeLinesPattern, that.excludeLinesPattern) && Objects.equals(this.inputFields, that.inputFields) && Objects.equals(this.hasHeaderRow, that.hasHeaderRow) && - Objects.equals(this.separator, that.separator) && + Objects.equals(this.delimiter, that.delimiter) && Objects.equals(this.shouldTrimFields, that.shouldTrimFields) && Objects.equals(this.grokPattern, that.grokPattern) && Objects.equals(this.timestampField, that.timestampField) && @@ -397,7 +336,7 @@ public class LogStructure implements ToXContentObject { private String excludeLinesPattern; private List inputFields; private Boolean hasHeaderRow; - private Character separator; + private Character delimiter; private Boolean shouldTrimFields; private String grokPattern; private String timestampField; @@ -441,7 +380,6 @@ public class LogStructure implements ToXContentObject { public Builder setFormat(Format format) { this.format = Objects.requireNonNull(format); - this.separator = format.separator(); return this; } @@ -465,13 +403,13 @@ public class LogStructure implements ToXContentObject { return this; } - public Builder setShouldTrimFields(Boolean shouldTrimFields) { - this.shouldTrimFields = shouldTrimFields; + public Builder setDelimiter(Character delimiter) { + this.delimiter = delimiter; return this; } - public Builder setSeparator(Character separator) { - this.separator = separator; + public Builder setShouldTrimFields(Boolean shouldTrimFields) { + this.shouldTrimFields = shouldTrimFields; return this; } @@ -542,28 +480,22 @@ public class LogStructure implements ToXContentObject { if (hasHeaderRow != null) { throw new IllegalArgumentException("Has header row may not be specified for [" + format + "] structures."); } - if (separator != null) { - throw new IllegalArgumentException("Separator may not be specified for [" + format + "] structures."); + if (delimiter != null) { + throw new IllegalArgumentException("Delimiter may not be specified for [" + format + "] structures."); } if (grokPattern != null) { throw new IllegalArgumentException("Grok pattern may not be specified for [" + format + "] structures."); } break; - case CSV: - case TSV: - case SEMI_COLON_SEPARATED_VALUES: - case PIPE_SEPARATED_VALUES: + case DELIMITED: if (inputFields == null || inputFields.isEmpty()) { throw new IllegalArgumentException("Input fields must be specified for [" + format + "] structures."); } if (hasHeaderRow == null) { throw new IllegalArgumentException("Has header row must be specified for [" + format + "] structures."); } - Character expectedSeparator = format.separator(); - assert expectedSeparator != null; - if (expectedSeparator.equals(separator) == false) { - throw new IllegalArgumentException("Separator must be [" + expectedSeparator + "] for [" + format + - "] structures."); + if (delimiter == null) { + throw new IllegalArgumentException("Delimiter must be specified for [" + format + "] structures."); } if (grokPattern != null) { throw new IllegalArgumentException("Grok pattern may not be specified for [" + format + "] structures."); @@ -576,8 +508,8 @@ public class LogStructure implements ToXContentObject { if (hasHeaderRow != null) { throw new IllegalArgumentException("Has header row may not be specified for [" + format + "] structures."); } - if (separator != null) { - throw new IllegalArgumentException("Separator may not be specified for [" + format + "] structures."); + if (delimiter != null) { + throw new IllegalArgumentException("Delimiter may not be specified for [" + format + "] structures."); } if (shouldTrimFields != null) { throw new IllegalArgumentException("Should trim fields may not be specified for [" + format + "] structures."); @@ -607,7 +539,7 @@ public class LogStructure implements ToXContentObject { } return new LogStructure(numLinesAnalyzed, numMessagesAnalyzed, sampleStart, charset, hasByteOrderMarker, format, - multilineStartPattern, excludeLinesPattern, inputFields, hasHeaderRow, separator, shouldTrimFields, grokPattern, + multilineStartPattern, excludeLinesPattern, inputFields, hasHeaderRow, delimiter, shouldTrimFields, grokPattern, timestampField, timestampFormats, needClientTimezone, mappings, explanation); } } diff --git a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureFinderManager.java b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureFinderManager.java index a8fd9d7eb89..e747a588dfd 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureFinderManager.java +++ b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureFinderManager.java @@ -69,10 +69,10 @@ public final class LogStructureFinderManager { new JsonLogStructureFinderFactory(), new XmlLogStructureFinderFactory(), // ND-JSON will often also be valid (although utterly weird) CSV, so JSON must come before CSV - new CsvLogStructureFinderFactory(), - new TsvLogStructureFinderFactory(), - new SemiColonSeparatedValuesLogStructureFinderFactory(), - new PipeSeparatedValuesLogStructureFinderFactory(), + new DelimitedLogStructureFinderFactory(',', 2, false), + new DelimitedLogStructureFinderFactory('\t', 2, false), + new DelimitedLogStructureFinderFactory(';', 4, false), + new DelimitedLogStructureFinderFactory('|', 5, true), new TextLogStructureFinderFactory() )); diff --git a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureUtils.java b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureUtils.java index b1dfee22ee6..71a68c39991 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureUtils.java +++ b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureUtils.java @@ -21,12 +21,12 @@ import java.util.TreeMap; import java.util.stream.Collectors; import java.util.stream.Stream; -final class LogStructureUtils { +public final class LogStructureUtils { - static final String DEFAULT_TIMESTAMP_FIELD = "@timestamp"; - static final String MAPPING_TYPE_SETTING = "type"; - static final String MAPPING_FORMAT_SETTING = "format"; - static final String MAPPING_PROPERTIES_SETTING = "properties"; + public static final String DEFAULT_TIMESTAMP_FIELD = "@timestamp"; + public static final String MAPPING_TYPE_SETTING = "type"; + public static final String MAPPING_FORMAT_SETTING = "format"; + public static final String MAPPING_PROPERTIES_SETTING = "properties"; // NUMBER Grok pattern doesn't support scientific notation, so we extend it private static final Grok NUMBER_GROK = new Grok(Grok.getBuiltinPatterns(), "^%{NUMBER}(?:[eE][+-]?[0-3]?[0-9]{1,2})?$"); diff --git a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/PipeSeparatedValuesLogStructureFinderFactory.java b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/PipeSeparatedValuesLogStructureFinderFactory.java deleted file mode 100644 index 085599de847..00000000000 --- a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/PipeSeparatedValuesLogStructureFinderFactory.java +++ /dev/null @@ -1,38 +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; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.ml.logstructurefinder; - -import org.supercsv.prefs.CsvPreference; - -import java.io.IOException; -import java.util.List; - -public class PipeSeparatedValuesLogStructureFinderFactory implements LogStructureFinderFactory { - - private static final CsvPreference PIPE_PREFERENCE = new CsvPreference.Builder('"', '|', "\n").build(); - - /** - * Rules are: - * - The file must be valid pipe (|) separated values - * - It must contain at least two complete records - * - There must be at least five fields per record (otherwise files with coincidental - * or no pipe characters could be treated as pipe separated) - * - Every pipe separated value record except the last must have the same number of fields - * The reason the last record is allowed to have fewer fields than the others is that - * it could have been truncated when the file was sampled. - */ - @Override - public boolean canCreateFromSample(List explanation, String sample) { - return SeparatedValuesLogStructureFinder.canCreateFromSample(explanation, sample, 5, PIPE_PREFERENCE, "pipe separated values"); - } - - @Override - public LogStructureFinder createFromSample(List explanation, String sample, String charsetName, Boolean hasByteOrderMarker) - throws IOException { - return SeparatedValuesLogStructureFinder.makeSeparatedValuesLogStructureFinder(explanation, sample, charsetName, hasByteOrderMarker, - PIPE_PREFERENCE, true); - } -} diff --git a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/SemiColonSeparatedValuesLogStructureFinderFactory.java b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/SemiColonSeparatedValuesLogStructureFinderFactory.java deleted file mode 100644 index e0e80fa7465..00000000000 --- a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/SemiColonSeparatedValuesLogStructureFinderFactory.java +++ /dev/null @@ -1,37 +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; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.ml.logstructurefinder; - -import org.supercsv.prefs.CsvPreference; - -import java.io.IOException; -import java.util.List; - -public class SemiColonSeparatedValuesLogStructureFinderFactory implements LogStructureFinderFactory { - - /** - * Rules are: - * - The file must be valid semi-colon separated values - * - It must contain at least two complete records - * - There must be at least four fields per record (otherwise files with coincidental - * or no semi-colons could be treated as semi-colon separated) - * - Every semi-colon separated value record except the last must have the same number of fields - * The reason the last record is allowed to have fewer fields than the others is that - * it could have been truncated when the file was sampled. - */ - @Override - public boolean canCreateFromSample(List explanation, String sample) { - return SeparatedValuesLogStructureFinder.canCreateFromSample(explanation, sample, 4, - CsvPreference.EXCEL_NORTH_EUROPE_PREFERENCE, "semi-colon separated values"); - } - - @Override - public LogStructureFinder createFromSample(List explanation, String sample, String charsetName, Boolean hasByteOrderMarker) - throws IOException { - return SeparatedValuesLogStructureFinder.makeSeparatedValuesLogStructureFinder(explanation, sample, charsetName, hasByteOrderMarker, - CsvPreference.EXCEL_NORTH_EUROPE_PREFERENCE, false); - } -} diff --git a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/TsvLogStructureFinderFactory.java b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/TsvLogStructureFinderFactory.java index 733b32346fb..1b53a33f31e 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/TsvLogStructureFinderFactory.java +++ b/x-pack/plugin/ml/log-structure-finder/src/main/java/org/elasticsearch/xpack/ml/logstructurefinder/TsvLogStructureFinderFactory.java @@ -23,13 +23,13 @@ public class TsvLogStructureFinderFactory implements LogStructureFinderFactory { */ @Override public boolean canCreateFromSample(List explanation, String sample) { - return SeparatedValuesLogStructureFinder.canCreateFromSample(explanation, sample, 2, CsvPreference.TAB_PREFERENCE, "TSV"); + return DelimitedLogStructureFinder.canCreateFromSample(explanation, sample, 2, CsvPreference.TAB_PREFERENCE, "TSV"); } @Override public LogStructureFinder createFromSample(List explanation, String sample, String charsetName, Boolean hasByteOrderMarker) throws IOException { - return SeparatedValuesLogStructureFinder.makeSeparatedValuesLogStructureFinder(explanation, sample, charsetName, hasByteOrderMarker, + return DelimitedLogStructureFinder.makeDelimitedLogStructureFinder(explanation, sample, charsetName, hasByteOrderMarker, CsvPreference.TAB_PREFERENCE, false); } } diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/CsvLogStructureFinderFactoryTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/CsvLogStructureFinderFactoryTests.java deleted file mode 100644 index f53ee008d69..00000000000 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/CsvLogStructureFinderFactoryTests.java +++ /dev/null @@ -1,38 +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; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.ml.logstructurefinder; - -public class CsvLogStructureFinderFactoryTests extends LogStructureTestCase { - - private LogStructureFinderFactory factory = new CsvLogStructureFinderFactory(); - - // No need to check JSON or XML because they come earlier in the order we check formats - - public void testCanCreateFromSampleGivenCsv() { - - assertTrue(factory.canCreateFromSample(explanation, CSV_SAMPLE)); - } - - public void testCanCreateFromSampleGivenTsv() { - - assertFalse(factory.canCreateFromSample(explanation, TSV_SAMPLE)); - } - - public void testCanCreateFromSampleGivenSemiColonSeparatedValues() { - - assertFalse(factory.canCreateFromSample(explanation, SEMI_COLON_SEPARATED_VALUES_SAMPLE)); - } - - public void testCanCreateFromSampleGivenPipeSeparatedValues() { - - assertFalse(factory.canCreateFromSample(explanation, PIPE_SEPARATED_VALUES_SAMPLE)); - } - - public void testCanCreateFromSampleGivenText() { - - assertFalse(factory.canCreateFromSample(explanation, TEXT_SAMPLE)); - } -} diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinderFactoryTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinderFactoryTests.java new file mode 100644 index 00000000000..d9eadbc8f0f --- /dev/null +++ b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinderFactoryTests.java @@ -0,0 +1,93 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.logstructurefinder; + +public class DelimitedLogStructureFinderFactoryTests extends LogStructureTestCase { + + private LogStructureFinderFactory csvFactory = new DelimitedLogStructureFinderFactory(',', 2, false); + private LogStructureFinderFactory tsvFactory = new DelimitedLogStructureFinderFactory('\t', 2, false); + private LogStructureFinderFactory semiColonDelimitedfactory = new DelimitedLogStructureFinderFactory(';', 4, false); + private LogStructureFinderFactory pipeDelimitedFactory = new DelimitedLogStructureFinderFactory('|', 5, true); + + // CSV - no need to check JSON or XML because they come earlier in the order we check formats + + public void testCanCreateCsvFromSampleGivenCsv() { + + assertTrue(csvFactory.canCreateFromSample(explanation, CSV_SAMPLE)); + } + + public void testCanCreateCsvFromSampleGivenTsv() { + + assertFalse(csvFactory.canCreateFromSample(explanation, TSV_SAMPLE)); + } + + public void testCanCreateCsvFromSampleGivenSemiColonDelimited() { + + assertFalse(csvFactory.canCreateFromSample(explanation, SEMI_COLON_DELIMITED_SAMPLE)); + } + + public void testCanCreateCsvFromSampleGivenPipeDelimited() { + + assertFalse(csvFactory.canCreateFromSample(explanation, PIPE_DELIMITED_SAMPLE)); + } + + public void testCanCreateCsvFromSampleGivenText() { + + assertFalse(csvFactory.canCreateFromSample(explanation, TEXT_SAMPLE)); + } + + // TSV - no need to check JSON, XML or CSV because they come earlier in the order we check formats + + public void testCanCreateTsvFromSampleGivenTsv() { + + assertTrue(tsvFactory.canCreateFromSample(explanation, TSV_SAMPLE)); + } + + public void testCanCreateTsvFromSampleGivenSemiColonDelimited() { + + assertFalse(tsvFactory.canCreateFromSample(explanation, SEMI_COLON_DELIMITED_SAMPLE)); + } + + public void testCanCreateTsvFromSampleGivenPipeDelimited() { + + assertFalse(tsvFactory.canCreateFromSample(explanation, PIPE_DELIMITED_SAMPLE)); + } + + public void testCanCreateTsvFromSampleGivenText() { + + assertFalse(tsvFactory.canCreateFromSample(explanation, TEXT_SAMPLE)); + } + + // Semi-colon delimited - no need to check JSON, XML, CSV or TSV because they come earlier in the order we check formats + + public void testCanCreateSemiColonDelimitedFromSampleGivenSemiColonDelimited() { + + assertTrue(semiColonDelimitedfactory.canCreateFromSample(explanation, SEMI_COLON_DELIMITED_SAMPLE)); + } + + public void testCanCreateSemiColonDelimitedFromSampleGivenPipeDelimited() { + + assertFalse(semiColonDelimitedfactory.canCreateFromSample(explanation, PIPE_DELIMITED_SAMPLE)); + } + + public void testCanCreateSemiColonDelimitedFromSampleGivenText() { + + assertFalse(semiColonDelimitedfactory.canCreateFromSample(explanation, TEXT_SAMPLE)); + } + + // Pipe delimited - no need to check JSON, XML, CSV, TSV or semi-colon delimited + // values because they come earlier in the order we check formats + + public void testCanCreatePipeDelimitedFromSampleGivenPipeDelimited() { + + assertTrue(pipeDelimitedFactory.canCreateFromSample(explanation, PIPE_DELIMITED_SAMPLE)); + } + + public void testCanCreatePipeDelimitedFromSampleGivenText() { + + assertFalse(pipeDelimitedFactory.canCreateFromSample(explanation, TEXT_SAMPLE)); + } +} diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/SeparatedValuesLogStructureFinderTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinderTests.java similarity index 65% rename from x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/SeparatedValuesLogStructureFinderTests.java rename to x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinderTests.java index b62832a0a19..57c297cf8d5 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/SeparatedValuesLogStructureFinderTests.java +++ b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/DelimitedLogStructureFinderTests.java @@ -12,27 +12,27 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collections; -import static org.elasticsearch.xpack.ml.logstructurefinder.SeparatedValuesLogStructureFinder.levenshteinFieldwiseCompareRows; -import static org.elasticsearch.xpack.ml.logstructurefinder.SeparatedValuesLogStructureFinder.levenshteinDistance; +import static org.elasticsearch.xpack.ml.logstructurefinder.DelimitedLogStructureFinder.levenshteinFieldwiseCompareRows; +import static org.elasticsearch.xpack.ml.logstructurefinder.DelimitedLogStructureFinder.levenshteinDistance; import static org.hamcrest.Matchers.arrayContaining; -public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase { +public class DelimitedLogStructureFinderTests extends LogStructureTestCase { - private LogStructureFinderFactory factory = new CsvLogStructureFinderFactory(); + private LogStructureFinderFactory csvFactory = new DelimitedLogStructureFinderFactory(',', 2, false); public void testCreateConfigsGivenCompleteCsv() throws Exception { String sample = "time,message\n" + "2018-05-17T13:41:23,hello\n" + "2018-05-17T13:41:32,hello again\n"; - assertTrue(factory.canCreateFromSample(explanation, sample)); + assertTrue(csvFactory.canCreateFromSample(explanation, sample)); String charset = randomFrom(POSSIBLE_CHARSETS); Boolean hasByteOrderMarker = randomHasByteOrderMarker(charset); - LogStructureFinder structureFinder = factory.createFromSample(explanation, sample, charset, hasByteOrderMarker); + LogStructureFinder structureFinder = csvFactory.createFromSample(explanation, sample, charset, hasByteOrderMarker); LogStructure structure = structureFinder.getStructure(); - assertEquals(LogStructure.Format.CSV, structure.getFormat()); + assertEquals(LogStructure.Format.DELIMITED, structure.getFormat()); assertEquals(charset, structure.getCharset()); if (hasByteOrderMarker == null) { assertNull(structure.getHasByteOrderMarker()); @@ -41,7 +41,7 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase } assertEquals("^\"?time\"?,\"?message\"?", structure.getExcludeLinesPattern()); assertEquals("^\"?\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); - assertEquals(Character.valueOf(','), structure.getSeparator()); + assertEquals(Character.valueOf(','), structure.getDelimiter()); assertTrue(structure.getHasHeaderRow()); assertNull(structure.getShouldTrimFields()); assertEquals(Arrays.asList("time", "message"), structure.getInputFields()); @@ -55,15 +55,15 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase "\"hello\n" + "world\",2018-05-17T13:41:23,1\n" + "\"hello again\n"; // note that this last record is truncated - assertTrue(factory.canCreateFromSample(explanation, sample)); + assertTrue(csvFactory.canCreateFromSample(explanation, sample)); String charset = randomFrom(POSSIBLE_CHARSETS); Boolean hasByteOrderMarker = randomHasByteOrderMarker(charset); - LogStructureFinder structureFinder = factory.createFromSample(explanation, sample, charset, hasByteOrderMarker); + LogStructureFinder structureFinder = csvFactory.createFromSample(explanation, sample, charset, hasByteOrderMarker); LogStructure structure = structureFinder.getStructure(); - assertEquals(LogStructure.Format.CSV, structure.getFormat()); + assertEquals(LogStructure.Format.DELIMITED, structure.getFormat()); assertEquals(charset, structure.getCharset()); if (hasByteOrderMarker == null) { assertNull(structure.getHasByteOrderMarker()); @@ -72,7 +72,7 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase } assertEquals("^\"?message\"?,\"?time\"?,\"?count\"?", structure.getExcludeLinesPattern()); assertEquals("^.*?,\"?\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); - assertEquals(Character.valueOf(','), structure.getSeparator()); + assertEquals(Character.valueOf(','), structure.getDelimiter()); assertTrue(structure.getHasHeaderRow()); assertNull(structure.getShouldTrimFields()); assertEquals(Arrays.asList("message", "time", "count"), structure.getInputFields()); @@ -88,15 +88,15 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase "2,2016-12-31 15:15:01,2016-12-31 15:15:09,1,.00,1,N,264,264,2,1,0,0.5,0,0,0.3,1.8,,\n" + "1,2016-12-01 00:00:01,2016-12-01 00:10:22,1,1.60,1,N,163,143,2,9,0.5,0.5,0,0,0.3,10.3,,\n" + "1,2016-12-01 00:00:01,2016-12-01 00:11:01,1,1.40,1,N,164,229,1,9,0.5,0.5,2.05,0,0.3,12.35,,\n"; - assertTrue(factory.canCreateFromSample(explanation, sample)); + assertTrue(csvFactory.canCreateFromSample(explanation, sample)); String charset = randomFrom(POSSIBLE_CHARSETS); Boolean hasByteOrderMarker = randomHasByteOrderMarker(charset); - LogStructureFinder structureFinder = factory.createFromSample(explanation, sample, charset, hasByteOrderMarker); + LogStructureFinder structureFinder = csvFactory.createFromSample(explanation, sample, charset, hasByteOrderMarker); LogStructure structure = structureFinder.getStructure(); - assertEquals(LogStructure.Format.CSV, structure.getFormat()); + assertEquals(LogStructure.Format.DELIMITED, structure.getFormat()); assertEquals(charset, structure.getCharset()); if (hasByteOrderMarker == null) { assertNull(structure.getHasByteOrderMarker()); @@ -108,7 +108,7 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase "\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?,\"?\"?,\"?\"?", structure.getExcludeLinesPattern()); assertEquals("^.*?,\"?\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); - assertEquals(Character.valueOf(','), structure.getSeparator()); + assertEquals(Character.valueOf(','), structure.getDelimiter()); assertTrue(structure.getHasHeaderRow()); assertNull(structure.getShouldTrimFields()); assertEquals(Arrays.asList("VendorID", "tpep_pickup_datetime", "tpep_dropoff_datetime", "passenger_count", "trip_distance", @@ -126,15 +126,15 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase "2,2016-12-31 15:15:01,2016-12-31 15:15:09,1,.00,1,N,264,264,2,1,0,0.5,0,0,0.3,1.8,,\n" + "1,2016-12-01 00:00:01,2016-12-01 00:10:22,1,1.60,1,N,163,143,2,9,0.5,0.5,0,0,0.3,10.3,,\n" + "1,2016-12-01 00:00:01,2016-12-01 00:11:01,1,1.40,1,N,164,229,1,9,0.5,0.5,2.05,0,0.3,12.35,,\n"; - assertTrue(factory.canCreateFromSample(explanation, sample)); + assertTrue(csvFactory.canCreateFromSample(explanation, sample)); String charset = randomFrom(POSSIBLE_CHARSETS); Boolean hasByteOrderMarker = randomHasByteOrderMarker(charset); - LogStructureFinder structureFinder = factory.createFromSample(explanation, sample, charset, hasByteOrderMarker); + LogStructureFinder structureFinder = csvFactory.createFromSample(explanation, sample, charset, hasByteOrderMarker); LogStructure structure = structureFinder.getStructure(); - assertEquals(LogStructure.Format.CSV, structure.getFormat()); + assertEquals(LogStructure.Format.DELIMITED, structure.getFormat()); assertEquals(charset, structure.getCharset()); if (hasByteOrderMarker == null) { assertNull(structure.getHasByteOrderMarker()); @@ -146,7 +146,7 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase "\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?", structure.getExcludeLinesPattern()); assertEquals("^.*?,\"?\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); - assertEquals(Character.valueOf(','), structure.getSeparator()); + assertEquals(Character.valueOf(','), structure.getDelimiter()); assertTrue(structure.getHasHeaderRow()); assertNull(structure.getShouldTrimFields()); assertEquals(Arrays.asList("VendorID", "tpep_pickup_datetime", "tpep_dropoff_datetime", "passenger_count", "trip_distance", @@ -161,15 +161,15 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase String sample = "\"pos_id\",\"trip_id\",\"latitude\",\"longitude\",\"altitude\",\"timestamp\"\n" + "\"1\",\"3\",\"4703.7815\",\"1527.4713\",\"359.9\",\"2017-01-19 16:19:04.742113\"\n" + "\"2\",\"3\",\"4703.7815\",\"1527.4714\",\"359.9\",\"2017-01-19 16:19:05.741890\"\n"; - assertTrue(factory.canCreateFromSample(explanation, sample)); + assertTrue(csvFactory.canCreateFromSample(explanation, sample)); String charset = randomFrom(POSSIBLE_CHARSETS); Boolean hasByteOrderMarker = randomHasByteOrderMarker(charset); - LogStructureFinder structureFinder = factory.createFromSample(explanation, sample, charset, hasByteOrderMarker); + LogStructureFinder structureFinder = csvFactory.createFromSample(explanation, sample, charset, hasByteOrderMarker); LogStructure structure = structureFinder.getStructure(); - assertEquals(LogStructure.Format.CSV, structure.getFormat()); + assertEquals(LogStructure.Format.DELIMITED, structure.getFormat()); assertEquals(charset, structure.getCharset()); if (hasByteOrderMarker == null) { assertNull(structure.getHasByteOrderMarker()); @@ -179,7 +179,7 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase assertEquals("^\"?pos_id\"?,\"?trip_id\"?,\"?latitude\"?,\"?longitude\"?,\"?altitude\"?,\"?timestamp\"?", structure.getExcludeLinesPattern()); assertNull(structure.getMultilineStartPattern()); - assertEquals(Character.valueOf(','), structure.getSeparator()); + assertEquals(Character.valueOf(','), structure.getDelimiter()); assertTrue(structure.getHasHeaderRow()); assertNull(structure.getShouldTrimFields()); assertEquals(Arrays.asList("pos_id", "trip_id", "latitude", "longitude", "altitude", "timestamp"), structure.getInputFields()); @@ -195,8 +195,8 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase "2014-06-23 00:00:01Z,JBU,877.5927,farequote\n" + "2014-06-23 00:00:01Z,KLM,1355.4812,farequote\n"; - Tuple header = SeparatedValuesLogStructureFinder.findHeaderFromSample(explanation, - SeparatedValuesLogStructureFinder.readRows(withHeader, CsvPreference.EXCEL_PREFERENCE).v1()); + Tuple header = DelimitedLogStructureFinder.findHeaderFromSample(explanation, + DelimitedLogStructureFinder.readRows(withHeader, CsvPreference.EXCEL_PREFERENCE).v1()); assertTrue(header.v1()); assertThat(header.v2(), arrayContaining("time", "airline", "responsetime", "sourcetype")); @@ -208,8 +208,8 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase "2014-06-23 00:00:01Z,JBU,877.5927,farequote\n" + "2014-06-23 00:00:01Z,KLM,1355.4812,farequote\n"; - Tuple header = SeparatedValuesLogStructureFinder.findHeaderFromSample(explanation, - SeparatedValuesLogStructureFinder.readRows(withoutHeader, CsvPreference.EXCEL_PREFERENCE).v1()); + Tuple header = DelimitedLogStructureFinder.findHeaderFromSample(explanation, + DelimitedLogStructureFinder.readRows(withoutHeader, CsvPreference.EXCEL_PREFERENCE).v1()); assertFalse(header.v1()); assertThat(header.v2(), arrayContaining("column1", "column2", "column3", "column4")); @@ -251,43 +251,43 @@ public class SeparatedValuesLogStructureFinderTests extends LogStructureTestCase public void testLineHasUnescapedQuote() { - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("a,b,c", CsvPreference.EXCEL_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"a\",b,c", CsvPreference.EXCEL_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"a,b\",c", CsvPreference.EXCEL_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"a,b,c\"", CsvPreference.EXCEL_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("a,\"b\",c", CsvPreference.EXCEL_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("a,b,\"c\"", CsvPreference.EXCEL_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("a,\"b\"\"\",c", CsvPreference.EXCEL_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("a,b,\"c\"\"\"", CsvPreference.EXCEL_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"\"\"a\",b,c", CsvPreference.EXCEL_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"a\"\"\",b,c", CsvPreference.EXCEL_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"a,\"\"b\",c", CsvPreference.EXCEL_PREFERENCE)); - assertTrue(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("between\"words,b,c", CsvPreference.EXCEL_PREFERENCE)); - assertTrue(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("x and \"y\",b,c", CsvPreference.EXCEL_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("a,b,c", CsvPreference.EXCEL_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"a\",b,c", CsvPreference.EXCEL_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"a,b\",c", CsvPreference.EXCEL_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"a,b,c\"", CsvPreference.EXCEL_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("a,\"b\",c", CsvPreference.EXCEL_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("a,b,\"c\"", CsvPreference.EXCEL_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("a,\"b\"\"\",c", CsvPreference.EXCEL_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("a,b,\"c\"\"\"", CsvPreference.EXCEL_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"\"\"a\",b,c", CsvPreference.EXCEL_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"a\"\"\",b,c", CsvPreference.EXCEL_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"a,\"\"b\",c", CsvPreference.EXCEL_PREFERENCE)); + assertTrue(DelimitedLogStructureFinder.lineHasUnescapedQuote("between\"words,b,c", CsvPreference.EXCEL_PREFERENCE)); + assertTrue(DelimitedLogStructureFinder.lineHasUnescapedQuote("x and \"y\",b,c", CsvPreference.EXCEL_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("a\tb\tc", CsvPreference.TAB_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"a\"\tb\tc", CsvPreference.TAB_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"a\tb\"\tc", CsvPreference.TAB_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"a\tb\tc\"", CsvPreference.TAB_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("a\t\"b\"\tc", CsvPreference.TAB_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("a\tb\t\"c\"", CsvPreference.TAB_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("a\t\"b\"\"\"\tc", CsvPreference.TAB_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("a\tb\t\"c\"\"\"", CsvPreference.TAB_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"\"\"a\"\tb\tc", CsvPreference.TAB_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"a\"\"\"\tb\tc", CsvPreference.TAB_PREFERENCE)); - assertFalse(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("\"a\t\"\"b\"\tc", CsvPreference.TAB_PREFERENCE)); - assertTrue(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("between\"words\tb\tc", CsvPreference.TAB_PREFERENCE)); - assertTrue(SeparatedValuesLogStructureFinder.lineHasUnescapedQuote("x and \"y\"\tb\tc", CsvPreference.TAB_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("a\tb\tc", CsvPreference.TAB_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"a\"\tb\tc", CsvPreference.TAB_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"a\tb\"\tc", CsvPreference.TAB_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"a\tb\tc\"", CsvPreference.TAB_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("a\t\"b\"\tc", CsvPreference.TAB_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("a\tb\t\"c\"", CsvPreference.TAB_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("a\t\"b\"\"\"\tc", CsvPreference.TAB_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("a\tb\t\"c\"\"\"", CsvPreference.TAB_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"\"\"a\"\tb\tc", CsvPreference.TAB_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"a\"\"\"\tb\tc", CsvPreference.TAB_PREFERENCE)); + assertFalse(DelimitedLogStructureFinder.lineHasUnescapedQuote("\"a\t\"\"b\"\tc", CsvPreference.TAB_PREFERENCE)); + assertTrue(DelimitedLogStructureFinder.lineHasUnescapedQuote("between\"words\tb\tc", CsvPreference.TAB_PREFERENCE)); + assertTrue(DelimitedLogStructureFinder.lineHasUnescapedQuote("x and \"y\"\tb\tc", CsvPreference.TAB_PREFERENCE)); } public void testRowContainsDuplicateNonEmptyValues() { - assertFalse(SeparatedValuesLogStructureFinder.rowContainsDuplicateNonEmptyValues(Collections.singletonList("a"))); - assertFalse(SeparatedValuesLogStructureFinder.rowContainsDuplicateNonEmptyValues(Collections.singletonList(""))); - assertFalse(SeparatedValuesLogStructureFinder.rowContainsDuplicateNonEmptyValues(Arrays.asList("a", "b", "c"))); - assertTrue(SeparatedValuesLogStructureFinder.rowContainsDuplicateNonEmptyValues(Arrays.asList("a", "b", "a"))); - assertTrue(SeparatedValuesLogStructureFinder.rowContainsDuplicateNonEmptyValues(Arrays.asList("a", "b", "b"))); - assertFalse(SeparatedValuesLogStructureFinder.rowContainsDuplicateNonEmptyValues(Arrays.asList("a", "", ""))); - assertFalse(SeparatedValuesLogStructureFinder.rowContainsDuplicateNonEmptyValues(Arrays.asList("", "a", ""))); + assertFalse(DelimitedLogStructureFinder.rowContainsDuplicateNonEmptyValues(Collections.singletonList("a"))); + assertFalse(DelimitedLogStructureFinder.rowContainsDuplicateNonEmptyValues(Collections.singletonList(""))); + assertFalse(DelimitedLogStructureFinder.rowContainsDuplicateNonEmptyValues(Arrays.asList("a", "b", "c"))); + assertTrue(DelimitedLogStructureFinder.rowContainsDuplicateNonEmptyValues(Arrays.asList("a", "b", "a"))); + assertTrue(DelimitedLogStructureFinder.rowContainsDuplicateNonEmptyValues(Arrays.asList("a", "b", "b"))); + assertFalse(DelimitedLogStructureFinder.rowContainsDuplicateNonEmptyValues(Arrays.asList("a", "", ""))); + assertFalse(DelimitedLogStructureFinder.rowContainsDuplicateNonEmptyValues(Arrays.asList("", "a", ""))); } } diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/JsonLogStructureFinderFactoryTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/JsonLogStructureFinderFactoryTests.java index 39ef3b9eedb..cdbffa8259e 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/JsonLogStructureFinderFactoryTests.java +++ b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/JsonLogStructureFinderFactoryTests.java @@ -29,14 +29,14 @@ public class JsonLogStructureFinderFactoryTests extends LogStructureTestCase { assertFalse(factory.canCreateFromSample(explanation, TSV_SAMPLE)); } - public void testCanCreateFromSampleGivenSemiColonSeparatedValues() { + public void testCanCreateFromSampleGivenSemiColonDelimited() { - assertFalse(factory.canCreateFromSample(explanation, SEMI_COLON_SEPARATED_VALUES_SAMPLE)); + assertFalse(factory.canCreateFromSample(explanation, SEMI_COLON_DELIMITED_SAMPLE)); } - public void testCanCreateFromSampleGivenPipeSeparatedValues() { + public void testCanCreateFromSampleGivenPipeDelimited() { - assertFalse(factory.canCreateFromSample(explanation, PIPE_SEPARATED_VALUES_SAMPLE)); + assertFalse(factory.canCreateFromSample(explanation, PIPE_DELIMITED_SAMPLE)); } public void testCanCreateFromSampleGivenText() { diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/JsonLogStructureFinderTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/JsonLogStructureFinderTests.java index 2f727747bbf..917054919dd 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/JsonLogStructureFinderTests.java +++ b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/JsonLogStructureFinderTests.java @@ -29,7 +29,7 @@ public class JsonLogStructureFinderTests extends LogStructureTestCase { } assertNull(structure.getExcludeLinesPattern()); assertNull(structure.getMultilineStartPattern()); - assertNull(structure.getSeparator()); + assertNull(structure.getDelimiter()); assertNull(structure.getHasHeaderRow()); assertNull(structure.getShouldTrimFields()); assertNull(structure.getGrokPattern()); diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureFinderManagerTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureFinderManagerTests.java index 1f8691de8cf..520a55510c7 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureFinderManagerTests.java +++ b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureFinderManagerTests.java @@ -61,7 +61,7 @@ public class LogStructureFinderManagerTests extends LogStructureTestCase { public void testMakeBestStructureGivenCsv() throws Exception { assertThat(structureFinderManager.makeBestStructureFinder(explanation, "time,message\n" + "2018-05-17T13:41:23,hello\n", StandardCharsets.UTF_8.name(), randomBoolean()), - instanceOf(SeparatedValuesLogStructureFinder.class)); + instanceOf(DelimitedLogStructureFinder.class)); } public void testMakeBestStructureGivenText() throws Exception { diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureTestCase.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureTestCase.java index 5f9a87ef2a7..6b718fef6c7 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureTestCase.java +++ b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureTestCase.java @@ -34,14 +34,14 @@ public abstract class LogStructureTestCase extends ESTestCase { "\"level\":\"INFO\",\"pid\":42,\"thread\":\"0x7fff7d2a8000\",\"message\":\"message 2\",\"class\":\"ml\"," + "\"method\":\"core::SomeNoiseMaker\",\"file\":\"Noisemaker.cc\",\"line\":333}\n"; - protected static final String PIPE_SEPARATED_VALUES_SAMPLE = "2018-01-06 16:56:14.295748|INFO |VirtualServer |1 |" + + protected static final String PIPE_DELIMITED_SAMPLE = "2018-01-06 16:56:14.295748|INFO |VirtualServer |1 |" + "listening on 0.0.0.0:9987, :::9987\n" + "2018-01-06 17:19:44.465252|INFO |VirtualServer |1 |client " + "'User1'(id:2) changed default admin channelgroup to 'Guest'(id:8)\n" + "2018-01-06 17:21:25.764368|INFO |VirtualServer |1 |client " + "'User1'(id:2) was added to channelgroup 'Channel Admin'(id:5) by client 'User1'(id:2) in channel 'Default Channel'(id:1)"; - protected static final String SEMI_COLON_SEPARATED_VALUES_SAMPLE = "\"pos_id\";\"trip_id\";\"latitude\";\"longitude\";\"altitude\";" + + protected static final String SEMI_COLON_DELIMITED_SAMPLE = "\"pos_id\";\"trip_id\";\"latitude\";\"longitude\";\"altitude\";" + "\"timestamp\"\n" + "\"1\";\"3\";\"4703.7815\";\"1527.4713\";\"359.9\";\"2017-01-19 16:19:04.742113\"\n" + "\"2\";\"3\";\"4703.7815\";\"1527.4714\";\"359.9\";\"2017-01-19 16:19:05.741890\"\n" + diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureTests.java index 738928ed28a..302946dcaa8 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureTests.java +++ b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/LogStructureTests.java @@ -43,14 +43,12 @@ public class LogStructureTests extends AbstractXContentTestCase { builder.setExcludeLinesPattern(randomAlphaOfLength(100)); } - if (format.isSeparatedValues() || (format.supportsNesting() && randomBoolean())) { + if (format == LogStructure.Format.DELIMITED || (format.supportsNesting() && randomBoolean())) { builder.setInputFields(Arrays.asList(generateRandomStringArray(10, 10, false, false))); } - if (format.isSeparatedValues()) { + if (format == LogStructure.Format.DELIMITED) { builder.setHasHeaderRow(randomBoolean()); - if (rarely()) { - builder.setSeparator(format.separator()); - } + builder.setDelimiter(randomFrom(',', '\t', ';', '|')); } if (format.isSemiStructured()) { builder.setGrokPattern(randomAlphaOfLength(100)); diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/PipeSeparatedValuesLogStructureFinderFactoryTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/PipeSeparatedValuesLogStructureFinderFactoryTests.java deleted file mode 100644 index 3fd2fb7840a..00000000000 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/PipeSeparatedValuesLogStructureFinderFactoryTests.java +++ /dev/null @@ -1,23 +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; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.ml.logstructurefinder; - -public class PipeSeparatedValuesLogStructureFinderFactoryTests extends LogStructureTestCase { - - private LogStructureFinderFactory factory = new PipeSeparatedValuesLogStructureFinderFactory(); - - // No need to check JSON, XML, CSV, TSV or semi-colon separated values because they come earlier in the order we check formats - - public void testCanCreateFromSampleGivenPipeSeparatedValues() { - - assertTrue(factory.canCreateFromSample(explanation, PIPE_SEPARATED_VALUES_SAMPLE)); - } - - public void testCanCreateFromSampleGivenText() { - - assertFalse(factory.canCreateFromSample(explanation, TEXT_SAMPLE)); - } -} diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/SemiColonSeparatedValuesLogStructureFinderFactoryTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/SemiColonSeparatedValuesLogStructureFinderFactoryTests.java deleted file mode 100644 index 64dad7e078c..00000000000 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/SemiColonSeparatedValuesLogStructureFinderFactoryTests.java +++ /dev/null @@ -1,28 +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; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.ml.logstructurefinder; - -public class SemiColonSeparatedValuesLogStructureFinderFactoryTests extends LogStructureTestCase { - - private LogStructureFinderFactory factory = new SemiColonSeparatedValuesLogStructureFinderFactory(); - - // No need to check JSON, XML, CSV or TSV because they come earlier in the order we check formats - - public void testCanCreateFromSampleGivenSemiColonSeparatedValues() { - - assertTrue(factory.canCreateFromSample(explanation, SEMI_COLON_SEPARATED_VALUES_SAMPLE)); - } - - public void testCanCreateFromSampleGivenPipeSeparatedValues() { - - assertFalse(factory.canCreateFromSample(explanation, PIPE_SEPARATED_VALUES_SAMPLE)); - } - - public void testCanCreateFromSampleGivenText() { - - assertFalse(factory.canCreateFromSample(explanation, TEXT_SAMPLE)); - } -} diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TextLogStructureFinderFactoryTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TextLogStructureFinderFactoryTests.java index 267ce375d6e..c1b30cc7496 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TextLogStructureFinderFactoryTests.java +++ b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TextLogStructureFinderFactoryTests.java @@ -9,8 +9,8 @@ public class TextLogStructureFinderFactoryTests extends LogStructureTestCase { private LogStructureFinderFactory factory = new TextLogStructureFinderFactory(); - // No need to check JSON, XML, CSV, TSV, semi-colon separated values or pipe - // separated values because they come earlier in the order we check formats + // No need to check JSON, XML, CSV, TSV, semi-colon delimited values or pipe + // delimited values because they come earlier in the order we check formats public void testCanCreateFromSampleGivenText() { diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TextLogStructureFinderTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TextLogStructureFinderTests.java index 7c6a58bb683..c9e153a82c4 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TextLogStructureFinderTests.java +++ b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TextLogStructureFinderTests.java @@ -34,7 +34,7 @@ public class TextLogStructureFinderTests extends LogStructureTestCase { } assertNull(structure.getExcludeLinesPattern()); assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); - assertNull(structure.getSeparator()); + assertNull(structure.getDelimiter()); assertNull(structure.getHasHeaderRow()); assertNull(structure.getShouldTrimFields()); assertEquals("\\[%{TIMESTAMP_ISO8601:timestamp}\\]\\[%{LOGLEVEL:loglevel} \\]\\[.*", structure.getGrokPattern()); diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TsvLogStructureFinderFactoryTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TsvLogStructureFinderFactoryTests.java deleted file mode 100644 index 1c8acc14d32..00000000000 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/TsvLogStructureFinderFactoryTests.java +++ /dev/null @@ -1,33 +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; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.ml.logstructurefinder; - -public class TsvLogStructureFinderFactoryTests extends LogStructureTestCase { - - private LogStructureFinderFactory factory = new TsvLogStructureFinderFactory(); - - // No need to check JSON, XML or CSV because they come earlier in the order we check formats - - public void testCanCreateFromSampleGivenTsv() { - - assertTrue(factory.canCreateFromSample(explanation, TSV_SAMPLE)); - } - - public void testCanCreateFromSampleGivenSemiColonSeparatedValues() { - - assertFalse(factory.canCreateFromSample(explanation, SEMI_COLON_SEPARATED_VALUES_SAMPLE)); - } - - public void testCanCreateFromSampleGivenPipeSeparatedValues() { - - assertFalse(factory.canCreateFromSample(explanation, PIPE_SEPARATED_VALUES_SAMPLE)); - } - - public void testCanCreateFromSampleGivenText() { - - assertFalse(factory.canCreateFromSample(explanation, TEXT_SAMPLE)); - } -} diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/XmlLogStructureFinderFactoryTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/XmlLogStructureFinderFactoryTests.java index 27eb4ede040..b6dc3a56f1d 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/XmlLogStructureFinderFactoryTests.java +++ b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/XmlLogStructureFinderFactoryTests.java @@ -26,14 +26,14 @@ public class XmlLogStructureFinderFactoryTests extends LogStructureTestCase { assertFalse(factory.canCreateFromSample(explanation, TSV_SAMPLE)); } - public void testCanCreateFromSampleGivenSemiColonSeparatedValues() { + public void testCanCreateFromSampleGivenSemiColonDelimited() { - assertFalse(factory.canCreateFromSample(explanation, SEMI_COLON_SEPARATED_VALUES_SAMPLE)); + assertFalse(factory.canCreateFromSample(explanation, SEMI_COLON_DELIMITED_SAMPLE)); } - public void testCanCreateFromSampleGivenPipeSeparatedValues() { + public void testCanCreateFromSampleGivenPipeDelimited() { - assertFalse(factory.canCreateFromSample(explanation, PIPE_SEPARATED_VALUES_SAMPLE)); + assertFalse(factory.canCreateFromSample(explanation, PIPE_DELIMITED_SAMPLE)); } public void testCanCreateFromSampleGivenText() { diff --git a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/XmlLogStructureFinderTests.java b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/XmlLogStructureFinderTests.java index 0d04df152ef..de653d7bcd0 100644 --- a/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/XmlLogStructureFinderTests.java +++ b/x-pack/plugin/ml/log-structure-finder/src/test/java/org/elasticsearch/xpack/ml/logstructurefinder/XmlLogStructureFinderTests.java @@ -29,7 +29,7 @@ public class XmlLogStructureFinderTests extends LogStructureTestCase { } assertNull(structure.getExcludeLinesPattern()); assertEquals("^\\s* Date: Fri, 31 Aug 2018 13:08:32 +0300 Subject: [PATCH 12/16] Different handling for security specific errors in the CLI. Fix for https://github.com/elastic/elasticsearch/issues/33230 (#33255) --- docs/reference/sql/endpoints/cli.asciidoc | 9 +++++++++ .../main/java/org/elasticsearch/xpack/sql/cli/Cli.java | 5 +++++ 2 files changed, 14 insertions(+) diff --git a/docs/reference/sql/endpoints/cli.asciidoc b/docs/reference/sql/endpoints/cli.asciidoc index 0908c2344bb..eef2fbfbf59 100644 --- a/docs/reference/sql/endpoints/cli.asciidoc +++ b/docs/reference/sql/endpoints/cli.asciidoc @@ -22,6 +22,15 @@ the first parameter: $ ./bin/elasticsearch-sql-cli https://some.server:9200 -------------------------------------------------- +If security is enabled on your cluster, you can pass the username +and password in the form `username:password@host_name:port` +to the SQL CLI: + +[source,bash] +-------------------------------------------------- +$ ./bin/elasticsearch-sql-cli https://sql_user:strongpassword@some.server:9200 +-------------------------------------------------- + Once the CLI is running you can use any <> that Elasticsearch supports: diff --git a/x-pack/plugin/sql/sql-cli/src/main/java/org/elasticsearch/xpack/sql/cli/Cli.java b/x-pack/plugin/sql/sql-cli/src/main/java/org/elasticsearch/xpack/sql/cli/Cli.java index 357a4bcb5a7..6431f10a492 100644 --- a/x-pack/plugin/sql/sql-cli/src/main/java/org/elasticsearch/xpack/sql/cli/Cli.java +++ b/x-pack/plugin/sql/sql-cli/src/main/java/org/elasticsearch/xpack/sql/cli/Cli.java @@ -27,6 +27,7 @@ import org.elasticsearch.xpack.sql.client.Version; import org.jline.terminal.TerminalBuilder; import java.io.IOException; import java.net.ConnectException; +import java.sql.SQLInvalidAuthorizationSpecException; import java.util.Arrays; import java.util.List; import java.util.logging.LogManager; @@ -139,6 +140,10 @@ public class Cli extends LoggingAwareCommand { // Most likely Elasticsearch is not running throw new UserException(ExitCodes.IO_ERROR, "Cannot connect to the server " + con.connectionString() + " - " + ex.getCause().getMessage()); + } else if (ex.getCause() != null && ex.getCause() instanceof SQLInvalidAuthorizationSpecException) { + throw new UserException(ExitCodes.NOPERM, + "Cannot establish a secure connection to the server " + + con.connectionString() + " - " + ex.getCause().getMessage()); } else { // Most likely we connected to something other than Elasticsearch throw new UserException(ExitCodes.DATA_ERROR, From a88f8789a0a9260de0774ee7b76446c2a792294f Mon Sep 17 00:00:00 2001 From: Pablo Musa Date: Fri, 31 Aug 2018 14:48:55 +0200 Subject: [PATCH 13/16] Highlight that index_phrases only works if no slop is used (#33303) Highlight that `index_phrases` only works if no slop is used at query time. --- docs/reference/mapping/types/text.asciidoc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/reference/mapping/types/text.asciidoc b/docs/reference/mapping/types/text.asciidoc index e2336bd5cb0..db64e87412e 100644 --- a/docs/reference/mapping/types/text.asciidoc +++ b/docs/reference/mapping/types/text.asciidoc @@ -99,7 +99,7 @@ The following parameters are accepted by `text` fields: `index_phrases`:: If enabled, two-term word combinations ('shingles') are indexed into a separate - field. This allows exact phrase queries to run more efficiently, at the expense + field. This allows exact phrase queries (no slop) to run more efficiently, at the expense of a larger index. Note that this works best when stopwords are not removed, as phrases containing stopwords will not use the subsidiary field and will fall back to a standard phrase query. Accepts `true` or `false` (default). @@ -171,4 +171,4 @@ PUT my_index -------------------------------- // CONSOLE <1> `min_chars` must be greater than zero, defaults to 2 -<2> `max_chars` must be greater than or equal to `min_chars` and less than 20, defaults to 5 \ No newline at end of file +<2> `max_chars` must be greater than or equal to `min_chars` and less than 20, defaults to 5 From f6a570880c5c44adea4ad1f35e950c51335d8ac7 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Fri, 31 Aug 2018 16:01:54 +0300 Subject: [PATCH 14/16] Work around to be able to generate eclipse projects (#33295) * Work around to be able to generate eclipse projects https://github.com/gradle/gradle/issues/6582 --- build.gradle | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/build.gradle b/build.gradle index 36d3a543d89..f8282ca5ae8 100644 --- a/build.gradle +++ b/build.gradle @@ -16,7 +16,9 @@ * specific language governing permissions and limitations * under the License. */ + import com.github.jengelman.gradle.plugins.shadow.ShadowPlugin +import org.apache.tools.ant.taskdefs.condition.Os import org.elasticsearch.gradle.BuildPlugin import org.elasticsearch.gradle.LoggedExec import org.elasticsearch.gradle.Version @@ -24,14 +26,9 @@ import org.elasticsearch.gradle.VersionCollection import org.elasticsearch.gradle.VersionProperties import org.elasticsearch.gradle.plugin.PluginBuildPlugin import org.gradle.plugins.ide.eclipse.model.SourceFolder -import org.gradle.util.GradleVersion -import org.gradle.util.DistributionLocator -import org.apache.tools.ant.taskdefs.condition.Os -import org.apache.tools.ant.filters.ReplaceTokens import java.nio.file.Files import java.nio.file.Path -import java.security.MessageDigest plugins { id 'com.gradle.build-scan' version '1.13.2' @@ -512,6 +509,16 @@ allprojects { tasks.cleanEclipse.dependsOn(wipeEclipseSettings) // otherwise the eclipse merging is *super confusing* tasks.eclipse.dependsOn(cleanEclipse, copyEclipseSettings) + + // work arround https://github.com/gradle/gradle/issues/6582 + tasks.eclipseProject.mustRunAfter tasks.cleanEclipseProject + tasks.matching { it.name == 'eclipseClasspath' }.all { + it.mustRunAfter { tasks.cleanEclipseClasspath } + } + tasks.matching { it.name == 'eclipseJdt' }.all { + it.mustRunAfter { tasks.cleanEclipseJdt } + } + tasks.copyEclipseSettings.mustRunAfter tasks.wipeEclipseSettings } allprojects { From 0c4b3162be7c2a6bd03a44ab15821ef49a776738 Mon Sep 17 00:00:00 2001 From: Andrei Stefan Date: Fri, 31 Aug 2018 16:12:01 +0300 Subject: [PATCH 15/16] SQL: test coverage for JdbcResultSet (#32813) * Tests for JdbcResultSet * Added VARCHAR conversion for different types * Made error messages consistent: they now contain both the type that fails to be converted and the value itself --- .../xpack/sql/jdbc/jdbc/JdbcResultSet.java | 65 +- .../xpack/sql/jdbc/jdbc/TypeConverter.java | 87 +- .../jdbc/jdbc/JdbcPreparedStatementTests.java | 42 +- .../qa/sql/nosecurity/JdbcResultSetIT.java | 16 + .../xpack/qa/sql/jdbc/ResultSetTestCase.java | 1516 ++++++++++++++++- .../qa/sql/jdbc/SimpleExampleTestCase.java | 3 +- 6 files changed, 1621 insertions(+), 108 deletions(-) create mode 100644 x-pack/qa/sql/no-security/src/test/java/org/elasticsearch/xpack/qa/sql/nosecurity/JdbcResultSetIT.java diff --git a/x-pack/plugin/sql/jdbc/src/main/java/org/elasticsearch/xpack/sql/jdbc/jdbc/JdbcResultSet.java b/x-pack/plugin/sql/jdbc/src/main/java/org/elasticsearch/xpack/sql/jdbc/jdbc/JdbcResultSet.java index 201ae251ca0..ebdeaef15ca 100644 --- a/x-pack/plugin/sql/jdbc/src/main/java/org/elasticsearch/xpack/sql/jdbc/jdbc/JdbcResultSet.java +++ b/x-pack/plugin/sql/jdbc/src/main/java/org/elasticsearch/xpack/sql/jdbc/jdbc/JdbcResultSet.java @@ -133,72 +133,37 @@ class JdbcResultSet implements ResultSet, JdbcWrapper { @Override public boolean getBoolean(int columnIndex) throws SQLException { - Object val = column(columnIndex); - try { - return val != null ? (Boolean) val : false; - } catch (ClassCastException cce) { - throw new SQLException("unable to convert column " + columnIndex + " to a boolean", cce); - } + return column(columnIndex) != null ? getObject(columnIndex, Boolean.class) : false; } @Override public byte getByte(int columnIndex) throws SQLException { - Object val = column(columnIndex); - try { - return val != null ? ((Number) val).byteValue() : 0; - } catch (ClassCastException cce) { - throw new SQLException("unable to convert column " + columnIndex + " to a byte", cce); - } + return column(columnIndex) != null ? getObject(columnIndex, Byte.class) : 0; } @Override public short getShort(int columnIndex) throws SQLException { - Object val = column(columnIndex); - try { - return val != null ? ((Number) val).shortValue() : 0; - } catch (ClassCastException cce) { - throw new SQLException("unable to convert column " + columnIndex + " to a short", cce); - } + return column(columnIndex) != null ? getObject(columnIndex, Short.class) : 0; } @Override public int getInt(int columnIndex) throws SQLException { - Object val = column(columnIndex); - try { - return val != null ? ((Number) val).intValue() : 0; - } catch (ClassCastException cce) { - throw new SQLException("unable to convert column " + columnIndex + " to an int", cce); - } + return column(columnIndex) != null ? getObject(columnIndex, Integer.class) : 0; } @Override public long getLong(int columnIndex) throws SQLException { - Object val = column(columnIndex); - try { - return val != null ? ((Number) val).longValue() : 0; - } catch (ClassCastException cce) { - throw new SQLException("unable to convert column " + columnIndex + " to a long", cce); - } + return column(columnIndex) != null ? getObject(columnIndex, Long.class) : 0; } @Override public float getFloat(int columnIndex) throws SQLException { - Object val = column(columnIndex); - try { - return val != null ? ((Number) val).floatValue() : 0; - } catch (ClassCastException cce) { - throw new SQLException("unable to convert column " + columnIndex + " to a float", cce); - } + return column(columnIndex) != null ? getObject(columnIndex, Float.class) : 0; } @Override public double getDouble(int columnIndex) throws SQLException { - Object val = column(columnIndex); - try { - return val != null ? ((Number) val).doubleValue() : 0; - } catch (ClassCastException cce) { - throw new SQLException("unable to convert column " + columnIndex + " to a double", cce); - } + return column(columnIndex) != null ? getObject(columnIndex, Double.class) : 0; } @Override @@ -272,15 +237,29 @@ class JdbcResultSet implements ResultSet, JdbcWrapper { @Override public Date getDate(String columnLabel) throws SQLException { + // TODO: the error message in case the value in the column cannot be converted to a Date refers to a column index + // (for example - "unable to convert column 4 to a long") and not to the column name, which is a bit confusing. + // Should we reconsider this? Maybe by catching the exception here and rethrowing it with the columnLabel instead. return getDate(column(columnLabel)); } private Long dateTime(int columnIndex) throws SQLException { Object val = column(columnIndex); + JDBCType type = cursor.columns().get(columnIndex - 1).type; try { + // TODO: the B6 appendix of the jdbc spec does mention CHAR, VARCHAR, LONGVARCHAR, DATE, TIMESTAMP as supported + // jdbc types that should be handled by getDate and getTime methods. From all of those we support VARCHAR and + // TIMESTAMP. Should we consider the VARCHAR conversion as a later enhancement? + if (JDBCType.TIMESTAMP.equals(type)) { + // the cursor can return an Integer if the date-since-epoch is small enough, XContentParser (Jackson) will + // return the "smallest" data type for numbers when parsing + // TODO: this should probably be handled server side + return val == null ? null : ((Number) val).longValue(); + }; return val == null ? null : (Long) val; } catch (ClassCastException cce) { - throw new SQLException("unable to convert column " + columnIndex + " to a long", cce); + throw new SQLException( + format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Long", val, type.getName()), cce); } } diff --git a/x-pack/plugin/sql/jdbc/src/main/java/org/elasticsearch/xpack/sql/jdbc/jdbc/TypeConverter.java b/x-pack/plugin/sql/jdbc/src/main/java/org/elasticsearch/xpack/sql/jdbc/jdbc/TypeConverter.java index 3b5180b71f7..7b638d8bd09 100644 --- a/x-pack/plugin/sql/jdbc/src/main/java/org/elasticsearch/xpack/sql/jdbc/jdbc/TypeConverter.java +++ b/x-pack/plugin/sql/jdbc/src/main/java/org/elasticsearch/xpack/sql/jdbc/jdbc/TypeConverter.java @@ -10,7 +10,6 @@ import org.elasticsearch.xpack.sql.type.DataType; import java.sql.Date; import java.sql.JDBCType; -import java.sql.SQLDataException; import java.sql.SQLException; import java.sql.SQLFeatureNotSupportedException; import java.sql.Time; @@ -56,9 +55,10 @@ final class TypeConverter { } - private static final long DAY_IN_MILLIS = 60 * 60 * 24; + private static final long DAY_IN_MILLIS = 60 * 60 * 24 * 1000; private static final Map, JDBCType> javaToJDBC; + static { Map, JDBCType> aMap = Arrays.stream(DataType.values()) .filter(dataType -> dataType.javaClass() != null @@ -120,6 +120,7 @@ final class TypeConverter { } } + static long convertFromCalendarToUTC(long value, Calendar cal) { if (cal == null) { return value; @@ -143,11 +144,15 @@ final class TypeConverter { return (T) convert(val, columnType); } - if (type.isInstance(val)) { + // converting a Long to a Timestamp shouldn't be possible according to the spec, + // it feels a little brittle to check this scenario here and I don't particularly like it + // TODO: can we do any better or should we go over the spec and allow getLong(date) to be valid? + if (!(type == Long.class && columnType == JDBCType.TIMESTAMP) && type.isInstance(val)) { try { return type.cast(val); } catch (ClassCastException cce) { - throw new SQLDataException("Unable to convert " + val.getClass().getName() + " to " + columnType, cce); + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a %s", val, + columnType.getName(), type.getName()), cce); } } @@ -205,7 +210,8 @@ final class TypeConverter { if (type == OffsetDateTime.class) { return (T) asOffsetDateTime(val, columnType); } - throw new SQLException("Conversion from type [" + columnType + "] to [" + type.getName() + "] not supported"); + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a %s", val, + columnType.getName(), type.getName())); } /** @@ -336,8 +342,11 @@ final class TypeConverter { case FLOAT: case DOUBLE: return Boolean.valueOf(Integer.signum(((Number) val).intValue()) != 0); + case VARCHAR: + return Boolean.valueOf((String) val); default: - throw new SQLException("Conversion from type [" + columnType + "] to [Boolean] not supported"); + throw new SQLException( + format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Boolean", val, columnType.getName())); } } @@ -355,10 +364,16 @@ final class TypeConverter { case FLOAT: case DOUBLE: return safeToByte(safeToLong(((Number) val).doubleValue())); + case VARCHAR: + try { + return Byte.valueOf((String) val); + } catch (NumberFormatException e) { + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Byte", val), e); + } default: } - throw new SQLException("Conversion from type [" + columnType + "] to [Byte] not supported"); + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Byte", val, columnType.getName())); } private static Short asShort(Object val, JDBCType columnType) throws SQLException { @@ -374,10 +389,16 @@ final class TypeConverter { case FLOAT: case DOUBLE: return safeToShort(safeToLong(((Number) val).doubleValue())); + case VARCHAR: + try { + return Short.valueOf((String) val); + } catch (NumberFormatException e) { + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Short", val), e); + } default: } - throw new SQLException("Conversion from type [" + columnType + "] to [Short] not supported"); + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Short", val, columnType.getName())); } private static Integer asInteger(Object val, JDBCType columnType) throws SQLException { @@ -393,10 +414,18 @@ final class TypeConverter { case FLOAT: case DOUBLE: return safeToInt(safeToLong(((Number) val).doubleValue())); + case VARCHAR: + try { + return Integer.valueOf((String) val); + } catch (NumberFormatException e) { + throw new SQLException( + format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to an Integer", val), e); + } default: } - throw new SQLException("Conversion from type [" + columnType + "] to [Integer] not supported"); + throw new SQLException( + format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to an Integer", val, columnType.getName())); } private static Long asLong(Object val, JDBCType columnType) throws SQLException { @@ -412,12 +441,21 @@ final class TypeConverter { case FLOAT: case DOUBLE: return safeToLong(((Number) val).doubleValue()); - case TIMESTAMP: - return ((Number) val).longValue(); + //TODO: should we support conversion to TIMESTAMP? + //The spec says that getLong() should support the following types conversions: + //TINYINT, SMALLINT, INTEGER, BIGINT, REAL, FLOAT, DOUBLE, DECIMAL, NUMERIC, BIT, BOOLEAN, CHAR, VARCHAR, LONGVARCHAR + //case TIMESTAMP: + // return ((Number) val).longValue(); + case VARCHAR: + try { + return Long.valueOf((String) val); + } catch (NumberFormatException e) { + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Long", val), e); + } default: } - throw new SQLException("Conversion from type [" + columnType + "] to [Long] not supported"); + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Long", val, columnType.getName())); } private static Float asFloat(Object val, JDBCType columnType) throws SQLException { @@ -433,10 +471,16 @@ final class TypeConverter { case FLOAT: case DOUBLE: return Float.valueOf((((float) ((Number) val).doubleValue()))); + case VARCHAR: + try { + return Float.valueOf((String) val); + } catch (NumberFormatException e) { + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Float", val), e); + } default: } - throw new SQLException("Conversion from type [" + columnType + "] to [Float] not supported"); + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Float", val, columnType.getName())); } private static Double asDouble(Object val, JDBCType columnType) throws SQLException { @@ -451,32 +495,41 @@ final class TypeConverter { case REAL: case FLOAT: case DOUBLE: + return Double.valueOf(((Number) val).doubleValue()); + case VARCHAR: + try { + return Double.valueOf((String) val); + } catch (NumberFormatException e) { + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Double", val), e); + } default: } - throw new SQLException("Conversion from type [" + columnType + "] to [Double] not supported"); + throw new SQLException( + format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Double", val, columnType.getName())); } private static Date asDate(Object val, JDBCType columnType) throws SQLException { if (columnType == JDBCType.TIMESTAMP) { return new Date(utcMillisRemoveTime(((Number) val).longValue())); } - throw new SQLException("Conversion from type [" + columnType + "] to [Date] not supported"); + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Date", val, columnType.getName())); } private static Time asTime(Object val, JDBCType columnType) throws SQLException { if (columnType == JDBCType.TIMESTAMP) { return new Time(utcMillisRemoveDate(((Number) val).longValue())); } - throw new SQLException("Conversion from type [" + columnType + "] to [Time] not supported"); + throw new SQLException(format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Time", val, columnType.getName())); } private static Timestamp asTimestamp(Object val, JDBCType columnType) throws SQLException { if (columnType == JDBCType.TIMESTAMP) { return new Timestamp(((Number) val).longValue()); } - throw new SQLException("Conversion from type [" + columnType + "] to [Timestamp] not supported"); + throw new SQLException( + format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Timestamp", val, columnType.getName())); } private static byte[] asByteArray(Object val, JDBCType columnType) { diff --git a/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/jdbc/JdbcPreparedStatementTests.java b/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/jdbc/JdbcPreparedStatementTests.java index 9da06f6537c..35a3ec57487 100644 --- a/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/jdbc/JdbcPreparedStatementTests.java +++ b/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/jdbc/JdbcPreparedStatementTests.java @@ -25,6 +25,7 @@ import java.util.Date; import java.util.Locale; import java.util.Map; +import static java.lang.String.format; import static java.sql.JDBCType.BIGINT; import static java.sql.JDBCType.BOOLEAN; import static java.sql.JDBCType.DOUBLE; @@ -68,7 +69,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { JdbcPreparedStatement jps = createJdbcPreparedStatement(); SQLException sqle = expectThrows(SQLException.class, () -> jps.setObject(1, true, Types.TIMESTAMP)); - assertEquals("Conversion from type [BOOLEAN] to [Timestamp] not supported", sqle.getMessage()); + assertEquals("Unable to convert value [true] of type [BOOLEAN] to a Timestamp", sqle.getMessage()); } public void testSettingStringValues() throws SQLException { @@ -92,7 +93,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { JdbcPreparedStatement jps = createJdbcPreparedStatement(); SQLException sqle = expectThrows(SQLException.class, () -> jps.setObject(1, "foo bar", Types.INTEGER)); - assertEquals("Conversion from type [VARCHAR] to [Integer] not supported", sqle.getMessage()); + assertEquals("Unable to convert value [foo bar] of type [VARCHAR] to an Integer", sqle.getMessage()); } public void testSettingByteTypeValues() throws SQLException { @@ -128,7 +129,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { JdbcPreparedStatement jps = createJdbcPreparedStatement(); SQLException sqle = expectThrows(SQLException.class, () -> jps.setObject(1, (byte) 6, Types.TIMESTAMP)); - assertEquals("Conversion from type [TINYINT] to [Timestamp] not supported", sqle.getMessage()); + assertEquals("Unable to convert value [6] of type [TINYINT] to a Timestamp", sqle.getMessage()); } public void testSettingShortTypeValues() throws SQLException { @@ -161,7 +162,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { JdbcPreparedStatement jps = createJdbcPreparedStatement(); SQLException sqle = expectThrows(SQLException.class, () -> jps.setObject(1, (short) 6, Types.TIMESTAMP)); - assertEquals("Conversion from type [SMALLINT] to [Timestamp] not supported", sqle.getMessage()); + assertEquals("Unable to convert value [6] of type [SMALLINT] to a Timestamp", sqle.getMessage()); sqle = expectThrows(SQLException.class, () -> jps.setObject(1, 256, Types.TINYINT)); assertEquals("Numeric " + 256 + " out of range", sqle.getMessage()); @@ -195,7 +196,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { int someInt = randomInt(); SQLException sqle = expectThrows(SQLException.class, () -> jps.setObject(1, someInt, Types.TIMESTAMP)); - assertEquals("Conversion from type [INTEGER] to [Timestamp] not supported", sqle.getMessage()); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [INTEGER] to a Timestamp", someInt), sqle.getMessage()); Integer randomIntNotShort = randomIntBetween(32768, Integer.MAX_VALUE); sqle = expectThrows(SQLException.class, () -> jps.setObject(1, randomIntNotShort, Types.SMALLINT)); @@ -236,7 +237,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { long someLong = randomLong(); SQLException sqle = expectThrows(SQLException.class, () -> jps.setObject(1, someLong, Types.TIMESTAMP)); - assertEquals("Conversion from type [BIGINT] to [Timestamp] not supported", sqle.getMessage()); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [BIGINT] to a Timestamp", someLong), sqle.getMessage()); Long randomLongNotShort = randomLongBetween(Integer.MAX_VALUE + 1, Long.MAX_VALUE); sqle = expectThrows(SQLException.class, () -> jps.setObject(1, randomLongNotShort, Types.INTEGER)); @@ -277,7 +278,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { float someFloat = randomFloat(); SQLException sqle = expectThrows(SQLException.class, () -> jps.setObject(1, someFloat, Types.TIMESTAMP)); - assertEquals("Conversion from type [REAL] to [Timestamp] not supported", sqle.getMessage()); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [REAL] to a Timestamp", someFloat), sqle.getMessage()); Float floatNotInt = 5_155_000_000f; sqle = expectThrows(SQLException.class, () -> jps.setObject(1, floatNotInt, Types.INTEGER)); @@ -316,7 +317,8 @@ public class JdbcPreparedStatementTests extends ESTestCase { double someDouble = randomDouble(); SQLException sqle = expectThrows(SQLException.class, () -> jps.setObject(1, someDouble, Types.TIMESTAMP)); - assertEquals("Conversion from type [DOUBLE] to [Timestamp] not supported", sqle.getMessage()); + assertEquals( + format(Locale.ROOT, "Unable to convert value [%.128s] of type [DOUBLE] to a Timestamp", someDouble), sqle.getMessage()); Double doubleNotInt = 5_155_000_000d; sqle = expectThrows(SQLException.class, () -> jps.setObject(1, doubleNotInt, Types.INTEGER)); @@ -361,7 +363,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { public void testSettingTimestampValues() throws SQLException { JdbcPreparedStatement jps = createJdbcPreparedStatement(); - Timestamp someTimestamp = new Timestamp(randomMillisSinceEpoch()); + Timestamp someTimestamp = new Timestamp(randomLong()); jps.setTimestamp(1, someTimestamp); assertEquals(someTimestamp.getTime(), ((Date)value(jps)).getTime()); assertEquals(TIMESTAMP, jdbcType(jps)); @@ -372,7 +374,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { assertEquals(1456708675000L, convertFromUTCtoCalendar(((Date)value(jps)), nonDefaultCal)); assertEquals(TIMESTAMP, jdbcType(jps)); - long beforeEpochTime = -randomMillisSinceEpoch(); + long beforeEpochTime = randomLongBetween(Long.MIN_VALUE, 0); jps.setTimestamp(1, new Timestamp(beforeEpochTime), nonDefaultCal); assertEquals(beforeEpochTime, convertFromUTCtoCalendar(((Date)value(jps)), nonDefaultCal)); assertTrue(value(jps) instanceof java.util.Date); @@ -384,7 +386,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { public void testThrownExceptionsWhenSettingTimestampValues() throws SQLException { JdbcPreparedStatement jps = createJdbcPreparedStatement(); - Timestamp someTimestamp = new Timestamp(randomMillisSinceEpoch()); + Timestamp someTimestamp = new Timestamp(randomLong()); SQLException sqle = expectThrows(SQLFeatureNotSupportedException.class, () -> jps.setObject(1, someTimestamp, Types.INTEGER)); assertEquals("Conversion from type java.sql.Timestamp to INTEGER not supported", sqle.getMessage()); @@ -416,12 +418,12 @@ public class JdbcPreparedStatementTests extends ESTestCase { public void testSettingSqlDateValues() throws SQLException { JdbcPreparedStatement jps = createJdbcPreparedStatement(); - java.sql.Date someSqlDate = new java.sql.Date(randomMillisSinceEpoch()); + java.sql.Date someSqlDate = new java.sql.Date(randomLong()); jps.setDate(1, someSqlDate); assertEquals(someSqlDate.getTime(), ((Date)value(jps)).getTime()); assertEquals(TIMESTAMP, jdbcType(jps)); - someSqlDate = new java.sql.Date(randomMillisSinceEpoch()); + someSqlDate = new java.sql.Date(randomLong()); Calendar nonDefaultCal = randomCalendar(); jps.setDate(1, someSqlDate, nonDefaultCal); assertEquals(someSqlDate.getTime(), convertFromUTCtoCalendar(((Date)value(jps)), nonDefaultCal)); @@ -435,17 +437,17 @@ public class JdbcPreparedStatementTests extends ESTestCase { public void testThrownExceptionsWhenSettingSqlDateValues() throws SQLException { JdbcPreparedStatement jps = createJdbcPreparedStatement(); - java.sql.Date someSqlDate = new java.sql.Date(randomMillisSinceEpoch()); + java.sql.Date someSqlDate = new java.sql.Date(randomLong()); SQLException sqle = expectThrows(SQLFeatureNotSupportedException.class, - () -> jps.setObject(1, new java.sql.Date(randomMillisSinceEpoch()), Types.DOUBLE)); + () -> jps.setObject(1, new java.sql.Date(randomLong()), Types.DOUBLE)); assertEquals("Conversion from type " + someSqlDate.getClass().getName() + " to DOUBLE not supported", sqle.getMessage()); } public void testSettingCalendarValues() throws SQLException { JdbcPreparedStatement jps = createJdbcPreparedStatement(); Calendar someCalendar = randomCalendar(); - someCalendar.setTimeInMillis(randomMillisSinceEpoch()); + someCalendar.setTimeInMillis(randomLong()); jps.setObject(1, someCalendar); assertEquals(someCalendar.getTime(), (Date) value(jps)); @@ -472,7 +474,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { public void testSettingDateValues() throws SQLException { JdbcPreparedStatement jps = createJdbcPreparedStatement(); - Date someDate = new Date(randomMillisSinceEpoch()); + Date someDate = new Date(randomLong()); jps.setObject(1, someDate); assertEquals(someDate, (Date) value(jps)); @@ -486,7 +488,7 @@ public class JdbcPreparedStatementTests extends ESTestCase { public void testThrownExceptionsWhenSettingDateValues() throws SQLException { JdbcPreparedStatement jps = createJdbcPreparedStatement(); - Date someDate = new Date(randomMillisSinceEpoch()); + Date someDate = new Date(randomLong()); SQLException sqle = expectThrows(SQLFeatureNotSupportedException.class, () -> jps.setObject(1, someDate, Types.BIGINT)); assertEquals("Conversion from type " + someDate.getClass().getName() + " to BIGINT not supported", sqle.getMessage()); @@ -549,10 +551,6 @@ public class JdbcPreparedStatementTests extends ESTestCase { assertEquals("Conversion from type byte[] to DOUBLE not supported", sqle.getMessage()); } - private long randomMillisSinceEpoch() { - return randomLongBetween(0, System.currentTimeMillis()); - } - private JdbcPreparedStatement createJdbcPreparedStatement() throws SQLException { return new JdbcPreparedStatement(null, JdbcConfiguration.create("jdbc:es://l:1", null, 0), "?"); } diff --git a/x-pack/qa/sql/no-security/src/test/java/org/elasticsearch/xpack/qa/sql/nosecurity/JdbcResultSetIT.java b/x-pack/qa/sql/no-security/src/test/java/org/elasticsearch/xpack/qa/sql/nosecurity/JdbcResultSetIT.java new file mode 100644 index 00000000000..30756a11f62 --- /dev/null +++ b/x-pack/qa/sql/no-security/src/test/java/org/elasticsearch/xpack/qa/sql/nosecurity/JdbcResultSetIT.java @@ -0,0 +1,16 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.qa.sql.nosecurity; + +import org.elasticsearch.xpack.qa.sql.jdbc.ResultSetTestCase; + +/* + * Integration testing class for "no security" (cluster running without the Security plugin, + * or the Security is disbled) scenario. Runs all tests in the base class. + */ +public class JdbcResultSetIT extends ResultSetTestCase { +} diff --git a/x-pack/qa/sql/src/main/java/org/elasticsearch/xpack/qa/sql/jdbc/ResultSetTestCase.java b/x-pack/qa/sql/src/main/java/org/elasticsearch/xpack/qa/sql/jdbc/ResultSetTestCase.java index 861a6dccaba..447fc4f17e1 100644 --- a/x-pack/qa/sql/src/main/java/org/elasticsearch/xpack/qa/sql/jdbc/ResultSetTestCase.java +++ b/x-pack/qa/sql/src/main/java/org/elasticsearch/xpack/qa/sql/jdbc/ResultSetTestCase.java @@ -5,55 +5,1067 @@ */ package org.elasticsearch.xpack.qa.sql.jdbc; +import org.elasticsearch.client.Request; +import org.elasticsearch.common.CheckedBiFunction; +import org.elasticsearch.common.CheckedConsumer; +import org.elasticsearch.common.CheckedFunction; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.xpack.sql.jdbc.jdbc.JdbcConfiguration; +import org.elasticsearch.xpack.sql.jdbc.jdbcx.JdbcDataSource; +import org.elasticsearch.xpack.sql.type.DataType; + import java.io.IOException; +import java.io.InputStream; +import java.io.Reader; +import java.sql.Blob; +import java.sql.Clob; import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.JDBCType; +import java.sql.NClob; import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.ResultSetMetaData; import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; import java.sql.Timestamp; +import java.sql.Types; +import java.util.Arrays; +import java.util.Calendar; import java.util.Date; +import java.util.GregorianCalendar; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Locale; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.TimeZone; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.lang.String.format; +import static java.util.Calendar.DAY_OF_MONTH; +import static java.util.Calendar.ERA; +import static java.util.Calendar.HOUR_OF_DAY; +import static java.util.Calendar.MILLISECOND; +import static java.util.Calendar.MINUTE; +import static java.util.Calendar.MONTH; +import static java.util.Calendar.SECOND; +import static java.util.Calendar.YEAR; public class ResultSetTestCase extends JdbcIntegrationTestCase { - public void testGettingTimestamp() throws Exception { - long randomMillis = randomLongBetween(0, System.currentTimeMillis()); + + static final Set fieldsNames = Stream.of("test_byte", "test_integer", "test_long", "test_short", "test_double", + "test_float", "test_keyword") + .collect(Collectors.toCollection(HashSet::new)); + static final Map,JDBCType> dateTimeTestingFields = new HashMap,JDBCType>(); + static final String SELECT_ALL_FIELDS = "SELECT test_boolean, test_byte, test_integer," + + "test_long, test_short, test_double, test_float, test_keyword, test_date FROM test"; + static final String SELECT_WILDCARD = "SELECT * FROM test"; + static { + dateTimeTestingFields.put(new Tuple("test_boolean", true), DataType.BOOLEAN.jdbcType); + dateTimeTestingFields.put(new Tuple("test_byte", 1), DataType.BYTE.jdbcType); + dateTimeTestingFields.put(new Tuple("test_integer", 1), DataType.INTEGER.jdbcType); + dateTimeTestingFields.put(new Tuple("test_long", 1L), DataType.LONG.jdbcType); + dateTimeTestingFields.put(new Tuple("test_short", 1), DataType.SHORT.jdbcType); + dateTimeTestingFields.put(new Tuple("test_double", 1d), DataType.DOUBLE.jdbcType); + dateTimeTestingFields.put(new Tuple("test_float", 1f), DataType.FLOAT.jdbcType); + dateTimeTestingFields.put(new Tuple("test_keyword", "true"), DataType.KEYWORD.jdbcType); + } + + // Byte values testing + public void testGettingValidByteWithoutCasting() throws Exception { + byte random1 = randomByte(); + byte random2 = randomValueOtherThan(random1, () -> randomByte()); + byte random3 = randomValueOtherThanMany(Arrays.asList(random1, random2)::contains, () -> randomByte()); + + createTestDataForByteValueTests(random1, random2, random3); + + doWithQuery("SELECT test_byte, test_null_byte, test_keyword FROM test", (results) -> { + ResultSetMetaData resultSetMetaData = results.getMetaData(); + + results.next(); + assertEquals(3, resultSetMetaData.getColumnCount()); + assertEquals(Types.TINYINT, resultSetMetaData.getColumnType(1)); + assertEquals(Types.TINYINT, resultSetMetaData.getColumnType(2)); + assertEquals(random1, results.getByte(1)); + assertEquals(random1, results.getByte("test_byte")); + assertEquals(random1, (byte) results.getObject("test_byte", Byte.class)); + assertTrue(results.getObject(1) instanceof Byte); + + assertEquals(0, results.getByte(2)); + assertTrue(results.wasNull()); + assertEquals(null, results.getObject("test_null_byte")); + assertTrue(results.wasNull()); + + assertTrue(results.next()); + assertEquals(random2, results.getByte(1)); + assertEquals(random2, results.getByte("test_byte")); + assertTrue(results.getObject(1) instanceof Byte); + assertEquals(random3, results.getByte("test_keyword")); + + assertFalse(results.next()); + }); + } + + public void testGettingValidByteWithCasting() throws Exception { + Map map = createTestDataForNumericValueTypes(() -> randomByte()); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + for(Entry e : map.entrySet()) { + byte actual = results.getObject(e.getKey(), Byte.class); + if (e.getValue() instanceof Double) { + assertEquals("For field " + e.getKey(), Math.round(e.getValue().doubleValue()), results.getByte(e.getKey())); + assertEquals("For field " + e.getKey(), Math.round(e.getValue().doubleValue()), actual); + } else if (e.getValue() instanceof Float) { + assertEquals("For field " + e.getKey(), Math.round(e.getValue().floatValue()), results.getByte(e.getKey())); + assertEquals("For field " + e.getKey(), Math.round(e.getValue().floatValue()), actual); + } else { + assertEquals("For field " + e.getKey(), e.getValue().byteValue(), results.getByte(e.getKey())); + assertEquals("For field " + e.getKey(), e.getValue().byteValue(), actual); + } + } + }); + } + + public void testGettingInvalidByte() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_keyword").field("type", "keyword").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + + int intNotByte = randomIntBetween(Byte.MAX_VALUE + 1, Integer.MAX_VALUE); + long longNotByte = randomLongBetween(Byte.MAX_VALUE + 1, Long.MAX_VALUE); + short shortNotByte = (short) randomIntBetween(Byte.MAX_VALUE + 1, Short.MAX_VALUE); + double doubleNotByte = randomDoubleBetween(Byte.MAX_VALUE + 1, Double.MAX_VALUE, true); + float floatNotByte = randomFloatBetween(Byte.MAX_VALUE + 1, Float.MAX_VALUE); + String randomString = randomUnicodeOfCodepointLengthBetween(128, 256); + long randomDate = randomLong(); + + String doubleErrorMessage = (doubleNotByte > Long.MAX_VALUE || doubleNotByte < Long.MIN_VALUE) ? + Double.toString(doubleNotByte) : Long.toString(Math.round(doubleNotByte)); + + index("test", "1", builder -> { + builder.field("test_integer", intNotByte); + builder.field("test_long", longNotByte); + builder.field("test_short", shortNotByte); + builder.field("test_double", doubleNotByte); + builder.field("test_float", floatNotByte); + builder.field("test_keyword", randomString); + builder.field("test_date", randomDate); + }); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + + SQLException sqle = expectThrows(SQLException.class, () -> results.getByte("test_integer")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", intNotByte), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_integer", Byte.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", intNotByte), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getByte("test_short")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", shortNotByte), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_short", Byte.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", shortNotByte), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getByte("test_long")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Long.toString(longNotByte)), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_long", Byte.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Long.toString(longNotByte)), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getByte("test_double")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", doubleErrorMessage), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_double", Byte.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", doubleErrorMessage), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getByte("test_float")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Double.toString(floatNotByte)), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_float", Byte.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Double.toString(floatNotByte)), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getByte("test_keyword")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Byte", randomString), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_keyword", Byte.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Byte", randomString), + sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getByte("test_date")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Byte", randomDate), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_date", Byte.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Byte", randomDate), + sqle.getMessage()); + }); + } + + // Short values testing + public void testGettingValidShortWithoutCasting() throws Exception { + short random1 = randomShort(); + short random2 = randomValueOtherThan(random1, () -> randomShort()); + short random3 = randomValueOtherThanMany(Arrays.asList(random1, random2)::contains, () -> randomShort()); + + createTestDataForShortValueTests(random1, random2, random3); + + doWithQuery("SELECT test_short, test_null_short, test_keyword FROM test", (results) -> { + ResultSetMetaData resultSetMetaData = results.getMetaData(); + + results.next(); + assertEquals(3, resultSetMetaData.getColumnCount()); + assertEquals(Types.SMALLINT, resultSetMetaData.getColumnType(1)); + assertEquals(Types.SMALLINT, resultSetMetaData.getColumnType(2)); + assertEquals(random1, results.getShort(1)); + assertEquals(random1, results.getShort("test_short")); + assertEquals(random1, results.getObject("test_short")); + assertTrue(results.getObject(1) instanceof Short); + + assertEquals(0, results.getShort(2)); + assertTrue(results.wasNull()); + assertEquals(null, results.getObject("test_null_short")); + assertTrue(results.wasNull()); + + assertTrue(results.next()); + assertEquals(random2, results.getShort(1)); + assertEquals(random2, results.getShort("test_short")); + assertTrue(results.getObject(1) instanceof Short); + assertEquals(random3, results.getShort("test_keyword")); + + assertFalse(results.next()); + }); + } + + public void testGettingValidShortWithCasting() throws Exception { + Map map = createTestDataForNumericValueTypes(() -> randomShort()); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + for(Entry e : map.entrySet()) { + short actual = (short) results.getObject(e.getKey(), Short.class); + if (e.getValue() instanceof Double) { + assertEquals("For field " + e.getKey(), Math.round(e.getValue().doubleValue()), results.getShort(e.getKey())); + assertEquals("For field " + e.getKey(), Math.round(e.getValue().doubleValue()), actual); + } else if (e.getValue() instanceof Float) { + assertEquals("For field " + e.getKey(), Math.round(e.getValue().floatValue()), results.getShort(e.getKey())); + assertEquals("For field " + e.getKey(), Math.round(e.getValue().floatValue()), actual); + } else { + assertEquals("For field " + e.getKey(), + e.getValue().shortValue(), results.getShort(e.getKey())); + assertEquals("For field " + e.getKey(), e.getValue().shortValue(), actual); + } + } + }); + } + + public void testGettingInvalidShort() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_keyword").field("type", "keyword").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + + int intNotShort = randomIntBetween(Short.MAX_VALUE + 1, Integer.MAX_VALUE); + long longNotShort = randomLongBetween(Short.MAX_VALUE + 1, Long.MAX_VALUE); + double doubleNotShort = randomDoubleBetween(Short.MAX_VALUE + 1, Double.MAX_VALUE, true); + float floatNotShort = randomFloatBetween(Short.MAX_VALUE + 1, Float.MAX_VALUE); + String randomString = randomUnicodeOfCodepointLengthBetween(128, 256); + long randomDate = randomLong(); + + String doubleErrorMessage = (doubleNotShort > Long.MAX_VALUE || doubleNotShort < Long.MIN_VALUE) ? + Double.toString(doubleNotShort) : Long.toString(Math.round(doubleNotShort)); + + index("test", "1", builder -> { + builder.field("test_integer", intNotShort); + builder.field("test_long", longNotShort); + builder.field("test_double", doubleNotShort); + builder.field("test_float", floatNotShort); + builder.field("test_keyword", randomString); + builder.field("test_date", randomDate); + }); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + + SQLException sqle = expectThrows(SQLException.class, () -> results.getShort("test_integer")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", intNotShort), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_integer", Short.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", intNotShort), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getShort("test_long")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Long.toString(longNotShort)), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_long", Short.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Long.toString(longNotShort)), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getShort("test_double")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", doubleErrorMessage), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_double", Short.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", doubleErrorMessage), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getShort("test_float")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Double.toString(floatNotShort)), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_float", Short.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Double.toString(floatNotShort)), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getShort("test_keyword")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Short", randomString), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_keyword", Short.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Short", randomString), + sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getShort("test_date")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Short", randomDate), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_date", Short.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Short", randomDate), + sqle.getMessage()); + }); + } + + // Integer values testing + public void testGettingValidIntegerWithoutCasting() throws Exception { + int random1 = randomInt(); + int random2 = randomValueOtherThan(random1, () -> randomInt()); + int random3 = randomValueOtherThanMany(Arrays.asList(random1, random2)::contains, () -> randomInt()); + + createTestDataForIntegerValueTests(random1, random2, random3); + + doWithQuery("SELECT test_integer,test_null_integer,test_keyword FROM test", (results) -> { + ResultSetMetaData resultSetMetaData = results.getMetaData(); + + results.next(); + assertEquals(3, resultSetMetaData.getColumnCount()); + assertEquals(Types.INTEGER, resultSetMetaData.getColumnType(1)); + assertEquals(Types.INTEGER, resultSetMetaData.getColumnType(2)); + assertEquals(random1, results.getInt(1)); + assertEquals(random1, results.getInt("test_integer")); + assertEquals(random1, (int) results.getObject("test_integer", Integer.class)); + assertTrue(results.getObject(1) instanceof Integer); + + assertEquals(0, results.getInt(2)); + assertTrue(results.wasNull()); + assertEquals(null, results.getObject("test_null_integer")); + assertTrue(results.wasNull()); + + assertTrue(results.next()); + assertEquals(random2, results.getInt(1)); + assertEquals(random2, results.getInt("test_integer")); + assertTrue(results.getObject(1) instanceof Integer); + assertEquals(random3, results.getInt("test_keyword")); + + assertFalse(results.next()); + }); + } + + public void testGettingValidIntegerWithCasting() throws Exception { + Map map = createTestDataForNumericValueTypes(() -> randomInt()); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + for(Entry e : map.entrySet()) { + int actual = results.getObject(e.getKey(), Integer.class); + if (e.getValue() instanceof Double) { + assertEquals("For field " + e.getKey(), Math.round(e.getValue().doubleValue()), results.getInt(e.getKey())); + assertEquals("For field " + e.getKey(), Math.round(e.getValue().doubleValue()), actual); + } else if (e.getValue() instanceof Float) { + assertEquals("For field " + e.getKey(), Math.round(e.getValue().floatValue()), results.getInt(e.getKey())); + assertEquals("For field " + e.getKey(), Math.round(e.getValue().floatValue()), actual); + } else { + assertEquals("For field " + e.getKey(), e.getValue().intValue(), results.getInt(e.getKey())); + assertEquals("For field " + e.getKey(), e.getValue().intValue(), actual); + } + } + }); + } + + public void testGettingInvalidInteger() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_keyword").field("type", "keyword").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + + long longNotInt = randomLongBetween(getMaxIntPlusOne(), Long.MAX_VALUE); + double doubleNotInt = randomDoubleBetween(getMaxIntPlusOne().doubleValue(), Double.MAX_VALUE, true); + float floatNotInt = randomFloatBetween(getMaxIntPlusOne().floatValue(), Float.MAX_VALUE); + String randomString = randomUnicodeOfCodepointLengthBetween(128, 256); + long randomDate = randomLong(); + + String doubleErrorMessage = (doubleNotInt > Long.MAX_VALUE || doubleNotInt < Long.MIN_VALUE) ? + Double.toString(doubleNotInt) : Long.toString(Math.round(doubleNotInt)); + + index("test", "1", builder -> { + builder.field("test_long", longNotInt); + builder.field("test_double", doubleNotInt); + builder.field("test_float", floatNotInt); + builder.field("test_keyword", randomString); + builder.field("test_date", randomDate); + }); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + + SQLException sqle = expectThrows(SQLException.class, () -> results.getInt("test_long")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Long.toString(longNotInt)), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_long", Integer.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Long.toString(longNotInt)), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getInt("test_double")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", doubleErrorMessage), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_double", Integer.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", doubleErrorMessage), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getInt("test_float")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Double.toString(floatNotInt)), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_float", Integer.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Double.toString(floatNotInt)), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getInt("test_keyword")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to an Integer", randomString), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_keyword", Integer.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to an Integer", randomString), + sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getInt("test_date")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to an Integer", randomDate), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_date", Integer.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to an Integer", randomDate), + sqle.getMessage()); + }); + } + + // Long values testing + public void testGettingValidLongWithoutCasting() throws Exception { + long random1 = randomLong(); + long random2 = randomValueOtherThan(random1, () -> randomLong()); + long random3 = randomValueOtherThanMany(Arrays.asList(random1, random2)::contains, () -> randomLong()); + + createTestDataForLongValueTests(random1, random2, random3); + + doWithQuery("SELECT test_long, test_null_long, test_keyword FROM test", (results) -> { + ResultSetMetaData resultSetMetaData = results.getMetaData(); + + results.next(); + assertEquals(3, resultSetMetaData.getColumnCount()); + assertEquals(Types.BIGINT, resultSetMetaData.getColumnType(1)); + assertEquals(Types.BIGINT, resultSetMetaData.getColumnType(2)); + assertEquals(random1, results.getLong(1)); + assertEquals(random1, results.getLong("test_long")); + assertEquals(random1, (long) results.getObject("test_long", Long.class)); + assertTrue(results.getObject(1) instanceof Long); + + assertEquals(0, results.getLong(2)); + assertTrue(results.wasNull()); + assertEquals(null, results.getObject("test_null_long")); + assertTrue(results.wasNull()); + + assertTrue(results.next()); + assertEquals(random2, results.getLong(1)); + assertEquals(random2, results.getLong("test_long")); + assertTrue(results.getObject(1) instanceof Long); + assertEquals(random3, results.getLong("test_keyword")); + + assertFalse(results.next()); + }); + } + + public void testGettingValidLongWithCasting() throws Exception { + Map map = createTestDataForNumericValueTypes(() -> randomLong()); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + for(Entry e : map.entrySet()) { + long actual = results.getObject(e.getKey(), Long.class); + if (e.getValue() instanceof Double || e.getValue() instanceof Float) { + assertEquals("For field " + e.getKey(), Math.round(e.getValue().doubleValue()), results.getLong(e.getKey())); + assertEquals("For field " + e.getKey(), Math.round(e.getValue().doubleValue()), actual); + } else { + assertEquals("For field " + e.getKey(), e.getValue().longValue(), results.getLong(e.getKey())); + assertEquals("For field " + e.getKey(), e.getValue().longValue(), actual); + } + } + }); + } + + public void testGettingInvalidLong() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_keyword").field("type", "keyword").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + + double doubleNotLong = randomDoubleBetween(getMaxLongPlusOne().doubleValue(), Double.MAX_VALUE, true); + float floatNotLong = randomFloatBetween(getMaxLongPlusOne().floatValue(), Float.MAX_VALUE); + String randomString = randomUnicodeOfCodepointLengthBetween(128, 256); + long randomDate = randomLong(); + + index("test", "1", builder -> { + builder.field("test_double", doubleNotLong); + builder.field("test_float", floatNotLong); + builder.field("test_keyword", randomString); + builder.field("test_date", randomDate); + }); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + + SQLException sqle = expectThrows(SQLException.class, () -> results.getLong("test_double")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Double.toString(doubleNotLong)), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_double", Long.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Double.toString(doubleNotLong)), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getLong("test_float")); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Double.toString(floatNotLong)), sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_float", Long.class)); + assertEquals(format(Locale.ROOT, "Numeric %s out of range", Double.toString(floatNotLong)), sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getLong("test_keyword")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Long", randomString), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_keyword", Long.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Long", randomString), + sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getLong("test_date")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Long", randomDate), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_date", Long.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Long", randomDate), + sqle.getMessage()); + }); + } + + // Double values testing + public void testGettingValidDoubleWithoutCasting() throws Exception { + double random1 = randomDouble(); + double random2 = randomValueOtherThan(random1, () -> randomDouble()); + double random3 = randomValueOtherThanMany(Arrays.asList(random1, random2)::contains, () -> randomDouble()); + + createTestDataForDoubleValueTests(random1, random2, random3); + + doWithQuery("SELECT test_double, test_null_double, test_keyword FROM test", (results) -> { + ResultSetMetaData resultSetMetaData = results.getMetaData(); + + results.next(); + assertEquals(3, resultSetMetaData.getColumnCount()); + assertEquals(Types.DOUBLE, resultSetMetaData.getColumnType(1)); + assertEquals(Types.DOUBLE, resultSetMetaData.getColumnType(2)); + assertEquals(random1, results.getDouble(1), 0.0d); + assertEquals(random1, results.getDouble("test_double"), 0.0d); + assertEquals(random1, results.getObject("test_double", Double.class), 0.0d); + assertTrue(results.getObject(1) instanceof Double); + + assertEquals(0, results.getDouble(2), 0.0d); + assertTrue(results.wasNull()); + assertEquals(null, results.getObject("test_null_double")); + assertTrue(results.wasNull()); + + assertTrue(results.next()); + assertEquals(random2, results.getDouble(1), 0.0d); + assertEquals(random2, results.getDouble("test_double"), 0.0d); + assertTrue(results.getObject(1) instanceof Double); + assertEquals(random3, results.getDouble("test_keyword"), 0.0d); + + assertFalse(results.next()); + }); + } + + public void testGettingValidDoubleWithCasting() throws Exception { + Map map = createTestDataForNumericValueTypes(() -> randomDouble()); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + for(Entry e : map.entrySet()) { + assertEquals("For field " + e.getKey(), e.getValue().doubleValue(), results.getDouble(e.getKey()), 0.0d); + assertEquals("For field " + e.getKey(), + e.getValue().doubleValue(), results.getObject(e.getKey(), Double.class), 0.0d); + } + }); + } + + public void testGettingInvalidDouble() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_keyword").field("type", "keyword").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + + String randomString = randomUnicodeOfCodepointLengthBetween(128, 256); + long randomDate = randomLong(); + + index("test", "1", builder -> { + builder.field("test_keyword", randomString); + builder.field("test_date", randomDate); + }); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + + SQLException sqle = expectThrows(SQLException.class, () -> results.getDouble("test_keyword")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Double", randomString), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_keyword", Double.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Double", randomString), + sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getDouble("test_date")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Double", randomDate), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_date", Double.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Double", randomDate), + sqle.getMessage()); + }); + } + + // Float values testing + public void testGettingValidFloatWithoutCasting() throws Exception { + float random1 = randomFloat(); + float random2 = randomValueOtherThan(random1, () -> randomFloat()); + float random3 = randomValueOtherThanMany(Arrays.asList(random1, random2)::contains, () -> randomFloat()); + + createTestDataForFloatValueTests(random1, random2, random3); + + doWithQuery("SELECT test_float, test_null_float, test_keyword FROM test", (results) -> { + ResultSetMetaData resultSetMetaData = results.getMetaData(); + + results.next(); + assertEquals(3, resultSetMetaData.getColumnCount()); + assertEquals(Types.REAL, resultSetMetaData.getColumnType(1)); + assertEquals(Types.REAL, resultSetMetaData.getColumnType(2)); + assertEquals(random1, results.getFloat(1), 0.0f); + assertEquals(random1, results.getFloat("test_float"), 0.0f); + assertEquals(random1, results.getObject("test_float", Float.class), 0.0f); + assertTrue(results.getObject(1) instanceof Float); + + assertEquals(0, results.getFloat(2), 0.0d); + assertTrue(results.wasNull()); + assertEquals(null, results.getObject("test_null_float")); + assertTrue(results.wasNull()); + + assertTrue(results.next()); + assertEquals(random2, results.getFloat(1), 0.0d); + assertEquals(random2, results.getFloat("test_float"), 0.0d); + assertTrue(results.getObject(1) instanceof Float); + assertEquals(random3, results.getFloat("test_keyword"), 0.0d); + + assertFalse(results.next()); + }); + } + + public void testGettingValidFloatWithCasting() throws Exception { + Map map = createTestDataForNumericValueTypes(() -> randomFloat()); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + for(Entry e : map.entrySet()) { + assertEquals("For field " + e.getKey(), e.getValue().floatValue(), results.getFloat(e.getKey()), 0.0f); + assertEquals("For field " + e.getKey(), + e.getValue().floatValue(), results.getObject(e.getKey(), Float.class), 0.0f); + } + }); + } + + public void testGettingInvalidFloat() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_keyword").field("type", "keyword").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + + String randomString = randomUnicodeOfCodepointLengthBetween(128, 256); + long randomDate = randomLong(); + + index("test", "1", builder -> { + builder.field("test_keyword", randomString); + builder.field("test_date", randomDate); + }); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + + SQLException sqle = expectThrows(SQLException.class, () -> results.getFloat("test_keyword")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Float", randomString), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_keyword", Float.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [VARCHAR] to a Float", randomString), + sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getFloat("test_date")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Float", randomDate), + sqle.getMessage()); + sqle = expectThrows(SQLException.class, () -> results.getObject("test_date", Float.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Float", randomDate), + sqle.getMessage()); + }); + } + + public void testGettingBooleanValues() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_boolean").field("type", "boolean").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + long randomDate1 = randomLong(); + long randomDate2 = randomLong(); + + // true values + indexSimpleDocumentWithTrueValues(randomDate1); + + // false values + index("test", "2", builder -> { + builder.field("test_boolean", false); + builder.field("test_byte", 0); + builder.field("test_integer", 0); + builder.field("test_long", 0L); + builder.field("test_short", 0); + builder.field("test_double", 0d); + builder.field("test_float", 0f); + builder.field("test_keyword", "false"); + builder.field("test_date", randomDate2); + }); + + // other (non 0 = true) values + index("test", "3", builder -> { + builder.field("test_byte", randomValueOtherThan((byte) 0, () -> randomByte())); + builder.field("test_integer", randomValueOtherThan(0, () -> randomInt())); + builder.field("test_long", randomValueOtherThan(0L, () -> randomLong())); + builder.field("test_short", randomValueOtherThan((short) 0, () -> randomShort())); + builder.field("test_double", randomValueOtherThanMany(i -> i < 1.0d && i > -1.0d && i < Double.MAX_VALUE + && i > Double.MIN_VALUE, + () -> randomDouble() * randomInt())); + builder.field("test_float", randomValueOtherThanMany(i -> i < 1.0f && i > -1.0f && i < Float.MAX_VALUE && i > Float.MIN_VALUE, + () -> randomFloat() * randomInt())); + builder.field("test_keyword", "1"); + }); + + // other false values + index("test", "4", builder -> { + builder.field("test_keyword", "0"); + }); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + assertEquals(true, results.getBoolean("test_boolean")); + for(String fld : fieldsNames) { + assertEquals("Expected: but was: for field " + fld, true, results.getBoolean(fld)); + assertEquals("Expected: but was: for field " + fld, true, results.getObject(fld, Boolean.class)); + } + SQLException sqle = expectThrows(SQLException.class, () -> results.getBoolean("test_date")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Boolean", randomDate1), + sqle.getMessage()); + + results.next(); + assertEquals(false, results.getBoolean("test_boolean")); + for(String fld : fieldsNames) { + assertEquals("Expected: but was: for field " + fld, false, results.getBoolean(fld)); + assertEquals("Expected: but was: for field " + fld, false, results.getObject(fld, Boolean.class)); + } + sqle = expectThrows(SQLException.class, () -> results.getBoolean("test_date")); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Boolean", randomDate2), + sqle.getMessage()); + + sqle = expectThrows(SQLException.class, () -> results.getObject("test_date", Boolean.class)); + assertEquals(format(Locale.ROOT, "Unable to convert value [%.128s] of type [TIMESTAMP] to a Boolean", randomDate2), + sqle.getMessage()); + + results.next(); + for(String fld : fieldsNames.stream() + .filter((f) -> !f.equals("test_keyword")).collect(Collectors.toCollection(HashSet::new))) { + assertEquals("Expected: but was: for field " + fld, true, results.getBoolean(fld)); + assertEquals("Expected: but was: for field " + fld, true, results.getObject(fld, Boolean.class)); + } + + results.next(); + assertEquals(false, results.getBoolean("test_keyword")); + assertEquals(false, results.getObject("test_keyword", Boolean.class)); + }); + } + + public void testGettingDateWithoutCalendar() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_boolean").field("type", "boolean").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + Long randomLongDate = randomLong(); + indexSimpleDocumentWithTrueValues(randomLongDate); + + String timeZoneId = randomKnownTimeZone(); + Calendar connCalendar = Calendar.getInstance(TimeZone.getTimeZone(timeZoneId), Locale.ROOT); + + doWithQueryAndTimezone(SELECT_ALL_FIELDS, timeZoneId, (results) -> { + results.next(); + connCalendar.setTimeInMillis(randomLongDate); + connCalendar.set(HOUR_OF_DAY, 0); + connCalendar.set(MINUTE, 0); + connCalendar.set(SECOND, 0); + connCalendar.set(MILLISECOND, 0); + + assertEquals(results.getDate("test_date"), new java.sql.Date(connCalendar.getTimeInMillis())); + assertEquals(results.getDate(9), new java.sql.Date(connCalendar.getTimeInMillis())); + assertEquals(results.getObject("test_date", java.sql.Date.class), + new java.sql.Date(randomLongDate - (randomLongDate % 86400000L))); + assertEquals(results.getObject(9, java.sql.Date.class), + new java.sql.Date(randomLongDate - (randomLongDate % 86400000L))); + + // bulk validation for all fields which are not of type date + validateErrorsForDateTimeTestsWithoutCalendar(results::getDate); + }); + } + + public void testGettingDateWithCalendar() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_boolean").field("type", "boolean").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + Long randomLongDate = randomLong(); + indexSimpleDocumentWithTrueValues(randomLongDate); + index("test", "2", builder -> { + builder.timeField("test_date", null); + }); + + String timeZoneId = randomKnownTimeZone(); + String anotherTZId = randomValueOtherThan(timeZoneId, () -> randomKnownTimeZone()); + Calendar c = Calendar.getInstance(TimeZone.getTimeZone(anotherTZId), Locale.ROOT); + + doWithQueryAndTimezone(SELECT_ALL_FIELDS, timeZoneId, (results) -> { + results.next(); + c.setTimeInMillis(randomLongDate); + c.set(HOUR_OF_DAY, 0); + c.set(MINUTE, 0); + c.set(SECOND, 0); + c.set(MILLISECOND, 0); + + assertEquals(results.getDate("test_date", c), new java.sql.Date(c.getTimeInMillis())); + assertEquals(results.getDate(9, c), new java.sql.Date(c.getTimeInMillis())); + + // bulk validation for all fields which are not of type date + validateErrorsForDateTimeTestsWithCalendar(c, results::getDate); + + results.next(); + assertNull(results.getDate("test_date")); + }); + } + + public void testGettingTimeWithoutCalendar() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_boolean").field("type", "boolean").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + Long randomLongDate = randomLong(); + indexSimpleDocumentWithTrueValues(randomLongDate); + + String timeZoneId = randomKnownTimeZone(); + Calendar c = Calendar.getInstance(TimeZone.getTimeZone(timeZoneId), Locale.ROOT); + + doWithQueryAndTimezone(SELECT_ALL_FIELDS, timeZoneId, (results) -> { + results.next(); + c.setTimeInMillis(randomLongDate); + c.set(ERA, GregorianCalendar.AD); + c.set(YEAR, 1970); + c.set(MONTH, 0); + c.set(DAY_OF_MONTH, 1); + + assertEquals(results.getTime("test_date"), new java.sql.Time(c.getTimeInMillis())); + assertEquals(results.getTime(9), new java.sql.Time(c.getTimeInMillis())); + assertEquals(results.getObject("test_date", java.sql.Time.class), + new java.sql.Time(randomLongDate % 86400000L)); + assertEquals(results.getObject(9, java.sql.Time.class), + new java.sql.Time(randomLongDate % 86400000L)); + + validateErrorsForDateTimeTestsWithoutCalendar(results::getTime); + }); + } + + public void testGettingTimeWithCalendar() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_boolean").field("type", "boolean").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + Long randomLongDate = randomLong(); + indexSimpleDocumentWithTrueValues(randomLongDate); + index("test", "2", builder -> { + builder.timeField("test_date", null); + }); + + String timeZoneId = randomKnownTimeZone(); + String anotherTZId = randomValueOtherThan(timeZoneId, () -> randomKnownTimeZone()); + Calendar c = Calendar.getInstance(TimeZone.getTimeZone(anotherTZId), Locale.ROOT); + + doWithQueryAndTimezone(SELECT_ALL_FIELDS, timeZoneId, (results) -> { + results.next(); + c.setTimeInMillis(randomLongDate); + c.set(ERA, GregorianCalendar.AD); + c.set(YEAR, 1970); + c.set(MONTH, 0); + c.set(DAY_OF_MONTH, 1); + + assertEquals(results.getTime("test_date", c), new java.sql.Time(c.getTimeInMillis())); + assertEquals(results.getTime(9, c), new java.sql.Time(c.getTimeInMillis())); + + validateErrorsForDateTimeTestsWithCalendar(c, results::getTime); + + results.next(); + assertNull(results.getTime("test_date")); + }); + } + + public void testGettingTimestampWithoutCalendar() throws Exception { + createIndex("library"); + updateMapping("library", builder -> { + builder.startObject("release_date").field("type", "date").endObject(); + builder.startObject("republish_date").field("type", "date").endObject(); + }); + long randomMillis = randomLong(); index("library", "1", builder -> { builder.field("name", "Don Quixote"); builder.field("page_count", 1072); - builder.timeField("release_date", new Date(randomMillis)); + builder.field("release_date", randomMillis); builder.timeField("republish_date", null); }); index("library", "2", builder -> { builder.field("name", "1984"); builder.field("page_count", 328); - builder.timeField("release_date", new Date(-649036800000L)); - builder.timeField("republish_date", new Date(599616000000L)); + builder.field("release_date", -649036800000L); + builder.field("republish_date", 599616000000L); }); - try (Connection connection = esJdbc()) { - try (PreparedStatement statement = connection.prepareStatement("SELECT name, release_date, republish_date FROM library")) { - try (ResultSet results = statement.executeQuery()) { - ResultSetMetaData resultSetMetaData = results.getMetaData(); + doWithQuery("SELECT name, release_date, republish_date FROM library", (results) -> { + ResultSetMetaData resultSetMetaData = results.getMetaData(); - results.next(); - assertEquals(3, resultSetMetaData.getColumnCount()); - assertEquals(randomMillis, results.getTimestamp("release_date").getTime()); - assertEquals(randomMillis, results.getTimestamp(2).getTime()); - assertTrue(results.getObject(2) instanceof Timestamp); - assertEquals(randomMillis, ((Timestamp) results.getObject("release_date")).getTime()); - - assertNull(results.getTimestamp(3)); - assertNull(results.getObject("republish_date")); + results.next(); + assertEquals(3, resultSetMetaData.getColumnCount()); + assertEquals(randomMillis, results.getTimestamp("release_date").getTime()); + assertEquals(randomMillis, results.getTimestamp(2).getTime()); + assertTrue(results.getObject(2) instanceof Timestamp); + assertEquals(randomMillis, ((Timestamp) results.getObject("release_date")).getTime()); + + assertNull(results.getTimestamp(3)); + assertNull(results.getObject("republish_date")); - assertTrue(results.next()); - assertEquals(599616000000L, results.getTimestamp("republish_date").getTime()); - assertEquals(-649036800000L, ((Timestamp) results.getObject(2)).getTime()); + assertTrue(results.next()); + assertEquals(599616000000L, results.getTimestamp("republish_date").getTime()); + assertEquals(-649036800000L, ((Timestamp) results.getObject(2)).getTime()); - assertFalse(results.next()); - } - } - } + assertFalse(results.next()); + }); + } + + public void testGettingTimestampWithCalendar() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_boolean").field("type", "boolean").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + Long randomLongDate = randomLong(); + indexSimpleDocumentWithTrueValues(randomLongDate); + index("test", "2", builder -> { + builder.timeField("test_date", null); + }); + + String timeZoneId = randomKnownTimeZone(); + String anotherTZId = randomValueOtherThan(timeZoneId, () -> randomKnownTimeZone()); + Calendar c = Calendar.getInstance(TimeZone.getTimeZone(anotherTZId), Locale.ROOT); + + doWithQueryAndTimezone(SELECT_ALL_FIELDS, timeZoneId, (results) -> { + results.next(); + c.setTimeInMillis(randomLongDate); + + assertEquals(results.getTimestamp("test_date", c), new java.sql.Timestamp(c.getTimeInMillis())); + assertEquals(results.getTimestamp(9, c), new java.sql.Timestamp(c.getTimeInMillis())); + + validateErrorsForDateTimeTestsWithCalendar(c, results::getTimestamp); + + results.next(); + assertNull(results.getTimestamp("test_date")); + }); + } + + public void testValidGetObjectCalls() throws Exception { + createIndex("test"); + updateMappingForNumericValuesTests("test"); + updateMapping("test", builder -> { + builder.startObject("test_boolean").field("type", "boolean").endObject(); + builder.startObject("test_date").field("type", "date").endObject(); + }); + + byte b = randomByte(); + int i = randomInt(); + long l = randomLong(); + short s = (short) randomIntBetween(Short.MIN_VALUE, Short.MAX_VALUE); + double d = randomDouble(); + float f = randomFloat(); + boolean randomBool = randomBoolean(); + Long randomLongDate = randomLong(); + String randomString = randomUnicodeOfCodepointLengthBetween(128, 256); + + index("test", "1", builder -> { + builder.field("test_byte", b); + builder.field("test_integer", i); + builder.field("test_long", l); + builder.field("test_short", s); + builder.field("test_double", d); + builder.field("test_float", f); + builder.field("test_keyword", randomString); + builder.field("test_date", randomLongDate); + builder.field("test_boolean", randomBool); + }); + + doWithQuery(SELECT_WILDCARD, (results) -> { + results.next(); + + assertEquals(b, results.getObject("test_byte")); + assertTrue(results.getObject("test_byte") instanceof Byte); + + assertEquals(i, results.getObject("test_integer")); + assertTrue(results.getObject("test_integer") instanceof Integer); + + assertEquals(l, results.getObject("test_long")); + assertTrue(results.getObject("test_long") instanceof Long); + + assertEquals(s, results.getObject("test_short")); + assertTrue(results.getObject("test_short") instanceof Short); + + assertEquals(d, results.getObject("test_double")); + assertTrue(results.getObject("test_double") instanceof Double); + + assertEquals(f, results.getObject("test_float")); + assertTrue(results.getObject("test_float") instanceof Float); + + assertEquals(randomString, results.getObject("test_keyword")); + assertTrue(results.getObject("test_keyword") instanceof String); + + assertEquals(new Date(randomLongDate), results.getObject("test_date")); + assertTrue(results.getObject("test_date") instanceof Timestamp); + + assertEquals(randomBool, results.getObject("test_boolean")); + assertTrue(results.getObject("test_boolean") instanceof Boolean); + }); } /* @@ -79,4 +1091,458 @@ public class ResultSetTestCase extends JdbcIntegrationTestCase { fail("Infinite recursive call on getObject() method"); } } + + public void testUnsupportedGetMethods() throws IOException, SQLException { + index("test", "1", builder -> { + builder.field("test", "test"); + }); + Connection conn = esJdbc(); + PreparedStatement statement = conn.prepareStatement("SELECT * FROM test"); + ResultSet r = statement.executeQuery(); + + r.next(); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getAsciiStream("test"), "AsciiStream not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getAsciiStream(1), "AsciiStream not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getArray("test"), "Array not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getArray(1), "Array not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getBigDecimal("test"), "BigDecimal not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getBigDecimal("test"), "BigDecimal not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getBinaryStream("test"), "BinaryStream not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getBinaryStream(1), "BinaryStream not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getBlob("test"), "Blob not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getBlob(1), "Blob not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getCharacterStream("test"), "CharacterStream not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getCharacterStream(1), "CharacterStream not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getClob("test"), "Clob not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getClob(1), "Clob not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getNCharacterStream("test"), "NCharacterStream not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getNCharacterStream(1), "NCharacterStream not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getNClob("test"), "NClob not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getNClob(1), "NClob not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getNString("test"), "NString not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getNString(1), "NString not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getRef("test"), "Ref not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getRef(1), "Ref not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getRowId("test"), "RowId not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getRowId(1), "RowId not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getSQLXML("test"), "SQLXML not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getSQLXML(1), "SQLXML not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getURL("test"), "URL not supported"); + assertThrowsUnsupportedAndExpectErrorMessage(() -> r.getURL(1), "URL not supported"); + } + + public void testUnsupportedUpdateMethods() throws IOException, SQLException { + index("test", "1", builder -> { + builder.field("test", "test"); + }); + Connection conn = esJdbc(); + PreparedStatement statement = conn.prepareStatement("SELECT * FROM test"); + ResultSet r = statement.executeQuery(); + + r.next(); + Blob b = null; + InputStream i = null; + Clob c = null; + NClob nc = null; + Reader rd = null; + + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBytes(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBytes("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateArray(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateArray("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateAsciiStream(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateAsciiStream("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateAsciiStream(1, null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateAsciiStream(1, null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateAsciiStream("", null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateAsciiStream("", null, 1L)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBigDecimal(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBigDecimal("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBinaryStream(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBinaryStream("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBinaryStream(1, null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBinaryStream(1, null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBinaryStream("", null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBinaryStream("", null, 1L)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBlob(1, b)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBlob(1, i)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBlob("", b)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBlob("", i)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBlob(1, null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBlob("", null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBoolean(1, false)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateBoolean("", false)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateByte(1, (byte) 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateByte("", (byte) 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateCharacterStream(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateCharacterStream("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateCharacterStream(1, null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateCharacterStream(1, null, 1L)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateCharacterStream("", null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateCharacterStream("", null, 1L)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateClob(1, c)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateClob(1, rd)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateClob("", c)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateClob("", rd)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateClob(1, null, 1L)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateClob("", null, 1L)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateDate(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateDate("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateDouble(1, 0d)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateDouble("", 0d)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateFloat(1, 0f)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateFloat("", 0f)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateInt(1, 0)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateInt("", 0)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateLong(1, 0L)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateLong("", 0L)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNCharacterStream(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNCharacterStream("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNCharacterStream(1, null, 1L)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNCharacterStream("", null, 1L)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNClob(1, nc)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNClob(1, rd)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNClob("", nc)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNClob("", rd)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNClob(1, null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNClob("", null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNString(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNString("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNull(1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateNull("")); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateObject(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateObject("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateObject(1, null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateObject("", null, 1)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateRef(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateRef("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateRow()); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateRowId(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateRowId("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateSQLXML(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateSQLXML("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateShort(1, (short) 0)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateShort("", (short) 0)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateString(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateString("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateTime(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateTime("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateTimestamp(1, null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateTimestamp("", null)); + assertThrowsWritesUnsupportedForUpdate(() -> r.insertRow()); + assertThrowsWritesUnsupportedForUpdate(() -> r.updateRow()); + assertThrowsWritesUnsupportedForUpdate(() -> r.deleteRow()); + assertThrowsWritesUnsupportedForUpdate(() -> r.cancelRowUpdates()); + assertThrowsWritesUnsupportedForUpdate(() -> r.moveToInsertRow()); + assertThrowsWritesUnsupportedForUpdate(() -> r.refreshRow()); + assertThrowsWritesUnsupportedForUpdate(() -> r.moveToCurrentRow()); + assertThrowsWritesUnsupportedForUpdate(() -> r.rowUpdated()); + assertThrowsWritesUnsupportedForUpdate(() -> r.rowInserted()); + assertThrowsWritesUnsupportedForUpdate(() -> r.rowDeleted()); + } + + private void doWithQuery(String query, CheckedConsumer consumer) throws SQLException { + try (Connection connection = esJdbc()) { + try (PreparedStatement statement = connection.prepareStatement(query)) { + try (ResultSet results = statement.executeQuery()) { + consumer.accept(results); + } + } + } + } + + private void doWithQueryAndTimezone(String query, String tz, CheckedConsumer consumer) throws SQLException { + try (Connection connection = esJdbc(tz)) { + try (PreparedStatement statement = connection.prepareStatement(query)) { + try (ResultSet results = statement.executeQuery()) { + consumer.accept(results); + } + } + } + } + + private void createIndex(String index) throws Exception { + Request request = new Request("PUT", "/" + index); + XContentBuilder createIndex = JsonXContent.contentBuilder().startObject(); + createIndex.startObject("settings"); + { + createIndex.field("number_of_shards", 1); + createIndex.field("number_of_replicas", 1); + } + createIndex.endObject(); + createIndex.startObject("mappings"); + { + createIndex.startObject("doc"); + { + createIndex.startObject("properties"); + {} + createIndex.endObject(); + } + createIndex.endObject(); + } + createIndex.endObject().endObject(); + request.setJsonEntity(Strings.toString(createIndex)); + client().performRequest(request); + } + + private void updateMapping(String index, CheckedConsumer body) throws Exception { + Request request = new Request("PUT", "/" + index + "/_mapping/doc"); + XContentBuilder updateMapping = JsonXContent.contentBuilder().startObject(); + updateMapping.startObject("properties"); + { + body.accept(updateMapping); + } + updateMapping.endObject().endObject(); + + request.setJsonEntity(Strings.toString(updateMapping)); + client().performRequest(request); + } + + private void createTestDataForByteValueTests(byte random1, byte random2, byte random3) throws Exception, IOException { + createIndex("test"); + updateMapping("test", builder -> { + builder.startObject("test_byte").field("type", "byte").endObject(); + builder.startObject("test_null_byte").field("type", "byte").endObject(); + builder.startObject("test_keyword").field("type", "keyword").endObject(); + }); + + index("test", "1", builder -> { + builder.field("test_byte", random1); + builder.field("test_null_byte", (Byte) null); + }); + index("test", "2", builder -> { + builder.field("test_byte", random2); + builder.field("test_keyword", random3); + }); + } + + private void createTestDataForShortValueTests(short random1, short random2, short random3) throws Exception, IOException { + createIndex("test"); + updateMapping("test", builder -> { + builder.startObject("test_short").field("type", "short").endObject(); + builder.startObject("test_null_short").field("type", "short").endObject(); + builder.startObject("test_keyword").field("type", "keyword").endObject(); + }); + + index("test", "1", builder -> { + builder.field("test_short", random1); + builder.field("test_null_short", (Short) null); + }); + index("test", "2", builder -> { + builder.field("test_short", random2); + builder.field("test_keyword", random3); + }); + } + + private void createTestDataForIntegerValueTests(int random1, int random2, int random3) throws Exception, IOException { + createIndex("test"); + updateMapping("test", builder -> { + builder.startObject("test_integer").field("type", "integer").endObject(); + builder.startObject("test_null_integer").field("type", "integer").endObject(); + builder.startObject("test_keyword").field("type", "keyword").endObject(); + }); + + index("test", "1", builder -> { + builder.field("test_integer", random1); + builder.field("test_null_integer", (Integer) null); + }); + index("test", "2", builder -> { + builder.field("test_integer", random2); + builder.field("test_keyword", random3); + }); + } + + private void createTestDataForLongValueTests(long random1, long random2, long random3) throws Exception, IOException { + createIndex("test"); + updateMapping("test", builder -> { + builder.startObject("test_long").field("type", "long").endObject(); + builder.startObject("test_null_long").field("type", "long").endObject(); + builder.startObject("test_keyword").field("type", "keyword").endObject(); + }); + + index("test", "1", builder -> { + builder.field("test_long", random1); + builder.field("test_null_long", (Long) null); + }); + index("test", "2", builder -> { + builder.field("test_long", random2); + builder.field("test_keyword", random3); + }); + } + + private void createTestDataForDoubleValueTests(double random1, double random2, double random3) throws Exception, IOException { + createIndex("test"); + updateMapping("test", builder -> { + builder.startObject("test_double").field("type", "double").endObject(); + builder.startObject("test_null_double").field("type", "double").endObject(); + builder.startObject("test_keyword").field("type", "keyword").endObject(); + }); + + index("test", "1", builder -> { + builder.field("test_double", random1); + builder.field("test_null_double", (Double) null); + }); + index("test", "2", builder -> { + builder.field("test_double", random2); + builder.field("test_keyword", random3); + }); + } + + private void createTestDataForFloatValueTests(float random1, float random2, float random3) throws Exception, IOException { + createIndex("test"); + updateMapping("test", builder -> { + builder.startObject("test_float").field("type", "float").endObject(); + builder.startObject("test_null_float").field("type", "float").endObject(); + builder.startObject("test_keyword").field("type", "keyword").endObject(); + }); + + index("test", "1", builder -> { + builder.field("test_float", random1); + builder.field("test_null_float", (Double) null); + }); + index("test", "2", builder -> { + builder.field("test_float", random2); + builder.field("test_keyword", random3); + }); + } + + private void indexSimpleDocumentWithTrueValues(Long randomLongDate) throws IOException { + index("test", "1", builder -> { + builder.field("test_boolean", true); + builder.field("test_byte", 1); + builder.field("test_integer", 1); + builder.field("test_long", 1L); + builder.field("test_short", 1); + builder.field("test_double", 1d); + builder.field("test_float", 1f); + builder.field("test_keyword", "true"); + builder.field("test_date", randomLongDate); + }); + } + + /** + * Creates test data for all numeric get* methods. All values random and different from the other numeric fields already generated. + * It returns a map containing the field name and its randomly generated value to be later used in checking the returned values. + */ + private Map createTestDataForNumericValueTypes(Supplier randomGenerator) throws Exception, IOException { + Map map = new HashMap(); + createIndex("test"); + updateMappingForNumericValuesTests("test"); + + index("test", "1", builder -> { + // random Byte + byte test_byte = randomValueOtherThanMany(map::containsValue, randomGenerator).byteValue(); + builder.field("test_byte", test_byte); + map.put("test_byte", test_byte); + + // random Integer + int test_integer = randomValueOtherThanMany(map::containsValue, randomGenerator).intValue(); + builder.field("test_integer", test_integer); + map.put("test_integer", test_integer); + + // random Short + int test_short = randomValueOtherThanMany(map::containsValue, randomGenerator).shortValue(); + builder.field("test_short", test_short); + map.put("test_short", test_short); + + // random Long + long test_long = randomValueOtherThanMany(map::containsValue, randomGenerator).longValue(); + builder.field("test_long", test_long); + map.put("test_long", test_long); + + // random Double + double test_double = randomValueOtherThanMany(map::containsValue, randomGenerator).doubleValue(); + builder.field("test_double", test_double); + map.put("test_double", test_double); + + // random Float + float test_float = randomValueOtherThanMany(map::containsValue, randomGenerator).floatValue(); + builder.field("test_float", test_float); + map.put("test_float", test_float); + }); + return map; + } + + private void updateMappingForNumericValuesTests(String indexName) throws Exception { + updateMapping(indexName, builder -> { + for(String field : fieldsNames) { + builder.startObject(field).field("type", field.substring(5)).endObject(); + } + }); + } + + private void assertThrowsUnsupportedAndExpectErrorMessage(ThrowingRunnable runnable, String message) { + SQLException sqle = expectThrows(SQLFeatureNotSupportedException.class, runnable); + assertEquals(message, sqle.getMessage()); + } + + private void assertThrowsWritesUnsupportedForUpdate(ThrowingRunnable r) { + assertThrowsUnsupportedAndExpectErrorMessage(r, "Writes not supported"); + } + + private void validateErrorsForDateTimeTestsWithoutCalendar(CheckedFunction method) { + SQLException sqle; + for(Entry,JDBCType> field : dateTimeTestingFields.entrySet()) { + sqle = expectThrows(SQLException.class, () -> method.apply(field.getKey().v1())); + assertEquals( + format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Long", + field.getKey().v2(), field.getValue()), sqle.getMessage()); + } + } + + private void validateErrorsForDateTimeTestsWithCalendar(Calendar c, CheckedBiFunction method) { + SQLException sqle; + for(Entry,JDBCType> field : dateTimeTestingFields.entrySet()) { + sqle = expectThrows(SQLException.class, () -> method.apply(field.getKey().v1(), c)); + assertEquals( + format(Locale.ROOT, "Unable to convert value [%.128s] of type [%s] to a Long", + field.getKey().v2(), field.getValue()), sqle.getMessage()); + } + } + + private float randomFloatBetween(float start, float end) { + float result = 0.0f; + while (result < start || result > end || Float.isNaN(result)) { + result = start + randomFloat() * (end - start); + } + + return result; + } + + private Long getMaxIntPlusOne() { + return Long.valueOf(Integer.MAX_VALUE) + 1L; + } + + private Double getMaxLongPlusOne() { + return Double.valueOf(Long.MAX_VALUE) + 1d; + } + + private Connection esJdbc(String timeZoneId) throws SQLException { + return randomBoolean() ? useDriverManager(timeZoneId) : useDataSource(timeZoneId); + } + + private Connection useDriverManager(String timeZoneId) throws SQLException { + String elasticsearchAddress = getProtocol() + "://" + elasticsearchAddress(); + String address = "jdbc:es://" + elasticsearchAddress; + Properties connectionProperties = connectionProperties(); + connectionProperties.put(JdbcConfiguration.TIME_ZONE, timeZoneId); + Connection connection = DriverManager.getConnection(address, connectionProperties); + + assertNotNull("The timezone should be specified", connectionProperties.getProperty(JdbcConfiguration.TIME_ZONE)); + return connection; + } + + private Connection useDataSource(String timeZoneId) throws SQLException { + String elasticsearchAddress = getProtocol() + "://" + elasticsearchAddress(); + JdbcDataSource dataSource = new JdbcDataSource(); + String address = "jdbc:es://" + elasticsearchAddress; + dataSource.setUrl(address); + Properties connectionProperties = connectionProperties(); + connectionProperties.put(JdbcConfiguration.TIME_ZONE, timeZoneId); + dataSource.setProperties(connectionProperties); + Connection connection = dataSource.getConnection(); + + assertNotNull("The timezone should be specified", connectionProperties.getProperty(JdbcConfiguration.TIME_ZONE)); + return connection; + } } diff --git a/x-pack/qa/sql/src/main/java/org/elasticsearch/xpack/qa/sql/jdbc/SimpleExampleTestCase.java b/x-pack/qa/sql/src/main/java/org/elasticsearch/xpack/qa/sql/jdbc/SimpleExampleTestCase.java index 7621743481a..f5d559d9bf0 100644 --- a/x-pack/qa/sql/src/main/java/org/elasticsearch/xpack/qa/sql/jdbc/SimpleExampleTestCase.java +++ b/x-pack/qa/sql/src/main/java/org/elasticsearch/xpack/qa/sql/jdbc/SimpleExampleTestCase.java @@ -25,7 +25,8 @@ public class SimpleExampleTestCase extends JdbcIntegrationTestCase { assertEquals("Don Quixote", results.getString(1)); assertEquals(1072, results.getInt(2)); SQLException e = expectThrows(SQLException.class, () -> results.getInt(1)); - assertTrue(e.getMessage(), e.getMessage().contains("unable to convert column 1 to an int")); + assertTrue(e.getMessage(), + e.getMessage().contains("Unable to convert value [Don Quixote] of type [VARCHAR] to an Integer")); assertFalse(results.next()); } // end::simple_example From a84a20844be0bbb39c7e063e33691f8d3e0a8380 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Fri, 31 Aug 2018 16:36:57 +0300 Subject: [PATCH 16/16] Lazy evaluate java9home (#33301) --- .../groovy/org/elasticsearch/gradle/BuildPlugin.groovy | 2 +- .../gradle/precommit/ForbiddenApisCliTask.java | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy index 6a9d4076eef..75b5676cc34 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy @@ -211,7 +211,7 @@ class BuildPlugin implements Plugin { project.rootProject.ext.minimumRuntimeVersion = minimumRuntimeVersion project.rootProject.ext.inFipsJvm = inFipsJvm project.rootProject.ext.gradleJavaVersion = JavaVersion.toVersion(gradleJavaVersion) - project.rootProject.ext.java9Home = findJavaHome("9") + project.rootProject.ext.java9Home = "${-> findJavaHome("9")}" } project.targetCompatibility = project.rootProject.ext.minimumRuntimeVersion diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/ForbiddenApisCliTask.java b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/ForbiddenApisCliTask.java index 46e5d84a2f2..aaa9564b0dc 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/ForbiddenApisCliTask.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/ForbiddenApisCliTask.java @@ -51,7 +51,8 @@ public class ForbiddenApisCliTask extends DefaultTask { private JavaVersion targetCompatibility; private FileCollection classesDirs; private SourceSet sourceSet; - private String javaHome; + // This needs to be an object so it can hold Groovy GStrings + private Object javaHome; @Input public JavaVersion getTargetCompatibility() { @@ -142,11 +143,11 @@ public class ForbiddenApisCliTask extends DefaultTask { } @Input - public String getJavaHome() { + public Object getJavaHome() { return javaHome; } - public void setJavaHome(String javaHome) { + public void setJavaHome(Object javaHome) { this.javaHome = javaHome; }