From f1a6f2a61fe5d01fc90f6233c1b80d9009749750 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 4 Jun 2018 18:07:36 -0400 Subject: [PATCH 01/10] Always enable soft-deletes when opening IW --- server/src/main/java/org/elasticsearch/index/store/Store.java | 1 + 1 file changed, 1 insertion(+) 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 d9f151907dfbb..d281d88aabf56 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -1634,6 +1634,7 @@ private static IndexWriter newIndexWriter(final IndexWriterConfig.OpenMode openM 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_DELETE_FIELD) .setCommitOnClose(false) .setIndexCommit(commit) // we don't want merges to happen here - we call maybe merge on the engine From a9979bc8c3c7aebe6741dce0974bd5fb003a5aef Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 4 Jun 2018 18:09:06 -0400 Subject: [PATCH 02/10] Load commit stats directly from SegmentInfos --- .../main/java/org/elasticsearch/index/engine/CommitStats.java | 4 ++-- .../src/main/java/org/elasticsearch/index/engine/Engine.java | 4 +--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/CommitStats.java b/server/src/main/java/org/elasticsearch/index/engine/CommitStats.java index 8fbbe3a9deaa9..21025046b8c57 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/CommitStats.java +++ b/server/src/main/java/org/elasticsearch/index/engine/CommitStats.java @@ -39,13 +39,13 @@ public final class CommitStats implements Streamable, ToXContentFragment { private String id; // lucene commit id in base 64; private int numDocs; - public CommitStats(SegmentInfos segmentInfos, int numDocs) { + public CommitStats(SegmentInfos segmentInfos) { // clone the map to protect against concurrent changes userData = MapBuilder.newMapBuilder().putAll(segmentInfos.getUserData()).immutableMap(); // lucene calls the current generation, last generation. generation = segmentInfos.getLastGeneration(); id = Base64.getEncoder().encodeToString(segmentInfos.getId()); - this.numDocs = numDocs; + numDocs = Lucene.getNumDocs(segmentInfos); } private CommitStats() { 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 d0aaab1dbc5a0..874f03ae434de 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -632,9 +632,7 @@ protected final void ensureOpen() { /** get commits stats for the last commit */ public CommitStats commitStats() { - try (Engine.Searcher searcher = acquireSearcher("commit_stats", Engine.SearcherScope.INTERNAL)) { - return new CommitStats(getLastCommittedSegmentInfos(), searcher.reader().numDocs()); - } + return new CommitStats(getLastCommittedSegmentInfos()); } /** From f56a6baa348dd26bc2c3f5795ed9226cae8975c5 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 4 Jun 2018 18:09:29 -0400 Subject: [PATCH 03/10] Use getSoftDelCount This reverts commit b12c2f61c5baeb7ba200748834ff69beec5351f5. --- .../elasticsearch/common/lucene/Lucene.java | 14 +----- .../org/elasticsearch/index/store/Store.java | 12 +----- .../indices/flush/SyncedFlushService.java | 17 +++----- .../recovery/PeerRecoveryTargetService.java | 4 +- .../index/store/CorruptedFileIT.java | 2 - .../PeerRecoveryTargetServiceTests.java | 32 -------------- .../test/InternalTestCluster.java | 43 +++++-------------- 7 files changed, 22 insertions(+), 102 deletions(-) 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 25138a2909606..de1575f2f8f69 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -44,7 +44,6 @@ import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SegmentReader; -import org.apache.lucene.index.SoftDeletesDirectoryReaderWrapper; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.FieldDoc; @@ -145,21 +144,11 @@ public static Iterable files(SegmentInfos infos) throws IOException { 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; } - /** - * Unlike {@link #getNumDocs(SegmentInfos)} this method returns a numDocs that always excludes soft-deleted docs. - * This method is expensive thus prefer using {@link #getNumDocs(SegmentInfos)} unless an exact numDocs is required. - */ - public static int getExactNumDocs(IndexCommit commit) throws IOException { - try (DirectoryReader reader = DirectoryReader.open(commit)) { - return new SoftDeletesDirectoryReaderWrapper(reader, Lucene.SOFT_DELETE_FIELD).numDocs(); - } - } - /** * Reads the segments infos from the given commit, failing if it fails to load */ @@ -235,6 +224,7 @@ public static void cleanLuceneIndex(Directory directory) throws IOException { } } try (IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(Lucene.STANDARD_ANALYZER) + .setSoftDeletesField(Lucene.SOFT_DELETE_FIELD) .setMergePolicy(NoMergePolicy.INSTANCE) // no merges .setCommitOnClose(false) // no commits .setOpenMode(IndexWriterConfig.OpenMode.CREATE))) // force creation - don't append... 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 d281d88aabf56..9909fa4ba0302 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -862,7 +862,7 @@ static LoadedMetadata loadMetadata(IndexCommit commit, Directory directory, Logg Map commitUserDataBuilder = new HashMap<>(); try { final SegmentInfos segmentCommitInfos = Store.readSegmentsInfo(commit, directory); - numDocs = Lucene.getExactNumDocs(commit != null ? commit : findIndexCommit(directory, segmentCommitInfos)); + numDocs = Lucene.getNumDocs(segmentCommitInfos); commitUserDataBuilder.putAll(segmentCommitInfos.getUserData()); Version maxVersion = segmentCommitInfos.getMinSegmentLuceneVersion(); // we don't know which version was used to write so we take the max version. for (SegmentCommitInfo info : segmentCommitInfos) { @@ -945,16 +945,6 @@ public static void hashFile(BytesRefBuilder fileHash, InputStream in, long size) assert fileHash.length() == len : Integer.toString(fileHash.length()) + " != " + Integer.toString(len); } - private static IndexCommit findIndexCommit(Directory directory, SegmentInfos sis) throws IOException { - List commits = DirectoryReader.listCommits(directory); - for (IndexCommit commit : commits) { - if (commit.getSegmentsFileName().equals(sis.getSegmentsFileName())) { - return commit; - } - } - throw new IOException("Index commit [" + sis.getSegmentsFileName() + "] is not found"); - } - @Override public Iterator iterator() { return metadata.values().iterator(); diff --git a/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java b/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java index dfe804cb1442b..52e0ac8ab860f 100644 --- a/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java +++ b/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java @@ -19,7 +19,6 @@ package org.elasticsearch.indices.flush; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.index.SegmentInfos; import org.elasticsearch.Assertions; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; @@ -42,13 +41,13 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.engine.CommitStats; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; @@ -468,19 +467,15 @@ public String executor() { } } - private PreSyncedFlushResponse performPreSyncedFlush(PreShardSyncedFlushRequest request) throws IOException { + private PreSyncedFlushResponse performPreSyncedFlush(PreShardSyncedFlushRequest request) { IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id()); FlushRequest flushRequest = new FlushRequest().force(false).waitIfOngoing(true); logger.trace("{} performing pre sync flush", request.shardId()); indexShard.flush(flushRequest); - try (Engine.IndexCommitRef commitRef = indexShard.acquireLastIndexCommit(false)) { - final SegmentInfos segmentInfos = Lucene.readSegmentInfos(commitRef.getIndexCommit()); - final int numDocs = Lucene.getExactNumDocs(commitRef.getIndexCommit()); - final Engine.CommitId commitId = new Engine.CommitId(segmentInfos.getId()); - final String syncId = segmentInfos.userData.get(Engine.SYNC_COMMIT_ID); - logger.trace("{} pre sync flush done. commit id {}, num docs {}", request.shardId(), commitId, numDocs); - return new PreSyncedFlushResponse(commitId, numDocs, syncId); - } + final CommitStats commitStats = indexShard.commitStats(); + final Engine.CommitId commitId = commitStats.getRawCommitId(); + logger.trace("{} pre sync flush done. commit id {}, num docs {}", request.shardId(), commitId, commitStats.getNumDocs()); + return new PreSyncedFlushResponse(commitId, commitStats.getNumDocs(), commitStats.syncId()); } private ShardSyncedFlushResponse performSyncedFlush(ShardSyncedFlushRequest request) { diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 3e09312bec86f..cb49eed25f8fe 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -289,7 +289,7 @@ public RecoveryResponse newInstance() { * @param recoveryTarget the target of the recovery * @return a snapshot of the store metadata */ - static Store.MetadataSnapshot getStoreMetadataSnapshot(final Logger logger, final RecoveryTarget recoveryTarget) { + private Store.MetadataSnapshot getStoreMetadataSnapshot(final RecoveryTarget recoveryTarget) { try { return recoveryTarget.indexShard().snapshotStoreMetadata(); } catch (final org.apache.lucene.index.IndexNotFoundException e) { @@ -312,7 +312,7 @@ private StartRecoveryRequest getStartRecoveryRequest(final RecoveryTarget recove final StartRecoveryRequest request; logger.trace("{} collecting local files for [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); - final Store.MetadataSnapshot metadataSnapshot = getStoreMetadataSnapshot(logger, recoveryTarget); + final Store.MetadataSnapshot metadataSnapshot = getStoreMetadataSnapshot(recoveryTarget); logger.trace("{} local file count [{}]", recoveryTarget.shardId(), metadataSnapshot.size()); final long startingSeqNo; diff --git a/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java b/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java index c4292410d0ac5..d3119bbc0fdc4 100644 --- a/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java +++ b/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java @@ -23,7 +23,6 @@ import org.apache.lucene.index.CheckIndex; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; @@ -108,7 +107,6 @@ import static org.hamcrest.Matchers.notNullValue; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE) -@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/30228") // What if DV is corrupted? public class CorruptedFileIT extends ESIntegTestCase { @Override 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 9c4c1c1e736fd..3b50fa649150c 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -24,7 +24,6 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.NoMergePolicy; -import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.common.UUIDs; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -32,10 +31,8 @@ import org.elasticsearch.index.translog.Translog; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import static org.hamcrest.Matchers.equalTo; @@ -111,33 +108,4 @@ public void testGetStartingSeqNo() throws Exception { closeShards(replica); } } - - public void testExactNumDocsInStoreMetadataSnapshot() throws Exception { - final IndexShard replica = newShard(false); - recoveryEmptyReplica(replica); - long flushedDocs = 0; - final int numDocs = scaledRandomIntBetween(1, 20); - final Set docIds = new HashSet<>(); - for (int i = 0; i < numDocs; i++) { - String id = Integer.toString(i); - docIds.add(id); - indexDoc(replica, "_doc", id); - if (randomBoolean()) { - replica.flush(new FlushRequest()); - flushedDocs = docIds.size(); - } - } - for (String id : randomSubsetOf(docIds)) { - deleteDoc(replica, "_doc", id); - docIds.remove(id); - if (randomBoolean()) { - replica.flush(new FlushRequest()); - flushedDocs = docIds.size(); - } - } - final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null, null); - assertThat(PeerRecoveryTargetService.getStoreMetadataSnapshot(logger, recoveryTarget).getNumDocs(), equalTo(flushedDocs)); - recoveryTarget.decRef(); - closeShards(replica); - } } 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 b0eef695b7ff1..d39a0ad2aedaf 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -26,8 +26,6 @@ import com.carrotsearch.randomizedtesting.generators.RandomStrings; import org.apache.logging.log4j.Logger; import org.apache.lucene.store.AlreadyClosedException; -import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; @@ -77,10 +75,7 @@ import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.CommitStats; import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.engine.EngineTestCase; -import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; @@ -1105,7 +1100,7 @@ public void beforeIndexDeletion() throws Exception { // ElasticsearchIntegrationTest must override beforeIndexDeletion() to avoid failures. assertNoPendingIndexOperations(); //check that shards that have same sync id also contain same number of documents - assertSameSyncIdSameDocs(); + assertSameSyncIdSameDocs(); assertOpenTranslogReferences(); } @@ -1116,16 +1111,16 @@ private void assertSameSyncIdSameDocs() { IndicesService indexServices = getInstance(IndicesService.class, nodeAndClient.name); for (IndexService indexService : indexServices) { for (IndexShard indexShard : indexService) { - Tuple commitStats = commitStats(indexShard); - if (commitStats != null) { - String syncId = commitStats.v1(); - long liveDocsOnShard = commitStats.v2(); - if (docsOnShards.get(syncId) != null) { - assertThat("sync id is equal but number of docs does not match on node " + nodeAndClient.name + - ". expected " + docsOnShards.get(syncId) + " but got " + liveDocsOnShard, docsOnShards.get(syncId), - equalTo(liveDocsOnShard)); - } else { - docsOnShards.put(syncId, liveDocsOnShard); + CommitStats commitStats = indexShard.commitStats(); + if (commitStats != null) { // null if the engine is closed or if the shard is recovering + String syncId = commitStats.getUserData().get(Engine.SYNC_COMMIT_ID); + if (syncId != null) { + long liveDocsOnShard = commitStats.getNumDocs(); + if (docsOnShards.get(syncId) != null) { + assertThat("sync id is equal but number of docs does not match on node " + nodeAndClient.name + ". expected " + docsOnShards.get(syncId) + " but got " + liveDocsOnShard, docsOnShards.get(syncId), equalTo(liveDocsOnShard)); + } else { + docsOnShards.put(syncId, liveDocsOnShard); + } } } } @@ -1133,22 +1128,6 @@ private void assertSameSyncIdSameDocs() { } } - private Tuple commitStats(IndexShard indexShard) { - try (Engine.IndexCommitRef commitRef = indexShard.acquireLastIndexCommit(false)) { - final String syncId = commitRef.getIndexCommit().getUserData().get(Engine.SYNC_COMMIT_ID); - // Only read if sync_id exists - if (Strings.hasText(syncId)) { - return Tuple.tuple(syncId, Lucene.getExactNumDocs(commitRef.getIndexCommit())); - } else { - return null; - } - } catch (IllegalIndexShardStateException ex) { - return null; // Shard is closed or not started yet. - } catch (IOException ex) { - throw new AssertionError(ex); - } - } - private void assertNoPendingIndexOperations() throws Exception { assertBusy(() -> { final Collection nodesAndClients = nodes.values(); From 02355b5af5f1d1969553e9ca8004fa44b9ccc9dc Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 4 Jun 2018 19:19:00 -0400 Subject: [PATCH 04/10] Configure soft-deletes field --- .../main/java/org/elasticsearch/index/shard/StoreRecovery.java | 1 + .../elasticsearch/index/translog/TruncateTranslogCommand.java | 2 ++ 2 files changed, 3 insertions(+) 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 54718c545a44e..0a03e8601b42d 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 @@ void addIndices(final RecoveryState.Index indexRecoveryStats, final Directory ta final Directory hardLinkOrCopyTarget = new org.apache.lucene.store.HardlinkCopyDirectoryWrapper(target); IndexWriterConfig iwc = new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETE_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/translog/TruncateTranslogCommand.java b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java index b8bd93e05a6f8..56a084196131f 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java @@ -33,6 +33,7 @@ import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.store.NativeFSLockFactory; import org.apache.lucene.store.OutputStreamDataOutput; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cli.EnvironmentAwareCommand; @@ -179,6 +180,7 @@ protected void execute(Terminal terminal, OptionSet options, Environment env) th terminal.println("Marking index with the new history uuid"); // commit the new histroy id IndexWriterConfig iwc = new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETE_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 From 956bf1b37221baaabe5c19c43c5ecae0f9160121 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 4 Jun 2018 19:39:16 -0400 Subject: [PATCH 05/10] Mute MatchPhrasePrefixQueryBuilderTests --- .../index/query/MatchPhrasePrefixQueryBuilderTests.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/test/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilderTests.java index 8b706e552bcbf..1a9caeebd965b 100644 --- a/server/src/test/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilderTests.java @@ -25,6 +25,7 @@ import org.apache.lucene.search.PointRangeQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.lucene.search.MultiPhrasePrefixQuery; import org.elasticsearch.search.internal.SearchContext; @@ -39,6 +40,7 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.notNullValue; +@LuceneTestCase.AwaitsFix(bugUrl = "should-be-fixed-by-upstream") public class MatchPhrasePrefixQueryBuilderTests extends AbstractQueryTestCase { @Override protected MatchPhrasePrefixQueryBuilder doCreateTestQueryBuilder() { From c31c44672a6fe6a29a7893895e08cfa3cac26875 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 4 Jun 2018 19:56:07 -0400 Subject: [PATCH 06/10] Soft-deletes field when restore from snapshot --- .../repositories/blobstore/BlobStoreRepository.java | 1 + 1 file changed, 1 insertion(+) 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 cc5cfcccf3beb..86538d1553756 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1436,6 +1436,7 @@ public void restore() throws IOException { // 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_DELETE_FIELD) .setOpenMode(IndexWriterConfig.OpenMode.CREATE) .setCommitOnClose(true)); writer.close(); From c5a57cbe0cfc9917bccfbf5cdd47976fcc7b94e7 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 4 Jun 2018 20:05:52 -0400 Subject: [PATCH 07/10] Configure soft-deletes PeerRecoveryTargetServiceTests --- .../indices/recovery/PeerRecoveryTargetServiceTests.java | 2 ++ 1 file changed, 2 insertions(+) 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 3b50fa649150c..7b1003a862481 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.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 void testGetStartingSeqNo() throws Exception { replica.close("test", false); final List commits = DirectoryReader.listCommits(replica.store().directory()); IndexWriterConfig iwc = new IndexWriterConfig(null) + .setSoftDeletesField(Lucene.SOFT_DELETE_FIELD) .setCommitOnClose(false) .setMergePolicy(NoMergePolicy.INSTANCE) .setOpenMode(IndexWriterConfig.OpenMode.APPEND); From 877b4f99f5dfb971fa485f27a2442f99450222be Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 4 Jun 2018 20:19:20 -0400 Subject: [PATCH 08/10] when prune commit files --- server/src/main/java/org/elasticsearch/common/lucene/Lucene.java | 1 + 1 file changed, 1 insertion(+) 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 de1575f2f8f69..5fe10d8fc684f 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -201,6 +201,7 @@ public static SegmentInfos pruneUnreferencedFiles(String segmentsFileName, Direc } final CommitPoint cp = new CommitPoint(si, directory); try (IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(Lucene.STANDARD_ANALYZER) + .setSoftDeletesField(Lucene.SOFT_DELETE_FIELD) .setIndexCommit(cp) .setCommitOnClose(false) .setMergePolicy(NoMergePolicy.INSTANCE) From fcf484ca7b1bd4b76cfa6bd06d4aa3b560496ff7 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 5 Jun 2018 08:59:52 -0400 Subject: [PATCH 09/10] Unmute MatchPhrasePrefixQueryBuilderTests --- .../index/query/MatchPhrasePrefixQueryBuilderTests.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilderTests.java index 21a3a17f620b9..e5da5d7f97146 100644 --- a/server/src/test/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilderTests.java @@ -25,7 +25,6 @@ import org.apache.lucene.search.PointRangeQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.lucene.search.MultiPhrasePrefixQuery; import org.elasticsearch.search.internal.SearchContext; @@ -40,7 +39,6 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.notNullValue; -@LuceneTestCase.AwaitsFix(bugUrl = "should-be-fixed-by-upstream") public class MatchPhrasePrefixQueryBuilderTests extends AbstractQueryTestCase { @Override protected MatchPhrasePrefixQueryBuilder doCreateTestQueryBuilder() { From 206fb221144c7fc4b5200190c3e3327e3f5399ff Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 4 Jun 2018 14:18:46 -0400 Subject: [PATCH 10/10] Upgrade to Lucene-7.4.0-snapshot-0a7c3f462f (#31073) This snapshot includes: - LUCENE-8341: Record soft deletes in SegmentCommitInfo which will resolve #30851 - LUCENE-8335: Enforce soft-deletes field up-front --- buildSrc/version.properties | 2 +- .../lucene-expressions-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../lucene-expressions-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../lucene-analyzers-icu-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../lucene-analyzers-icu-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - ...ne-analyzers-kuromoji-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + ...ne-analyzers-kuromoji-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - ...lucene-analyzers-nori-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + ...lucene-analyzers-nori-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - ...ne-analyzers-phonetic-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + ...ne-analyzers-phonetic-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - ...ene-analyzers-smartcn-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + ...ene-analyzers-smartcn-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - ...ene-analyzers-stempel-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + ...ene-analyzers-stempel-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - ...-analyzers-morfologik-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + ...-analyzers-morfologik-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - ...cene-analyzers-common-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + ...cene-analyzers-common-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - ...ucene-backward-codecs-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + ...ucene-backward-codecs-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../licenses/lucene-core-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../licenses/lucene-core-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../lucene-grouping-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../lucene-grouping-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../lucene-highlighter-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../lucene-highlighter-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../licenses/lucene-join-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../licenses/lucene-join-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../lucene-memory-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../lucene-memory-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../licenses/lucene-misc-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../licenses/lucene-misc-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../lucene-queries-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../lucene-queries-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../lucene-queryparser-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../lucene-queryparser-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../lucene-sandbox-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../lucene-sandbox-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../lucene-spatial-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../lucene-spatial-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - ...lucene-spatial-extras-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + ...lucene-spatial-extras-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../lucene-spatial3d-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../lucene-spatial3d-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../lucene-suggest-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../lucene-suggest-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - .../org/elasticsearch/index/engine/TranslogLeafReader.java | 6 +++--- .../licenses/lucene-core-7.4.0-snapshot-0a7c3f462f.jar.sha1 | 1 + .../licenses/lucene-core-7.4.0-snapshot-1cbadda4d3.jar.sha1 | 1 - 50 files changed, 28 insertions(+), 28 deletions(-) create mode 100644 modules/lang-expression/licenses/lucene-expressions-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 modules/lang-expression/licenses/lucene-expressions-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 plugins/analysis-icu/licenses/lucene-analyzers-icu-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 plugins/analysis-icu/licenses/lucene-analyzers-icu-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 plugins/analysis-nori/licenses/lucene-analyzers-nori-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 plugins/analysis-nori/licenses/lucene-analyzers-nori-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 plugins/analysis-stempel/licenses/lucene-analyzers-stempel-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 plugins/analysis-stempel/licenses/lucene-analyzers-stempel-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-analyzers-common-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-analyzers-common-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-backward-codecs-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-backward-codecs-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-core-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-core-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-grouping-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-grouping-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-highlighter-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-highlighter-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-join-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-join-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-memory-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-memory-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-misc-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-misc-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-queries-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-queries-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-queryparser-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-queryparser-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-sandbox-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-sandbox-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-spatial-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-spatial-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-spatial-extras-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-spatial-extras-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-spatial3d-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-spatial3d-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 server/licenses/lucene-suggest-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 server/licenses/lucene-suggest-7.4.0-snapshot-1cbadda4d3.jar.sha1 create mode 100644 x-pack/plugin/sql/sql-proto/licenses/lucene-core-7.4.0-snapshot-0a7c3f462f.jar.sha1 delete mode 100644 x-pack/plugin/sql/sql-proto/licenses/lucene-core-7.4.0-snapshot-1cbadda4d3.jar.sha1 diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 6009021da14ed..7aedd395b93b5 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -1,5 +1,5 @@ elasticsearch = 7.0.0-alpha1 -lucene = 7.4.0-snapshot-1cbadda4d3 +lucene = 7.4.0-snapshot-0a7c3f462f # optional dependencies spatial4j = 0.7 diff --git a/modules/lang-expression/licenses/lucene-expressions-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..25e2291d36e8b --- /dev/null +++ b/modules/lang-expression/licenses/lucene-expressions-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +bf2cfa0551ebdf08a2cf3079f3c74643bd9dbb76 \ No newline at end of file diff --git a/modules/lang-expression/licenses/lucene-expressions-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 3bbaa2ba0a715..0000000000000 --- a/modules/lang-expression/licenses/lucene-expressions-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -98c920972b2f5e8563540e805d87e6a3bc888972 \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analyzers-icu-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analyzers-icu-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..3fdd3366122cb --- /dev/null +++ b/plugins/analysis-icu/licenses/lucene-analyzers-icu-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +82d83fcac1d9c8948aa0247fc9c87f177ddbd59b \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analyzers-icu-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analyzers-icu-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 7f3d3b5ccf63c..0000000000000 --- a/plugins/analysis-icu/licenses/lucene-analyzers-icu-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -844e2b76f4bc6e646e1c3257d668ac598e03f36a \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..855d6ebe4aeb0 --- /dev/null +++ b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +73fd4364f2931e7c8303b5927e140a7d21116c36 \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 65423fff2a441..0000000000000 --- a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2f2bd2d67c7952e4ae14ab3f742824a45d0d1719 \ No newline at end of file diff --git a/plugins/analysis-nori/licenses/lucene-analyzers-nori-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/plugins/analysis-nori/licenses/lucene-analyzers-nori-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..091097f1a8477 --- /dev/null +++ b/plugins/analysis-nori/licenses/lucene-analyzers-nori-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +0a2c4417fa9a8be078864f590a5a66b98d551cf5 \ No newline at end of file diff --git a/plugins/analysis-nori/licenses/lucene-analyzers-nori-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/plugins/analysis-nori/licenses/lucene-analyzers-nori-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 04fa62ce64a1d..0000000000000 --- a/plugins/analysis-nori/licenses/lucene-analyzers-nori-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -46ad7ebcfcdbdb60dd54aae4d720356a7a51c7c0 \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..b18addf0b5819 --- /dev/null +++ b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +6fa179924f139a30fc0e5399256e1a44562ed32b \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 55bc8869196e0..0000000000000 --- a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -548e9f2b4d4a985dc174b2eee4007c0bd5642e68 \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..7b7141b6f407c --- /dev/null +++ b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +5ed135d34d7868b71a725257a46dc8d8735a15d4 \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index be66854321699..0000000000000 --- a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b90e66f4104f0234cfef335762f65a6fed695231 \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..73be96c477eab --- /dev/null +++ b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +875911b36b99c2103719f94559878a0ecb862fb6 \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index b77acdc34f31c..0000000000000 --- a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -929a4eb52b11f6d3f0df9c8eba014f5ee2464c67 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..0c85d3f6c8522 --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +e7191628df8cb72382a20da79224aef677117849 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index cce4b6ff18df5..0000000000000 --- a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0e6575a411b65cd95e0e54f04d3da278b68be521 \ No newline at end of file diff --git a/server/licenses/lucene-analyzers-common-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-analyzers-common-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..db3885eb62fab --- /dev/null +++ b/server/licenses/lucene-analyzers-common-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +8cd761f40c4a89ed977167f0518d12e409eaf3d8 \ No newline at end of file diff --git a/server/licenses/lucene-analyzers-common-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-analyzers-common-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 82585bb7ff3b3..0000000000000 --- a/server/licenses/lucene-analyzers-common-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0f75703c30756c31f7d09ec79191dab6fb35c958 \ No newline at end of file diff --git a/server/licenses/lucene-backward-codecs-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-backward-codecs-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..bd8711a4d53d9 --- /dev/null +++ b/server/licenses/lucene-backward-codecs-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +8c93ed67599d345b9359586248ab92342d7d3033 \ No newline at end of file diff --git a/server/licenses/lucene-backward-codecs-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-backward-codecs-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 981e6d1a1730f..0000000000000 --- a/server/licenses/lucene-backward-codecs-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c5c519fdea65726612f79e3dd942b7316966646e \ No newline at end of file diff --git a/server/licenses/lucene-core-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-core-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..36bf03bbbdb54 --- /dev/null +++ b/server/licenses/lucene-core-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +003ed080e5184661e606091cd321c229798b22f8 \ No newline at end of file diff --git a/server/licenses/lucene-core-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-core-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index cea13d14fe1c7..0000000000000 --- a/server/licenses/lucene-core-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f345b6aa3c550dafc63de3e5a5c404691e782336 \ No newline at end of file diff --git a/server/licenses/lucene-grouping-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-grouping-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..0f940ee9c7ac7 --- /dev/null +++ b/server/licenses/lucene-grouping-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +0b4be9f96edfd3dbcff5aa9b3f0914e86eb9cc51 \ No newline at end of file diff --git a/server/licenses/lucene-grouping-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-grouping-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index fcb173608efb8..0000000000000 --- a/server/licenses/lucene-grouping-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7a74855e37124a27af36390c9d15abe33614129e \ No newline at end of file diff --git a/server/licenses/lucene-highlighter-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-highlighter-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..fdc9336fb2ce2 --- /dev/null +++ b/server/licenses/lucene-highlighter-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +a5dcceb5bc017cee6ab5d3ee1943aca1ac6fe074 \ No newline at end of file diff --git a/server/licenses/lucene-highlighter-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-highlighter-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 79addefbfc615..0000000000000 --- a/server/licenses/lucene-highlighter-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0e3df4b469465ef101254fdcbb08ebd8a19f1f9d \ No newline at end of file diff --git a/server/licenses/lucene-join-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-join-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..62726ca415a48 --- /dev/null +++ b/server/licenses/lucene-join-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +b59e7441f121da969bef8eef2c0c61743b4230a8 \ No newline at end of file diff --git a/server/licenses/lucene-join-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-join-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index a9838db7caae4..0000000000000 --- a/server/licenses/lucene-join-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -05d236149c99c860e6b627a8f78ea32918c108c3 \ No newline at end of file diff --git a/server/licenses/lucene-memory-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-memory-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..a68093d2fc42e --- /dev/null +++ b/server/licenses/lucene-memory-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +46736dbb07b432f0a7c1b3080f62932c483e5cb9 \ No newline at end of file diff --git a/server/licenses/lucene-memory-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-memory-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 679c79788f500..0000000000000 --- a/server/licenses/lucene-memory-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d83e7e65eb268425f7bd5be2425d4a00b556bc47 \ No newline at end of file diff --git a/server/licenses/lucene-misc-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-misc-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..23e2b68f3dfcf --- /dev/null +++ b/server/licenses/lucene-misc-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +ee203718d525da0c6258a51a5a32d877089fe5af \ No newline at end of file diff --git a/server/licenses/lucene-misc-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-misc-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index c403d4c4f86b7..0000000000000 --- a/server/licenses/lucene-misc-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -440a998b5bf99871bec4272a219de01b25751d5c \ No newline at end of file diff --git a/server/licenses/lucene-queries-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-queries-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..5bac053813ea2 --- /dev/null +++ b/server/licenses/lucene-queries-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +cf17a332d8e42a45e8f013d5df408f4391d2620a \ No newline at end of file diff --git a/server/licenses/lucene-queries-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-queries-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 6b8897d1ae7b7..0000000000000 --- a/server/licenses/lucene-queries-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2a5c031155fdfa743af321150c0dd654a6ea3c71 \ No newline at end of file diff --git a/server/licenses/lucene-queryparser-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-queryparser-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..471aa797028a7 --- /dev/null +++ b/server/licenses/lucene-queryparser-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +04832303d70502d2ece44501cb1716f42e24fe35 \ No newline at end of file diff --git a/server/licenses/lucene-queryparser-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-queryparser-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index b6c6bf766101d..0000000000000 --- a/server/licenses/lucene-queryparser-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d021c9a461ff0f020d038ad5ecc5127973d4674a \ No newline at end of file diff --git a/server/licenses/lucene-sandbox-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-sandbox-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..486dafc10c73f --- /dev/null +++ b/server/licenses/lucene-sandbox-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +639313e3a9573779b6a28b45a7f57fc1f73ffa46 \ No newline at end of file diff --git a/server/licenses/lucene-sandbox-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-sandbox-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 92f64fca2c749..0000000000000 --- a/server/licenses/lucene-sandbox-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9877a14c53e69b39fff2bf10d49a61171746d940 \ No newline at end of file diff --git a/server/licenses/lucene-spatial-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-spatial-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..0a083b5a078ac --- /dev/null +++ b/server/licenses/lucene-spatial-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +6144b493ba3588a638858d0058054758acc619b9 \ No newline at end of file diff --git a/server/licenses/lucene-spatial-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-spatial-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 2f691988c4495..0000000000000 --- a/server/licenses/lucene-spatial-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7d7e5101b46a120efa311509948c0d1f9bf30155 \ No newline at end of file diff --git a/server/licenses/lucene-spatial-extras-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-spatial-extras-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..851b0d76d3e7a --- /dev/null +++ b/server/licenses/lucene-spatial-extras-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +9d00c6b8bbbbb496aecd555406267fee9e0af914 \ No newline at end of file diff --git a/server/licenses/lucene-spatial-extras-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-spatial-extras-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 86c147f961020..0000000000000 --- a/server/licenses/lucene-spatial-extras-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5a4c11db96ae70b9048243cc530fcbc76faa0978 \ No newline at end of file diff --git a/server/licenses/lucene-spatial3d-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-spatial3d-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..22ce3c7244338 --- /dev/null +++ b/server/licenses/lucene-spatial3d-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +159cdb6d36845690cb1972d02cc0b472bb14b7f3 \ No newline at end of file diff --git a/server/licenses/lucene-spatial3d-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-spatial3d-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 2fbdcdecf1a08..0000000000000 --- a/server/licenses/lucene-spatial3d-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -afb01af1450067b145ca2c1d737b5907288af560 \ No newline at end of file diff --git a/server/licenses/lucene-suggest-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/server/licenses/lucene-suggest-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..0724381bcc6a6 --- /dev/null +++ b/server/licenses/lucene-suggest-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +af1dd0218d58990cca5c1592d9722e67d233c996 \ No newline at end of file diff --git a/server/licenses/lucene-suggest-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/server/licenses/lucene-suggest-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index 1a86525735c05..0000000000000 --- a/server/licenses/lucene-suggest-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -473f0221e0b2ea45940d8ae6dcf16e39c81b18c2 \ No newline at end of file diff --git a/server/src/main/java/org/elasticsearch/index/engine/TranslogLeafReader.java b/server/src/main/java/org/elasticsearch/index/engine/TranslogLeafReader.java index cbe1721f07f71..40c8277d3991a 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/TranslogLeafReader.java +++ b/server/src/main/java/org/elasticsearch/index/engine/TranslogLeafReader.java @@ -56,13 +56,13 @@ final class TranslogLeafReader extends LeafReader { private final Translog.Index operation; private static final FieldInfo FAKE_SOURCE_FIELD = new FieldInfo(SourceFieldMapper.NAME, 1, false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, Collections.emptyMap(), - 0,0); + 0, 0, false); private static final FieldInfo FAKE_ROUTING_FIELD = new FieldInfo(RoutingFieldMapper.NAME, 2, false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, Collections.emptyMap(), - 0,0); + 0, 0, false); private static final FieldInfo FAKE_ID_FIELD = new FieldInfo(IdFieldMapper.NAME, 3, false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, Collections.emptyMap(), - 0,0); + 0, 0, false); private final Version indexVersionCreated; TranslogLeafReader(Translog.Index operation, Version indexVersionCreated) { diff --git a/x-pack/plugin/sql/sql-proto/licenses/lucene-core-7.4.0-snapshot-0a7c3f462f.jar.sha1 b/x-pack/plugin/sql/sql-proto/licenses/lucene-core-7.4.0-snapshot-0a7c3f462f.jar.sha1 new file mode 100644 index 0000000000000..36bf03bbbdb54 --- /dev/null +++ b/x-pack/plugin/sql/sql-proto/licenses/lucene-core-7.4.0-snapshot-0a7c3f462f.jar.sha1 @@ -0,0 +1 @@ +003ed080e5184661e606091cd321c229798b22f8 \ No newline at end of file diff --git a/x-pack/plugin/sql/sql-proto/licenses/lucene-core-7.4.0-snapshot-1cbadda4d3.jar.sha1 b/x-pack/plugin/sql/sql-proto/licenses/lucene-core-7.4.0-snapshot-1cbadda4d3.jar.sha1 deleted file mode 100644 index cea13d14fe1c7..0000000000000 --- a/x-pack/plugin/sql/sql-proto/licenses/lucene-core-7.4.0-snapshot-1cbadda4d3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f345b6aa3c550dafc63de3e5a5c404691e782336 \ No newline at end of file