From 08b0cfd7dd6ac08dbe6ad10f990d541bdbb82163 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 25 Jun 2018 21:02:08 +0200 Subject: [PATCH 01/44] Add `_source`-only snapshot repository This change adds a `_source` only snapshot repository that allows to wrap any existing repository as a _backend_ to snapshot only the `_source` part including live docs markers. Snapshots taken with the `source` repository won't include any index structures. The snapshot will be reduced in size and functionality such that it requires in-place reindexing during restore. The restore process will copy the `_source` data locally and reindexing all data during the recovery from snapshot phase. Users have 2 options for re-indexing: * full reindex: where the data will be reindexed with the original mapping * minimal reindex: where the data will be reindexed with a disabled mapping that results in an index that can only be accessed via `_id`. Both options allow using and updating the index while the latter is mainly for scan/scroll purposes and re-indexing after the fact. This feature aims mainly for disaster recovery use-cases where snapshot size is a concern or where time to restore is less of an issue. --- docs/plugins/repository-source-only.asciidoc | 39 +++ .../index/shard/StoreRecovery.java | 1 + .../org/elasticsearch/index/store/Store.java | 7 +- .../repositories/FilterRepository.java | 154 +++++++++++ .../repositories/RepositoriesService.java | 2 +- .../repositories/Repository.java | 18 +- .../blobstore/BlobStoreRepository.java | 15 +- .../snapshots/SnapshotShardsService.java | 3 +- .../index/shard/IndexShardTests.java | 3 +- .../index/shard/IndexShardTestCase.java | 5 +- .../snapshots/SourceOnlySnapshot.java | 188 +++++++++++++ .../SourceOnlySnapshotRepository.java | 253 ++++++++++++++++++ .../elasticsearch/xpack/core/XPackPlugin.java | 12 +- .../snapshots/SourceOnlySnapshotIT.java | 192 +++++++++++++ .../SourceOnlySnapshotShardTests.java | 208 ++++++++++++++ .../snapshots/SourceOnlySnapshotTests.java | 220 +++++++++++++++ .../rest-api-spec/test/snapshot/10_basic.yml | 73 +++++ 17 files changed, 1375 insertions(+), 18 deletions(-) create mode 100644 docs/plugins/repository-source-only.asciidoc create mode 100644 server/src/main/java/org/elasticsearch/repositories/FilterRepository.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml diff --git a/docs/plugins/repository-source-only.asciidoc b/docs/plugins/repository-source-only.asciidoc new file mode 100644 index 0000000000000..83409249a0ad1 --- /dev/null +++ b/docs/plugins/repository-source-only.asciidoc @@ -0,0 +1,39 @@ +[[repository-src-only]] +=== Source Only Repository + +The Source Only repository adds support for creating `_source` only snapshots using any other +available repository as it's storage backend. This allows using {ref}/modules-snapshots.html[Snapshot/Restore] +to create incremental, storage optimized, and minimal snapshots of an index. + + +[[repository-src-only-usage]] +==== Configuration + +The `_source` only repository always requires a delegate repository to be used as it's storage backend. +In order to use the `fs` respository: + +[source,js] +----------------------------------- +PUT _snapshot/my_src-only_repository +{ + "type": "source", + "settings": { + "delegate_type": "fs", + "location": "my_backup_location" + } +} +----------------------------------- +// CONSOLE +// TESTSETUP + +Since the `_source` only repository doesn't snapshot any index or doc-values structures but only stored +fields and index metadata, it's required to reindex the data during the restore process. This can either happen +as a full re-index based on the mapping of the original index or in a minimal form were only the internal data-structures +are recreated like the `_id` field in order to update the index. The latter can be configured in the repository settings +by setting `"minimal": true`. This allows updates and get operations but won't allow for aggregations or searches. + +A minimal restore is useful if the data is only needed to be re-indexed into another index or if individual documents should be +modified or deleted. + +During restore the re-indexing progress can be monitored via <> API that shows the per-document progress +under the `translog` recovery phase. 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 e9acfe3d8b06f..d0030c6e4005d 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -464,6 +464,7 @@ private void restore(final IndexShard indexShard, final Repository repository, f assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; indexShard.openEngineAndRecoverFromTranslog(); indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm()); + repository.applyPostRestoreOps(indexShard); indexShard.finalizeRecovery(); indexShard.postRecovery("restore done"); } catch (Exception e) { 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 001e263ea8ffb..ae1492f1a4995 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -155,12 +155,15 @@ protected void closeInternal() { public Store(ShardId shardId, IndexSettings indexSettings, DirectoryService directoryService, ShardLock shardLock) throws IOException { this(shardId, indexSettings, directoryService, shardLock, OnClose.EMPTY); } - public Store(ShardId shardId, IndexSettings indexSettings, DirectoryService directoryService, ShardLock shardLock, OnClose onClose) throws IOException { + this(shardId, indexSettings, directoryService.newDirectory(), shardLock, onClose); + } + + public Store(ShardId shardId, IndexSettings indexSettings, Directory dir, ShardLock shardLock, + OnClose onClose) throws IOException { super(shardId, indexSettings); final Settings settings = indexSettings.getSettings(); - Directory dir = directoryService.newDirectory(); final TimeValue refreshInterval = indexSettings.getValue(INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING); logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval); ByteSizeCachingDirectory sizeCachingDir = new ByteSizeCachingDirectory(dir, refreshInterval); diff --git a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java new file mode 100644 index 0000000000000..4b621b6e91666 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java @@ -0,0 +1,154 @@ +package org.elasticsearch.repositories; + +import org.apache.lucene.index.IndexCommit; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.metadata.RepositoryMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.component.Lifecycle; +import org.elasticsearch.common.component.LifecycleListener; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotInfo; +import org.elasticsearch.snapshots.SnapshotShardFailure; + +import java.io.IOException; +import java.util.List; + +public class FilterRepository implements Repository { + + private final Repository in; + + public FilterRepository(Repository in) { + this.in = in; + } + + @Override + public RepositoryMetaData getMetadata() { + return in.getMetadata(); + } + + @Override + public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { + return in.getSnapshotInfo(snapshotId); + } + + @Override + public MetaData getSnapshotGlobalMetaData(SnapshotId snapshotId) { + return in.getSnapshotGlobalMetaData(snapshotId); + } + + @Override + public IndexMetaData getSnapshotIndexMetaData(SnapshotId snapshotId, IndexId index) throws IOException { + return in.getSnapshotIndexMetaData(snapshotId, index); + } + + @Override + public RepositoryData getRepositoryData() { + return in.getRepositoryData(); + } + + @Override + public void initializeSnapshot(SnapshotId snapshotId, List indices, MetaData metaData) { + in.initializeSnapshot(snapshotId, indices, metaData); + } + + @Override + public SnapshotInfo finalizeSnapshot(SnapshotId snapshotId, List indices, long startTime, String failure, int totalShards, + List shardFailures, long repositoryStateId, boolean includeGlobalState) { + return in.finalizeSnapshot(snapshotId, indices, startTime, failure, totalShards, shardFailures, repositoryStateId, + includeGlobalState); + } + + @Override + public void deleteSnapshot(SnapshotId snapshotId, long repositoryStateId) { + in.deleteSnapshot(snapshotId, repositoryStateId); + } + + @Override + public long getSnapshotThrottleTimeInNanos() { + return in.getSnapshotThrottleTimeInNanos(); + } + + @Override + public long getRestoreThrottleTimeInNanos() { + return in.getRestoreThrottleTimeInNanos(); + } + + @Override + public String startVerification() { + return in.startVerification(); + } + + @Override + public void endVerification(String verificationToken) { + in.endVerification(verificationToken); + } + + @Override + public void verify(String verificationToken, DiscoveryNode localNode) { + in.verify(verificationToken, localNode); + } + + @Override + public boolean isReadOnly() { + return in.isReadOnly(); + } + + @Override + public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, + IndexShardSnapshotStatus snapshotStatus) { + in.snapshotShard(shard, store, snapshotId, indexId, snapshotIndexCommit, snapshotStatus); + } + + @Override + public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, + RecoveryState recoveryState) { + in.restoreShard(shard, snapshotId, version, indexId, snapshotShardId, recoveryState); + } + + @Override + public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId) { + return in.getShardSnapshotStatus(snapshotId, version, indexId, shardId); + } + + @Override + public void applyPostRestoreOps(IndexShard shard) throws IOException { + in.applyPostRestoreOps(shard); + } + + @Override + public Lifecycle.State lifecycleState() { + return in.lifecycleState(); + } + + @Override + public void addLifecycleListener(LifecycleListener listener) { + in.addLifecycleListener(listener); + } + + @Override + public void removeLifecycleListener(LifecycleListener listener) { + in.removeLifecycleListener(listener); + } + + @Override + public void start() { + in.start(); + } + + @Override + public void stop() { + in.stop(); + } + + @Override + public void close() { + in.close(); + } +} diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java index c6cbaa50cdf02..aef4381cd8b2d 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java @@ -398,7 +398,7 @@ private Repository createRepository(RepositoryMetaData repositoryMetaData) { "repository type [" + repositoryMetaData.type() + "] does not exist"); } try { - Repository repository = factory.create(repositoryMetaData); + Repository repository = factory.create(repositoryMetaData, typesRegistry::get); repository.start(); return repository; } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index c0b45259f9911..60055d235ad77 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -28,6 +28,7 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; @@ -35,6 +36,7 @@ import java.io.IOException; import java.util.List; +import java.util.function.Function; /** * An interface for interacting with a repository in snapshot and restore. @@ -46,7 +48,7 @@ *
    *
  • Master calls {@link #initializeSnapshot(SnapshotId, List, org.elasticsearch.cluster.metadata.MetaData)} * with list of indices that will be included into the snapshot
  • - *
  • Data nodes call {@link Repository#snapshotShard(IndexShard, SnapshotId, IndexId, IndexCommit, IndexShardSnapshotStatus)} + *
  • Data nodes call {@link Repository#snapshotShard(IndexShard, Store, SnapshotId, IndexId, IndexCommit, IndexShardSnapshotStatus)} * for each shard
  • *
  • When all shard calls return master calls {@link #finalizeSnapshot} with possible list of failures
  • *
@@ -63,6 +65,10 @@ interface Factory { * @param metadata metadata for the repository including name and settings */ Repository create(RepositoryMetaData metadata) throws Exception; + + default Repository create(RepositoryMetaData metaData, Function typeLookup) throws Exception { + return create(metaData); + } } /** @@ -188,14 +194,15 @@ SnapshotInfo finalizeSnapshot(SnapshotId snapshotId, List indices, long *

* As snapshot process progresses, implementation of this method should update {@link IndexShardSnapshotStatus} object and check * {@link IndexShardSnapshotStatus#isAborted()} to see if the snapshot process should be aborted. - * * @param shard shard to be snapshotted + * @param store store to be snapshotted * @param snapshotId snapshot id * @param indexId id for the index being snapshotted * @param snapshotIndexCommit commit point * @param snapshotStatus snapshot status */ - void snapshotShard(IndexShard shard, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus); + void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, + IndexShardSnapshotStatus snapshotStatus); /** * Restores snapshot of the shard. @@ -211,6 +218,11 @@ SnapshotInfo finalizeSnapshot(SnapshotId snapshotId, List indices, long */ void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState); + /** + * This allows a repository to apply operations after the snapshot has been restored as part of the translog recovery phase. + */ + default void applyPostRestoreOps(IndexShard shard) throws IOException {} + /** * Retrieve shard snapshot status for the stored snapshot * 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 cc1d27425e133..d3a292a819afd 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -846,8 +846,9 @@ private void writeAtomic(final String blobName, final BytesReference bytesRef, b } @Override - public void snapshotShard(IndexShard shard, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { - SnapshotContext snapshotContext = new SnapshotContext(shard, snapshotId, indexId, snapshotStatus, System.currentTimeMillis()); + public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, + IndexShardSnapshotStatus snapshotStatus) { + SnapshotContext snapshotContext = new SnapshotContext(store, snapshotId, indexId, snapshotStatus, System.currentTimeMillis()); try { snapshotContext.snapshot(snapshotIndexCommit); } catch (Exception e) { @@ -855,7 +856,7 @@ public void snapshotShard(IndexShard shard, SnapshotId snapshotId, IndexId index if (e instanceof IndexShardSnapshotFailedException) { throw (IndexShardSnapshotFailedException) e; } else { - throw new IndexShardSnapshotFailedException(shard.shardId(), e); + throw new IndexShardSnapshotFailedException(store.shardId(), e); } } } @@ -1158,15 +1159,15 @@ private class SnapshotContext extends Context { /** * Constructs new context * - * @param shard shard to be snapshotted + * @param store store to be snapshotted * @param snapshotId snapshot id * @param indexId the id of the index being snapshotted * @param snapshotStatus snapshot status to report progress */ - SnapshotContext(IndexShard shard, SnapshotId snapshotId, IndexId indexId, IndexShardSnapshotStatus snapshotStatus, long startTime) { - super(snapshotId, Version.CURRENT, indexId, shard.shardId()); + SnapshotContext(Store store, SnapshotId snapshotId, IndexId indexId, IndexShardSnapshotStatus snapshotStatus, long startTime) { + super(snapshotId, Version.CURRENT, indexId, store.shardId()); this.snapshotStatus = snapshotStatus; - this.store = shard.store(); + this.store = store; this.startTime = startTime; } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 33b4d85298799..88612dbcc5022 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -389,7 +389,8 @@ private void snapshot(final IndexShard indexShard, final Snapshot snapshot, fina try { // we flush first to make sure we get the latest writes snapshotted try (Engine.IndexCommitRef snapshotRef = indexShard.acquireLastIndexCommit(true)) { - repository.snapshotShard(indexShard, snapshot.getSnapshotId(), indexId, snapshotRef.getIndexCommit(), snapshotStatus); + repository.snapshotShard(indexShard, indexShard.store(), snapshot.getSnapshotId(), indexId, snapshotRef.getIndexCommit(), + snapshotStatus); if (logger.isDebugEnabled()) { final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy(); logger.debug("snapshot ({}) completed to {} with {}", snapshot, repository, lastSnapshotStatus); 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 03442be7f0662..f20ebd53205bf 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -2764,7 +2764,8 @@ public boolean isReadOnly() { } @Override - public void snapshotShard(IndexShard shard, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { + public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, + IndexShardSnapshotStatus snapshotStatus) { } @Override 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 d2a84589669a6..e095b6cb097fe 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 @@ -131,7 +131,7 @@ public void onRecoveryFailure(RecoveryState state, RecoveryFailedException e, bo }; protected ThreadPool threadPool; - private long primaryTerm; + protected long primaryTerm; @Override public void setUp() throws Exception { @@ -693,7 +693,8 @@ protected void snapshotShard(final IndexShard shard, Index index = shard.shardId().getIndex(); IndexId indexId = new IndexId(index.getName(), index.getUUID()); - repository.snapshotShard(shard, snapshot.getSnapshotId(), indexId, indexCommitRef.getIndexCommit(), snapshotStatus); + repository.snapshotShard(shard, shard.store(), snapshot.getSnapshotId(), indexId, indexCommitRef.getIndexCommit(), + snapshotStatus); } final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java new file mode 100644 index 0000000000000..19a42b15c9408 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java @@ -0,0 +1,188 @@ +/* + * 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.snapshots; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.index.CheckIndex; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SegmentCommitInfo; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.SoftDeletesDirectoryReaderWrapper; +import org.apache.lucene.index.StandardDirectoryReader; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.Lock; +import org.apache.lucene.store.TrackingDirectoryWrapper; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.elasticsearch.common.lucene.Lucene; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.FIELDS_EXTENSION; +import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.FIELDS_INDEX_EXTENSION; + +public final class SourceOnlySnapshot { + private final Directory targetDirectory; + private final String softDeletesField; + private final List createdFiles = new ArrayList<>(); + + public SourceOnlySnapshot(Directory targetDirectory, String softDeletesField) { + this.targetDirectory = targetDirectory; + this.softDeletesField = softDeletesField; + } + + public List getCreatedFiles() { + return createdFiles; + } + + public synchronized void syncSnapshot(IndexCommit commit) throws IOException { + long generation; + Map existingSegments = new HashMap<>(); + if (Lucene.indexExists(targetDirectory)) { + SegmentInfos existingsSegmentInfos = Lucene.readSegmentInfos(targetDirectory); + for (SegmentCommitInfo info : existingsSegmentInfos) { + existingSegments.put(new BytesRef(info.info.getId()), info); + } + generation = existingsSegmentInfos.getGeneration(); + } else { + generation = 1; + } + String segmentFileName; + try (Lock writeLock = targetDirectory.obtainLock(IndexWriter.WRITE_LOCK_NAME); + StandardDirectoryReader reader = (StandardDirectoryReader) DirectoryReader.open(commit)) { + SegmentInfos segmentInfos = reader.getSegmentInfos(); + DirectoryReader wrapper = wrapReader(reader); + List newInfos = new ArrayList<>(); + for (LeafReaderContext ctx : wrapper.leaves()) { + SegmentCommitInfo info = segmentInfos.info(ctx.ord); + LeafReader leafReader = ctx.reader(); + Bits liveDocs = leafReader.getLiveDocs(); + SegmentCommitInfo newInfo = syncSegment(info, liveDocs, leafReader.getFieldInfos(), existingSegments); + newInfos.add(newInfo); + } + segmentInfos.clear(); + segmentInfos.addAll(newInfos); + segmentInfos.setNextWriteGeneration(Math.max(segmentInfos.getGeneration(), generation)+1); + String pendingSegmentFileName = IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS, + "", segmentInfos.getGeneration()); + try (IndexOutput segnOutput = targetDirectory.createOutput(pendingSegmentFileName, IOContext.DEFAULT)) { + segmentInfos.write(targetDirectory, segnOutput); + } + targetDirectory.sync(Collections.singleton(pendingSegmentFileName)); + targetDirectory.sync(createdFiles); + segmentFileName = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", segmentInfos.getGeneration()); + targetDirectory.rename(pendingSegmentFileName, segmentFileName); + } + Lucene.pruneUnreferencedFiles(segmentFileName, targetDirectory); + assert assertCheckIndex(); + } + + private boolean assertCheckIndex() throws IOException { + ByteArrayOutputStream output = new ByteArrayOutputStream(1024); + try (CheckIndex checkIndex = new CheckIndex(targetDirectory)) { + checkIndex.setFailFast(true); + checkIndex.setCrossCheckTermVectors(false); + checkIndex.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false); + CheckIndex.Status status = checkIndex.checkIndex(); + if (status == null || status.clean == false) { + throw new RuntimeException("CheckIndex failed: " + output.toString(IOUtils.UTF_8)); + } + return true; + } + } + + DirectoryReader wrapReader(DirectoryReader reader) throws IOException { + return softDeletesField == null ? reader : new SoftDeletesDirectoryReaderWrapper(reader, softDeletesField); + } + + private SegmentCommitInfo syncSegment(SegmentCommitInfo segmentCommitInfo, Bits liveDocs, FieldInfos fieldInfos, + Map existingSegments) throws IOException { + SegmentInfo si = segmentCommitInfo.info; + Codec codec = si.getCodec(); + final String segmentSuffix = ""; + SegmentCommitInfo newInfo; + final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(targetDirectory); + BytesRef segmentId = new BytesRef(si.getId()); + boolean exists = existingSegments.containsKey(segmentId); + if (exists == false) { + SegmentInfo newSegmentInfo = new SegmentInfo(si.dir, si.getVersion(), si.getMinVersion(), si.name, si.maxDoc(), false, + si.getCodec(), si.getDiagnostics(), si.getId(), si.getAttributes(), si.getIndexSort()); // TODO should we drop the sort? + newInfo = new SegmentCommitInfo(newSegmentInfo, 0, 0, -1, -1, -1); + List fieldInfoCopy = new ArrayList<>(fieldInfos.size()); + for (FieldInfo fieldInfo : fieldInfos) { + fieldInfoCopy.add(new FieldInfo(fieldInfo.name, fieldInfo.number, + false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, fieldInfo.attributes(), 0, 0, false)); + } + FieldInfos newFieldInfos = new FieldInfos(fieldInfoCopy.toArray(new FieldInfo[0])); + codec.fieldInfosFormat().write(trackingDir, newSegmentInfo, segmentSuffix, newFieldInfos, IOContext.DEFAULT); + newInfo.setFieldInfosFiles(trackingDir.getCreatedFiles()); + String idxFile = IndexFileNames.segmentFileName(newSegmentInfo.name, segmentSuffix, FIELDS_INDEX_EXTENSION); + String dataFile = IndexFileNames.segmentFileName(newSegmentInfo.name, segmentSuffix, FIELDS_EXTENSION); + Directory sourceDir = newSegmentInfo.dir; + if (si.getUseCompoundFile()) { + sourceDir = codec.compoundFormat().getCompoundReader(sourceDir, si, IOContext.DEFAULT); + } + trackingDir.copyFrom(sourceDir, idxFile, idxFile, IOContext.DEFAULT); + trackingDir.copyFrom(sourceDir, dataFile, dataFile, IOContext.DEFAULT); + if (sourceDir != newSegmentInfo.dir) { + sourceDir.close(); + } + } else { + newInfo = existingSegments.get(segmentId); + assert newInfo.info.getUseCompoundFile() == false; + } + int deletes = segmentCommitInfo.getDelCount() + segmentCommitInfo.getSoftDelCount(); + if (liveDocs != null && deletes != 0 && deletes != newInfo.getDelCount()) { + if (newInfo.getDelCount() != 0) { + assert assertLiveDocs(liveDocs, deletes); + } + codec.liveDocsFormat().writeLiveDocs(liveDocs, trackingDir, newInfo, deletes - newInfo.getDelCount(), + IOContext.DEFAULT); + SegmentCommitInfo info = new SegmentCommitInfo(newInfo.info, deletes, 0, newInfo.getNextDelGen(), -1, -1); + info.setFieldInfosFiles(newInfo.getFieldInfosFiles()); + info.info.setFiles(trackingDir.getCreatedFiles()); + newInfo = info; + } + if (exists == false) { + newInfo.info.setFiles(trackingDir.getCreatedFiles()); + codec.segmentInfoFormat().write(trackingDir, newInfo.info, IOContext.DEFAULT); + } + createdFiles.addAll(trackingDir.getCreatedFiles()); + return newInfo; + + } + + private boolean assertLiveDocs(Bits liveDocs, int deletes) { + int actualDeletes = 0; + for (int i = 0; i < liveDocs.length(); i++ ) { + if (liveDocs.get(i) == false) { + actualDeletes++; + } + } + assert actualDeletes == deletes : " actual: " + actualDeletes + " deletes: " + deletes; + return true; + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java new file mode 100644 index 0000000000000..ae84b65adb13c --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -0,0 +1,253 @@ +package org.elasticsearch.snapshots; + +import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.store.FSDirectory; +import org.apache.lucene.store.HardlinkCopyDirectoryWrapper; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.SimpleFSDirectory; +import org.apache.lucene.util.Bits; +import org.elasticsearch.Version; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MappingMetaData; +import org.elasticsearch.cluster.metadata.RepositoryMetaData; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.lucene.uid.Versions; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.env.ShardLock; +import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.fieldvisitor.FieldsVisitor; +import org.elasticsearch.index.mapper.SourceToParse; +import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.shard.ShardPath; +import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.repositories.FilterRepository; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.Repository; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Path; +import java.util.Iterator; +import java.util.function.Function; + +/** + *

+ * This is a filter snapshot repository that only snapshots the minimal required information + * that is needed to recreate the index. In other words instead of snapshotting the entire shard + * with all it's lucene indexed fields, doc values, points etc. it only snapshots the the stored + * fields including _source and _routing as well as the live docs in oder to distinguish between + * live and deleted docs. + *

+ *

+ * The repository can wrap any other repository delegating the source only snapshot to it to and read + * from it. For instance a file repository of type fs by passing settings.delegate_type=fs + * at repository creation time. + *

+ * The repository supports two distinct restore options: + *
    + *
  • minimal restore: this option re-indexes all documents during restore with an empty mapping. The original mapping is + * stored in the restored indexes _meta mapping field. The minimal restore must be enabled by setting + * settings.restore_minimal=true.
  • + *
  • full restore: this option re-indexes all documents during restore with the original mapping. This option is the + * default. This option has a significant operational overhead compared to the minimal option but recreates a fully functional new + * index
  • + *
+ * + * Reindex operations are executed in a single thread and can be monitored via indices recovery stats. Every indexed document will be + * reported as a translog document. + * + */ +// TODO: as a followup we should rename translog phase to operation phase in the indices _recovery stats +public final class SourceOnlySnapshotRepository extends FilterRepository { + public static final Setting DELEGATE_TYPE = + new Setting<>("delegate_type", "", Function.identity(), Setting.Property.NodeScope); + + public static final Setting RESTORE_MINIMAL = Setting.boolSetting("restore_minimal", + false, Setting.Property.NodeScope); + + public static final String SNAPSHOT_DIR_NAME = "_snapshot"; + public static final String RESTORE_DIR_NAME = "_restore"; + private final boolean restoreMinimal; + + public static Repository.Factory newFactory() { + return new Repository.Factory() { + + @Override + public Repository create(RepositoryMetaData metadata) { + throw new UnsupportedOperationException(); + } + + @Override + public Repository create(RepositoryMetaData metaData, Function typeLookup) throws Exception { + String delegateType = DELEGATE_TYPE.get(metaData.settings()); + if (Strings.hasLength(delegateType) == false) { + throw new IllegalArgumentException(DELEGATE_TYPE.getKey() + " must be set"); + } + boolean restoreMinimal = RESTORE_MINIMAL.get(metaData.settings()); + Repository.Factory factory = typeLookup.apply(delegateType); + return new SourceOnlySnapshotRepository(factory.create(new RepositoryMetaData(metaData.name(), + delegateType, metaData.settings()), typeLookup), restoreMinimal); + } + }; + } + + public SourceOnlySnapshotRepository(Repository in, boolean restoreMinimal) { + super(in); + this.restoreMinimal = restoreMinimal; + } + + @Override + public IndexMetaData getSnapshotIndexMetaData(SnapshotId snapshotId, IndexId index) throws IOException { + IndexMetaData snapshotIndexMetaData = super.getSnapshotIndexMetaData(snapshotId, index); + if (restoreMinimal) { + // TODO: can we lie about the index.version.created here and produce an index with a new version since we reindex anyway? + + // for a minimal restore we basically disable indexing on all fields and only create an index + // that is fully functional from an operational perspective. ie. it will have all metadata fields like version/ + // seqID etc. and an indexed ID field such that we can potentially perform updates on them or delete documents. + ImmutableOpenMap mappings = snapshotIndexMetaData.getMappings(); + Iterator> iterator = mappings.iterator(); + IndexMetaData.Builder builder = IndexMetaData.builder(snapshotIndexMetaData); + while (iterator.hasNext()) { + ObjectObjectCursor next = iterator.next(); + MappingMetaData.Routing routing = next.value.routing(); + final String mapping; + if (routing.required()) { // we have to respect the routing to be on the safe side so we pass this one on. + mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + ", " + + "\"_routing\" : { \"required\" : true } } }"; + } else { + mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + " } }"; + } + System.out.println(mapping); + builder.putMapping(next.key, mapping); + } + return builder.build(); + } else { + return snapshotIndexMetaData; + } + + } + + @Override + public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, + IndexShardSnapshotStatus snapshotStatus) { + if (shard.mapperService().documentMapper() != null // if there is no mapping this is null + && shard.mapperService().documentMapper().sourceMapper().isComplete() == false) { + throw new IllegalStateException("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled " + + "or filters the source"); + } + ShardPath shardPath = shard.shardPath(); + Path dataPath = shardPath.getDataPath(); + // TODO should we have a snapshot tmp directory per shard that is maintained by the system? + Path snapPath = dataPath.resolve(SNAPSHOT_DIR_NAME); + try (FSDirectory directory = new SimpleFSDirectory(snapPath)) { + Store tempStore = new Store(store.shardId(), store.indexSettings(), directory, new ShardLock(store.shardId()) { + @Override + protected void closeInternal() { + // do nothing; + } + }, Store.OnClose.EMPTY); + SourceOnlySnapshot snapshot = new SourceOnlySnapshot(tempStore.directory(), null); + snapshot.syncSnapshot(snapshotIndexCommit); + store.incRef(); + try (DirectoryReader reader = DirectoryReader.open(tempStore.directory()); + ) { + IndexCommit indexCommit = reader.getIndexCommit(); + super.snapshotShard(shard, tempStore, snapshotId, indexId, indexCommit, snapshotStatus); + } finally { + store.decRef(); + } + } catch (IOException e) { + // why on earth does this super method not declare IOException + throw new UncheckedIOException(e); + } + } + + @Override + public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { + super.restoreShard(shard, snapshotId, version, indexId, snapshotShardId, recoveryState); + ShardPath shardPath = shard.shardPath(); + try { + Path restoreSourceCopy = shardPath.getDataPath().resolve(RESTORE_DIR_NAME); + try (HardlinkCopyDirectoryWrapper wrapper = new HardlinkCopyDirectoryWrapper(FSDirectory.open(restoreSourceCopy))) { + Lucene.cleanLuceneIndex(wrapper); + SegmentInfos segmentInfos = shard.store().readLastCommittedSegmentsInfo(); + for (String file : segmentInfos.files(true)) { + wrapper.copyFrom(shard.store().directory(), file, file, IOContext.DEFAULT); + } + } + Lucene.cleanLuceneIndex(shard.store().directory()); // wipe the old index + shard.store().createEmpty(); + } catch (IOException ex) { + // why on earth does this super method not declare IOException + throw new UncheckedIOException(ex); + } + } + + @Override + public void applyPostRestoreOps(IndexShard shard) throws IOException { + ShardPath shardPath = shard.shardPath(); + Path restoreSourceCopy = shardPath.getDataPath().resolve(RESTORE_DIR_NAME); + RecoveryState.Translog state = shard.recoveryState().getTranslog(); + assert state.totalOperations() == 0 : "translog state should have 0 total ops but got: " + state.totalOperations(); + state.reset(); + long sequenceNumber = 0; + try (FSDirectory dir = FSDirectory.open(restoreSourceCopy)) { + try (IndexReader reader = DirectoryReader.open(dir)) { + state.totalOperationsOnStart(reader.numDocs()); + state.totalOperations(reader.numDocs()); + long primaryTerm = shard.getPendingPrimaryTerm(); + FieldsVisitor rootFieldsVisitor = new FieldsVisitor(true); + for (LeafReaderContext ctx : reader.leaves()) { + LeafReader leafReader = ctx.reader(); + Bits liveDocs = leafReader.getLiveDocs(); + // TODO: we could do this in parallel per segment here or even per docID + // there is a lot of room for doing this multi-threaded but not for the first iteration + for (int i = 0; i < leafReader.maxDoc(); i++) { + if (liveDocs == null || liveDocs.get(i)) { + rootFieldsVisitor.reset(); + leafReader.document(i, rootFieldsVisitor); + rootFieldsVisitor.postProcess(shard.mapperService()); + Uid uid = rootFieldsVisitor.uid(); + BytesReference source = rootFieldsVisitor.source(); + if (source != null) { // nested fields don't have source. in this case we should be fine. + // TODO we should have a dedicated origin for this LOCAL_TRANSLOG_RECOVERY is misleading. + Engine.Result result = shard.applyTranslogOperation(new Translog.Index(uid.type(), uid.id(), + sequenceNumber++, primaryTerm, 1, source.toBytesRef().bytes, + rootFieldsVisitor.routing(), -1), Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY); + if (result.getResultType() != Engine.Result.Type.SUCCESS) { + throw new IllegalStateException("failed applying post restore operation result: " + result + .getResultType(), result.getFailure()); + } + state.incrementRecoveredOperations(); + } else { + assert restoreMinimal // in this case we don't have nested in the mapping. + || shard.mapperService().hasNested() : "_source is null but shard has no nested docs"; + } + } + } + } + shard.flush(new FlushRequest()); + } + Lucene.cleanLuceneIndex(dir); // clear the tmp index; + } + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java index aaa3effcfe8bb..e22ee73fa66c6 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java @@ -42,10 +42,13 @@ import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.persistent.PersistentTaskParams; import org.elasticsearch.plugins.ExtensiblePlugin; +import org.elasticsearch.plugins.RepositoryPlugin; import org.elasticsearch.plugins.ScriptPlugin; +import org.elasticsearch.repositories.Repository; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.snapshots.SourceOnlySnapshotRepository; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xpack.core.action.TransportXPackInfoAction; @@ -67,13 +70,15 @@ import java.time.Clock; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -public class XPackPlugin extends XPackClientPlugin implements ScriptPlugin, ExtensiblePlugin { +public class XPackPlugin extends XPackClientPlugin implements ScriptPlugin, ExtensiblePlugin, RepositoryPlugin { private static Logger logger = ESLoggerFactory.getLogger(XPackPlugin.class); private static DeprecationLogger deprecationLogger = new DeprecationLogger(logger); @@ -340,4 +345,9 @@ default Optional getRequiredFeature() { } } + @Override + public Map getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry) { + return Collections.singletonMap("source", SourceOnlySnapshotRepository.newFactory()); + } + } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java new file mode 100644 index 0000000000000..a076801b0418d --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -0,0 +1,192 @@ +package org.elasticsearch.snapshots; + +import org.elasticsearch.action.DocWriteResponse; +import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; +import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; +import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; +import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; +import org.elasticsearch.action.delete.DeleteResponse; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.metadata.MappingMetaData; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.env.Environment; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.node.Node; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.RepositoryPlugin; +import org.elasticsearch.repositories.Repository; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHits; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.test.ESIntegTestCase; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ExecutionException; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; + +public class SourceOnlySnapshotIT extends ESIntegTestCase { + + @Override + protected Collection> nodePlugins() { + Collection> classes = new ArrayList<>(super.nodePlugins()); + classes.add(MyPlugin.class); + return classes; + } + + public static final class MyPlugin extends Plugin implements RepositoryPlugin { + @Override + public Map getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry) { + return Collections.singletonMap("source", SourceOnlySnapshotRepository.newFactory()); + } + } + + /** + * Tests that a source only index snapshot + */ + public void testSnapshotAndRestore() throws Exception { + final String sourceIdx = "test-idx"; + IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, false, false); + + SearchResponse searchResponse = client().prepareSearch(sourceIdx).setSize(builders.length) + .addSort(SeqNoFieldMapper.NAME, SortOrder.ASC).get(); + SearchHits hits = searchResponse.getHits(); + assertEquals(builders.length, hits.totalHits); + long i = 0; + for (SearchHit hit : hits) { + String id = hit.getId(); + Map sourceAsMap = hit.getSourceAsMap(); + assertTrue(sourceAsMap.containsKey("field1")); + assertEquals(i++, hit.getSortValues()[0]); + assertEquals("bar "+id, sourceAsMap.get("field1")); + assertEquals("r"+id, hit.field("_routing").getValue()); + } + // ensure we can find hits + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() + .addIds("" + randomIntBetween(0, builders.length))).get(), 1); + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), builders.length); + } + + /** + * Tests that a source only index snapshot + */ + public void testSnapshotAndRestoreMinimal() throws Exception { + final String sourceIdx = "test-idx"; + boolean requireRouting = randomBoolean(); + IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting); + + SearchResponse searchResponse = client().prepareSearch(sourceIdx) + .addSort(SeqNoFieldMapper.NAME, SortOrder.ASC) + .setSize(builders.length).get(); + SearchHits hits = searchResponse.getHits(); + assertEquals(builders.length, hits.totalHits); + long i = 0; + for (SearchHit hit : hits) { + String id = hit.getId(); + Map sourceAsMap = hit.getSourceAsMap(); + assertTrue(sourceAsMap.containsKey("field1")); + assertEquals(i++, hit.getSortValues()[0]); + assertEquals("bar " + id, sourceAsMap.get("field1")); + assertEquals("r" + id, hit.field("_routing").getValue()); + } + GetMappingsResponse getMappingsResponse = client().admin().indices().prepareGetMappings(sourceIdx).get(); + ImmutableOpenMap mapping = getMappingsResponse + .getMappings().get(sourceIdx); + assertTrue(mapping.containsKey("_doc")); + if (requireRouting) { + assertEquals("{\"_doc\":{\"enabled\":false," + + "\"_meta\":{\"_doc\":{\"_routing\":{\"required\":true}," + + "\"properties\":{\"field1\":{\"type\":\"text\"," + + "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}}}}," + + "\"_routing\":{\"required\":true}}}", mapping.get("_doc").source().string()); + } else { + assertEquals("{\"_doc\":{\"enabled\":false," + + "\"_meta\":{\"_doc\":{\"properties\":{\"field1\":{\"type\":\"text\"," + + "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}}}}}}", mapping.get("_doc").source().string()); + } + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() + .addIds("" + randomIntBetween(0, builders.length))).get(), 1); + // ensure we can not find hits it's a minimal restore + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); + // make sure deletes work + String idToDelete = "" + randomIntBetween(0, builders.length); + DeleteResponse deleteResponse = client().prepareDelete(sourceIdx, "_doc", idToDelete).setRouting("r" + idToDelete).get(); + assertEquals(DocWriteResponse.Result.DELETED, deleteResponse.getResult()); + refresh(sourceIdx); + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery().addIds(idToDelete)).get(), 0); + } + + private IndexRequestBuilder[] snashotAndRestore(String sourceIdx, int numShards, boolean minimal, boolean requireRouting) + throws ExecutionException, InterruptedException, IOException { + logger.info("--> starting a master node and a data node"); + internalCluster().startMasterOnlyNode(); + internalCluster().startDataOnlyNode(); + + final Client client = client(); + final String repo = "test-repo"; + final String snapshot = "test-snap"; + + logger.info("--> creating repository"); + assertAcked(client.admin().cluster().preparePutRepository(repo).setType("source") + .setSettings(Settings.builder().put("location", randomRepoPath()) + .put("delegate_type", "fs") + .put("restore_minimal", minimal) + .put("compress", randomBoolean()))); + + CreateIndexRequestBuilder createIndexRequestBuilder = prepareCreate(sourceIdx, 0, Settings.builder() + .put("number_of_shards", numShards).put("number_of_replicas", 0)); + if (requireRouting) { + createIndexRequestBuilder.addMapping("_doc", "_routing", "required=true"); + } + assertAcked(createIndexRequestBuilder); + ensureGreen(); + + logger.info("--> indexing some data"); + IndexRequestBuilder[] builders = new IndexRequestBuilder[randomIntBetween(10, 100)]; + for (int i = 0; i < builders.length; i++) { + builders[i] = client().prepareIndex(sourceIdx, "_doc", + Integer.toString(i)).setSource("field1", "bar " + i).setRouting("r" + i); + } + indexRandom(true, builders); + flushAndRefresh(); + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery().addIds("0")).get(), 1); + + logger.info("--> snapshot the index"); + CreateSnapshotResponse createResponse = client.admin().cluster() + .prepareCreateSnapshot(repo, snapshot) + .setWaitForCompletion(true).setIndices(sourceIdx).get(); + assertEquals(SnapshotState.SUCCESS, createResponse.getSnapshotInfo().state()); + + logger.info("--> delete index and stop the data node"); + assertAcked(client.admin().indices().prepareDelete(sourceIdx).get()); + internalCluster().stopRandomDataNode(); + client().admin().cluster().prepareHealth().setTimeout("30s").setWaitForNodes("1"); + + logger.info("--> start a new data node"); + final Settings dataSettings = Settings.builder() + .put(Node.NODE_NAME_SETTING.getKey(), randomAlphaOfLength(5)) + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) // to get a new node id + .build(); + internalCluster().startDataOnlyNode(dataSettings); + client().admin().cluster().prepareHealth().setTimeout("30s").setWaitForNodes("2"); + + logger.info("--> restore the index and ensure all shards are allocated"); + RestoreSnapshotResponse restoreResponse = client().admin().cluster() + .prepareRestoreSnapshot(repo, snapshot).setWaitForCompletion(true) + .setIndices(sourceIdx).get(); + assertEquals(restoreResponse.getRestoreInfo().totalShards(), + restoreResponse.getRestoreInfo().successfulShards()); + ensureYellow(); + return builders; + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java new file mode 100644 index 0000000000000..a0cb2b291a8b0 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -0,0 +1,208 @@ +package org.elasticsearch.snapshots; + +import org.apache.lucene.document.Document; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.Term; +import org.apache.lucene.store.IndexInput; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.RepositoryMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.RecoverySource; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.TestShardRouting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.env.Environment; +import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.Uid; +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.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.fs.FsRepository; + +import java.io.IOException; +import java.nio.file.Path; + +public class SourceOnlySnapshotShardTests extends IndexShardTestCase { + + public void testSourceIncomplete() throws IOException { + ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), true, + ShardRoutingState.INITIALIZING, RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE); + 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 metaData = IndexMetaData.builder(shardRouting.getIndexName()) + .settings(settings) + .primaryTerm(0, primaryTerm) + .putMapping("_doc", + "{\"_source\":{\"enabled\": false}}").build(); + IndexShard shard = newShard(shardRouting, metaData); + recoverShardFromStore(shard); + + for (int i = 0; i < 1; i++) { + final String id = Integer.toString(i); + indexDoc(shard, "_doc", id); + } + SnapshotId snapshotId = new SnapshotId("test", "test"); + IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); + SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository(), false); + repository.start(); + try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); + IllegalStateException illegalStateException = expectThrows(IllegalStateException.class, () -> + repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef.getIndexCommit(), + indexShardSnapshotStatus)); + assertEquals("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled or filters the source" + , illegalStateException.getMessage()); + } + closeShards(shard); + } + + public void testIncrementalSnapshot() throws IOException { + IndexShard shard = newStartedShard(); + for (int i = 0; i < 10; i++) { + final String id = Integer.toString(i); + indexDoc(shard, "_doc", id); + } + + SnapshotId snapshotId = new SnapshotId("test", "test"); + IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); + SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository(), false); + repository.start(); + int totalFileCount = -1; + try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); + repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef.getIndexCommit(), indexShardSnapshotStatus); + IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); + assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); + totalFileCount = copy.getTotalFileCount(); + assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); + } + + indexDoc(shard, "_doc", Integer.toString(10)); + indexDoc(shard, "_doc", Integer.toString(11)); + snapshotId = new SnapshotId("test_1", "test_1"); + try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); + repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef.getIndexCommit(), indexShardSnapshotStatus); + IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); + // we processed the segments_N file plus _1.si, _1.fdx, _1.fnm, _1.fdt + assertEquals(5, copy.getIncrementalFileCount()); + // in total we have 4 more files than the previous snap since we don't count the segments_N twice + assertEquals(totalFileCount+4, copy.getTotalFileCount()); + assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); + } + deleteDoc(shard, "_doc", Integer.toString(10)); + snapshotId = new SnapshotId("test_2", "test_2"); + try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); + repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef.getIndexCommit(), indexShardSnapshotStatus); + IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); + // we processed the segments_N file plus _1_1.liv + assertEquals(2, copy.getIncrementalFileCount()); + // in total we have 5 more files than the previous snap since we don't count the segments_N twice + assertEquals(totalFileCount+5, copy.getTotalFileCount()); + assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); + } + closeShards(shard); + } + + private String randomDoc() { + return "{ \"value\" : \"" + randomAlphaOfLength(10) + "\"}"; + } + + public void testRestoreAndReindex() throws IOException { + IndexShard shard = newStartedShard(true); + int numInitialDocs = randomIntBetween(10, 100); + for (int i = 0; i < numInitialDocs; i++) { + final String id = Integer.toString(i); + indexDoc(shard, "_doc", id, randomDoc()); + if (randomBoolean()) { + shard.refresh("test"); + } + } + for (int i = 0; i < numInitialDocs; i++) { + final String id = Integer.toString(i); + if (randomBoolean()) { + if (rarely()) { + deleteDoc(shard, "_doc", id); + } else { + indexDoc(shard, "_doc", id, randomDoc()); + } + } + if (frequently()) { + shard.refresh("test"); + } + } + SnapshotId snapshotId = new SnapshotId("test", "test"); + IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); + SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository(), false); + repository.start(); + try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); + repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef.getIndexCommit(), indexShardSnapshotStatus); + IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); + assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); + assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); + } + shard.refresh("test"); + ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), true, + ShardRoutingState.INITIALIZING, + new RecoverySource.SnapshotRecoverySource(new Snapshot("src_only", snapshotId), Version.CURRENT, indexId.getId())); + IndexShard restoredShard = newShard(shardRouting); + restoredShard.mapperService().merge(shard.indexSettings().getIndexMetaData(), MapperService.MergeReason.MAPPING_RECOVERY); + DiscoveryNode discoveryNode = new DiscoveryNode("node_g", buildNewFakeTransportAddress(), Version.CURRENT); + restoredShard.markAsRecovering("test from snap", new RecoveryState(restoredShard.routingEntry(), discoveryNode, null)); + assertTrue(restoredShard.restoreFromRepository(repository)); + assertEquals(restoredShard.recoveryState().getStage(), RecoveryState.Stage.DONE); + assertEquals(restoredShard.recoveryState().getTranslog().recoveredOperations(), shard.docStats().getCount()); + assertEquals(IndexShardState.POST_RECOVERY, restoredShard.state()); + restoredShard.refresh("test"); + assertEquals(restoredShard.docStats().getCount(), shard.docStats().getCount()); + assertEquals(0, restoredShard.docStats().getDeleted()); + for (int i = 0; i < numInitialDocs; i++) { + Engine.Get get = new Engine.Get(false, false, "_doc", Integer.toString(i), new Term("_id", Uid.encodeId(Integer.toString(i)))); + Engine.GetResult original = shard.get(get); + Engine.GetResult restored = restoredShard.get(get); + assertEquals(original.exists(), restored.exists()); + if (original.exists()) { + Document document = original.docIdAndVersion().reader.document(original.docIdAndVersion().docId); + Document restoredDocument = restored.docIdAndVersion().reader.document(restored.docIdAndVersion().docId); + for (IndexableField field : document) { + assertEquals(document.get(field.name()), restoredDocument.get(field.name())); + } + } + IOUtils.close(original, restored); + } + + closeShards(shard, restoredShard); + } + + + /** Create a {@link Environment} with random path.home and path.repo **/ + private Environment createEnvironment() { + Path home = createTempDir(); + return TestEnvironment.newEnvironment(Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), home.toAbsolutePath()) + .put(Environment.PATH_REPO_SETTING.getKey(), home.resolve("repo").toAbsolutePath()) + .build()); + } + + /** Create a {@link Repository} with a random name **/ + private Repository createRepository() throws IOException { + Settings settings = Settings.builder().put("location", randomAlphaOfLength(10)).build(); + RepositoryMetaData repositoryMetaData = new RepositoryMetaData(randomAlphaOfLength(10), FsRepository.TYPE, settings); + return new FsRepository(repositoryMetaData, createEnvironment(), xContentRegistry()); + } + +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java new file mode 100644 index 0000000000000..c03e77f90522a --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java @@ -0,0 +1,220 @@ +/* + * 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.snapshots; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.FloatPoint; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.document.StringField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FilterMergePolicy; +import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy; +import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.SegmentCommitInfo; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.SnapshotDeletionPolicy; +import org.apache.lucene.index.StandardDirectoryReader; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.store.Directory; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; + +public class SourceOnlySnapshotTests extends ESTestCase { + public void testSourceOnlyRandom() throws IOException { + try (Directory dir = newDirectory(); Directory targetDir = newDirectory()) { + SnapshotDeletionPolicy deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); + try (RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig().setIndexDeletionPolicy + (deletionPolicy))) { + SourceOnlySnapshot snapshoter = new SourceOnlySnapshot(targetDir, writer.w.getConfig().getSoftDeletesField()); + writer.commit(); + int numDocs = scaledRandomIntBetween(100, 10000); + boolean appendOnly = randomBoolean(); + for (int i = 0; i < numDocs; i++) { + int docId = appendOnly ? i : randomIntBetween(0, 100); + Document d = newRandomDocument(i); + if (appendOnly) { + writer.addDocument(d); + } else { + writer.updateDocument(new Term("id", Integer.toString(docId)), d); + } + if (rarely()) { + if (randomBoolean()) { + writer.commit(); + } + IndexCommit snapshot = deletionPolicy.snapshot(); + try { + snapshoter.syncSnapshot(snapshot); + } finally { + deletionPolicy.release(snapshot); + } + } + } + if (randomBoolean()) { + writer.commit(); + } + IndexCommit snapshot = deletionPolicy.snapshot(); + try { + snapshoter.syncSnapshot(snapshot); + try (DirectoryReader snapReader = snapshoter.wrapReader(DirectoryReader.open(targetDir)); + DirectoryReader reader = snapshoter.wrapReader(DirectoryReader.open(snapshot))) { + assertEquals(snapReader.maxDoc(), reader.maxDoc()); + assertEquals(snapReader.numDocs(), reader.numDocs()); + for (int i = 0; i < snapReader.maxDoc(); i++) { + assertEquals(snapReader.document(i).get("_source"), reader.document(i).get("_source")); + } + } + } finally { + deletionPolicy.release(snapshot); + } + } + } + } + + private Document newRandomDocument(int id) { + Document doc = new Document(); + doc.add(new StringField("id", Integer.toString(id), Field.Store.YES)); + doc.add(new NumericDocValuesField("id", id)); + if (randomBoolean()) { + doc.add(new TextField("text", "the quick brown fox", Field.Store.NO)); + } + if (randomBoolean()) { + doc.add(new FloatPoint("float_point", 1.3f, 3.4f)); + } + doc.add(new StoredField("_source", randomRealisticUnicodeOfCodepointLengthBetween(5, 10))); + return doc; + } + + public void testSrcOnlySnap() throws IOException { + try (Directory dir = newDirectory()) { + SnapshotDeletionPolicy deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); + IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig() + .setSoftDeletesField("id") + .setIndexDeletionPolicy(deletionPolicy).setMergePolicy(new FilterMergePolicy(NoMergePolicy.INSTANCE) { + @Override + public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, MergeContext mergeContext) { + return randomBoolean(); + } + })); + Document doc = new Document(); + doc.add(new StringField("id", "1", Field.Store.YES)); + doc.add(new TextField("text", "the quick brown fox", Field.Store.NO)); + doc.add(new NumericDocValuesField("rank", 1)); + doc.add(new StoredField("src", "the quick brown fox")); + writer.addDocument(doc); + doc = new Document(); + doc.add(new StringField("id", "2", Field.Store.YES)); + doc.add(new TextField("text", "the quick blue fox", Field.Store.NO)); + doc.add(new NumericDocValuesField("rank", 2)); + doc.add(new StoredField("src", "the quick blue fox")); + doc.add(new StoredField("dummy", "foo")); // add a field only this segment has + writer.addDocument(doc); + writer.flush(); + doc = new Document(); + doc.add(new StringField("id", "1", Field.Store.YES)); + doc.add(new TextField("text", "the quick brown fox", Field.Store.NO)); + doc.add(new NumericDocValuesField("rank", 3)); + doc.add(new StoredField("src", "the quick brown fox")); + writer.softUpdateDocument(new Term("id", "1"), doc, new NumericDocValuesField("id", 1)); + writer.commit(); + IndexCommit snapshot = deletionPolicy.snapshot(); + Directory targetDir = newDirectory(); + SourceOnlySnapshot snapshoter = new SourceOnlySnapshot(targetDir, "id"); + snapshoter.syncSnapshot(snapshot); + + + StandardDirectoryReader reader = (StandardDirectoryReader) DirectoryReader.open(snapshot); + try(DirectoryReader snapReader = DirectoryReader.open(targetDir)) { + assertEquals(snapReader.maxDoc(), 3); + assertEquals(snapReader.numDocs(), 2); + for (int i = 0; i < 3; i++) { + assertEquals(snapReader.document(i).get("src"), reader.document(i).get("src")); + } + IndexSearcher searcher = new IndexSearcher(snapReader); + TopDocs id = searcher.search(new TermQuery(new Term("id", "1")), 10); + assertEquals(0, id.totalHits); + } + + snapshoter = new SourceOnlySnapshot(targetDir, "id"); + snapshoter.syncSnapshot(snapshot); + assertEquals(0, snapshoter.getCreatedFiles().size()); + deletionPolicy.release(snapshot); + // now add another doc + doc = new Document(); + doc.add(new StringField("id", "4", Field.Store.YES)); + doc.add(new TextField("text", "the quick blue fox", Field.Store.NO)); + doc.add(new NumericDocValuesField("rank", 2)); + doc.add(new StoredField("src", "the quick blue fox")); + writer.addDocument(doc); + doc = new Document(); + doc.add(new StringField("id", "5", Field.Store.YES)); + doc.add(new TextField("text", "the quick blue fox", Field.Store.NO)); + doc.add(new NumericDocValuesField("rank", 2)); + doc.add(new StoredField("src", "the quick blue fox")); + writer.addDocument(doc); + writer.commit(); + { + snapshot = deletionPolicy.snapshot(); + snapshoter = new SourceOnlySnapshot(targetDir, "id"); + snapshoter.syncSnapshot(snapshot); + assertEquals(4, snapshoter.getCreatedFiles().size()); + for (String file : snapshoter.getCreatedFiles()) { + String extension = IndexFileNames.getExtension(file); + switch (extension) { + case "fdt": + case "fdx": + case "fnm": + case "si": + break; + default: + fail("unexpected extension: " + extension); + } + } + try(DirectoryReader snapReader = DirectoryReader.open(targetDir)) { + assertEquals(snapReader.maxDoc(), 5); + assertEquals(snapReader.numDocs(), 4); + } + deletionPolicy.release(snapshot); + } + writer.deleteDocuments(new Term("id", "5")); + writer.commit(); + { + snapshot = deletionPolicy.snapshot(); + snapshoter = new SourceOnlySnapshot(targetDir, "id"); + snapshoter.syncSnapshot(snapshot); + assertEquals(1, snapshoter.getCreatedFiles().size()); + for (String file : snapshoter.getCreatedFiles()) { + String extension = IndexFileNames.getExtension(file); + switch (extension) { + case "liv": + break; + default: + fail("unexpected extension: " + extension); + } + } + try(DirectoryReader snapReader = DirectoryReader.open(targetDir)) { + assertEquals(snapReader.maxDoc(), 5); + assertEquals(snapReader.numDocs(), 3); + } + deletionPolicy.release(snapshot); + } + writer.close(); + targetDir.close(); + reader.close(); + } + } + +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml new file mode 100644 index 0000000000000..bba6d78050fb2 --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml @@ -0,0 +1,73 @@ +--- +setup: + + - do: + snapshot.create_repository: + repository: test_repo_restore_1 + body: + type: source + settings: + delegate_type: fs + location: "test_repo_restore_1_loc" + + - do: + indices.create: + index: test_index + body: + settings: + number_of_shards: 1 + number_of_replicas: 0 + + - do: + cluster.health: + wait_for_status: green + +--- +"Create a source only snapshot and then restore it": + + - do: + index: + index: test_index + type: _doc + id: 1 + body: { foo: bar } + - do: + indices.flush: + index: test_index + + - do: + snapshot.create: + repository: test_repo_restore_1 + snapshot: test_snapshot + wait_for_completion: true + + - match: { snapshot.snapshot: test_snapshot } + - match: { snapshot.state : SUCCESS } + - match: { snapshot.shards.successful: 1 } + - match: { snapshot.shards.failed : 0 } + - is_true: snapshot.version + - gt: { snapshot.version_id: 0} + + - do: + indices.close: + index : test_index + + - do: + snapshot.restore: + repository: test_repo_restore_1 + snapshot: test_snapshot + wait_for_completion: true + + - do: + indices.recovery: + index: test_index + + - match: { test_index.shards.0.type: SNAPSHOT } + - match: { test_index.shards.0.stage: DONE } + - match: { test_index.shards.0.translog.recovered: 1} + - match: { test_index.shards.0.translog.total: 1} + - match: { test_index.shards.0.translog.total_on_start: 1} + - match: { test_index.shards.0.index.files.recovered: 5} + - match: { test_index.shards.0.index.files.reused: 0} + - match: { test_index.shards.0.index.size.reused_in_bytes: 0} + - gt: { test_index.shards.0.index.size.recovered_in_bytes: 0} From 3c5a0ed82acaa4de90e7725db80d74ae6deea6da Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 14 Aug 2018 16:04:04 +0200 Subject: [PATCH 02/44] add license --- .../repositories/FilterRepository.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java index 4b621b6e91666..c421318f9bc15 100644 --- a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java @@ -1,3 +1,21 @@ +/* + * 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.repositories; import org.apache.lucene.index.IndexCommit; From 0645159c5485cc0517699dbf6ce2f488dc0e6d6b Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 15 Aug 2018 09:28:16 +0200 Subject: [PATCH 03/44] fix docs test setup --- docs/plugins/repository-source-only.asciidoc | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/plugins/repository-source-only.asciidoc b/docs/plugins/repository-source-only.asciidoc index 83409249a0ad1..ea3596edca2df 100644 --- a/docs/plugins/repository-source-only.asciidoc +++ b/docs/plugins/repository-source-only.asciidoc @@ -10,11 +10,11 @@ to create incremental, storage optimized, and minimal snapshots of an index. ==== Configuration The `_source` only repository always requires a delegate repository to be used as it's storage backend. -In order to use the `fs` respository: +In order to use the `fs` repository: [source,js] ----------------------------------- -PUT _snapshot/my_src-only_repository +PUT _snapshot/my_src_only_repository { "type": "source", "settings": { @@ -24,7 +24,6 @@ PUT _snapshot/my_src-only_repository } ----------------------------------- // CONSOLE -// TESTSETUP Since the `_source` only repository doesn't snapshot any index or doc-values structures but only stored fields and index metadata, it's required to reindex the data during the restore process. This can either happen From b28407f55ad985f7540a3943890c04248beddd47 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 15 Aug 2018 16:41:52 +0200 Subject: [PATCH 04/44] fix imports --- .../snapshots/SourceOnlySnapshotRepository.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index ae84b65adb13c..0e8dbc12c69a7 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -21,16 +21,11 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.env.ShardLock; -import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.fieldvisitor.FieldsVisitor; -import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; From 5cca4c28bc30565bc19193196d82ba7f86170ce8 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 16 Aug 2018 10:36:23 +0200 Subject: [PATCH 05/44] make sure on local reindex we don't parse the source if it's not necessary, skip translog and use append only optimization --- .../elasticsearch/index/engine/Engine.java | 13 +++++++-- .../index/engine/InternalEngine.java | 17 ++++++------ .../index/mapper/DocumentParser.java | 27 ++++++++++++++----- .../index/mapper/SourceToParse.java | 22 ++++++++++----- .../elasticsearch/index/shard/IndexShard.java | 6 ++--- .../SourceOnlySnapshotRepository.java | 22 ++++++++++----- 6 files changed, 73 insertions(+), 34 deletions(-) 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 31da7afc51a10..0ad3d4196c807 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -1122,11 +1122,20 @@ public enum Origin { PRIMARY, REPLICA, PEER_RECOVERY, - LOCAL_TRANSLOG_RECOVERY; + LOCAL_TRANSLOG_RECOVERY, + LOCAL_REINDEX; public boolean isRecovery() { return this == PEER_RECOVERY || this == LOCAL_TRANSLOG_RECOVERY; } + + public boolean skipTranslog() { + return this == LOCAL_TRANSLOG_RECOVERY || this == LOCAL_REINDEX; + } + + public boolean isPrimary() { + return this == PRIMARY || this == LOCAL_REINDEX; + } } public Origin origin() { @@ -1178,7 +1187,7 @@ public static class Index extends Operation { public Index(Term uid, ParsedDocument doc, long seqNo, long primaryTerm, long version, VersionType versionType, Origin origin, long startTime, long autoGeneratedIdTimestamp, boolean isRetry) { super(uid, seqNo, primaryTerm, version, versionType, origin, startTime); - assert (origin == Origin.PRIMARY) == (versionType != null) : "invalid version_type=" + versionType + " for origin=" + origin; + assert (origin.isPrimary()) == (versionType != null) : "invalid version_type=" + versionType + " for origin=" + origin; this.doc = doc; this.isRetry = isRetry; this.autoGeneratedIdTimestamp = autoGeneratedIdTimestamp; 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 a30127a24ae21..bba75ee3951f3 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -686,6 +686,7 @@ private boolean canOptimizeAddDocument(Index index) { + index.getAutoGeneratedIdTimestamp(); switch (index.origin()) { case PRIMARY: + case LOCAL_REINDEX: assert (index.version() == Versions.MATCH_ANY && index.versionType() == VersionType.INTERNAL) : "version: " + index.version() + " type: " + index.versionType(); return true; @@ -705,7 +706,7 @@ private boolean canOptimizeAddDocument(Index index) { } private boolean assertIncomingSequenceNumber(final Engine.Operation.Origin origin, final long seqNo) { - if (origin == Operation.Origin.PRIMARY) { + if (origin.isPrimary()) { assert assertOriginPrimarySequenceNumber(seqNo); } else { // sequence number should be set when operation origin is not primary @@ -722,7 +723,7 @@ protected boolean assertOriginPrimarySequenceNumber(final long seqNo) { } private long generateSeqNoForOperation(final Operation operation) { - assert operation.origin() == Operation.Origin.PRIMARY; + assert operation.origin().isPrimary(); return doGenerateSeqNoForOperation(operation); } @@ -777,7 +778,7 @@ public IndexResult index(Index index) throws IOException { */ final IndexingStrategy plan; - if (index.origin() == Operation.Origin.PRIMARY) { + if (index.origin().isPrimary()) { plan = planIndexingAsPrimary(index); } else { // non-primary mode (i.e., replica or recovery) @@ -794,7 +795,7 @@ public IndexResult index(Index index) throws IOException { indexResult = new IndexResult( plan.versionForIndexing, getPrimaryTerm(), plan.seqNoForIndexing, plan.currentNotFoundOrDeleted); } - if (index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { + if (index.origin().skipTranslog() == false) { final Translog.Location location; if (indexResult.getResultType() == Result.Type.SUCCESS) { location = translog.add(new Translog.Index(index, indexResult)); @@ -877,7 +878,7 @@ private IndexingStrategy planIndexingAsNonPrimary(Index index) throws IOExceptio } private IndexingStrategy planIndexingAsPrimary(Index index) throws IOException { - assert index.origin() == Operation.Origin.PRIMARY : "planing as primary but origin isn't. got " + index.origin(); + assert index.origin().isPrimary() : "planing as primary but origin isn't. got " + index.origin(); final IndexingStrategy plan; // resolve an external operation into an internal one which is safe to replay if (canOptimizeAddDocument(index)) { @@ -1089,7 +1090,7 @@ public DeleteResult delete(Delete delete) throws IOException { ensureOpen(); lastWriteNanos = delete.startTime(); final DeletionStrategy plan; - if (delete.origin() == Operation.Origin.PRIMARY) { + if (delete.origin().isPrimary()) { plan = planDeletionAsPrimary(delete); } else { plan = planDeletionAsNonPrimary(delete); @@ -1133,7 +1134,7 @@ public DeleteResult delete(Delete delete) throws IOException { } private DeletionStrategy planDeletionAsNonPrimary(Delete delete) throws IOException { - assert delete.origin() != Operation.Origin.PRIMARY : "planing as primary but got " + delete.origin(); + assert delete.origin().isPrimary() == false : "planing as primary but got " + delete.origin(); maxSeqNoOfNonAppendOnlyOperations.updateAndGet(curr -> Math.max(delete.seqNo(), curr)); assert maxSeqNoOfNonAppendOnlyOperations.get() >= delete.seqNo() : "max_seqno of non-append-only was not updated;" + "max_seqno non-append-only [" + maxSeqNoOfNonAppendOnlyOperations.get() + "], seqno of delete [" + delete.seqNo() + "]"; @@ -1166,7 +1167,7 @@ private DeletionStrategy planDeletionAsNonPrimary(Delete delete) throws IOExcept } private DeletionStrategy planDeletionAsPrimary(Delete delete) throws IOException { - assert delete.origin() == Operation.Origin.PRIMARY : "planing as primary but got " + delete.origin(); + assert delete.origin().isPrimary(): "planing as primary but got " + delete.origin(); // resolve operation from external to internal final VersionValue versionValue = resolveDocVersion(delete); assert incrementVersionLookup(); 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 0fd156c090531..d1e9e155710f9 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -23,6 +23,8 @@ import org.apache.lucene.index.IndexableField; import org.elasticsearch.Version; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.joda.FormatDateTimeFormatter; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; @@ -55,19 +57,27 @@ final class DocumentParser { this.docMapper = docMapper; } + private static final BytesArray EMPTY_SOURCE = new BytesArray("{}"); ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { validateType(source); final Mapping mapping = docMapper.mapping(); final ParseContext.InternalParseContext context; + final boolean skipParserValidation = source.mustValidateSource() == false && mapping.root.isEnabled() == false; final XContentType xContentType = source.getXContentType(); - + final BytesReference sourceRef = source.source(); try (XContentParser parser = XContentHelper.createParser(docMapperParser.getXContentRegistry(), - LoggingDeprecationHandler.INSTANCE, source.source(), xContentType)) { + LoggingDeprecationHandler.INSTANCE, sourceRef , xContentType)) { context = new ParseContext.InternalParseContext(indexSettings.getSettings(), docMapperParser, docMapper, source, parser); - validateStart(parser); - internalParseDocument(mapping, context, parser); - validateEnd(parser); + if (skipParserValidation) { + // no need to parse the source here root mapping is disabled and source has been validated before ie. + // comes from a translog or a local reindex. This prevent calling parser.skipChildren which essentially parses the source + internalParseDocument(mapping, context, parser, false); + } else { + validateStart(parser); + internalParseDocument(mapping, context, parser, true); + validateEnd(parser); + } } catch (Exception e) { throw wrapInMapperParsingException(source, e); } @@ -81,7 +91,8 @@ ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException 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, ParseContext.InternalParseContext context, XContentParser parser, + boolean skipChildren) throws IOException { final boolean emptyDoc = isEmptyDoc(mapping, parser); for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { @@ -90,7 +101,9 @@ private static void internalParseDocument(Mapping mapping, ParseContext.Internal if (mapping.root.isEnabled() == false) { // entire type is disabled - parser.skipChildren(); + if (skipChildren) { + parser.skipChildren(); + } } else if (emptyDoc == false) { parseObjectOrNested(context, mapping.root); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/SourceToParse.java b/server/src/main/java/org/elasticsearch/index/mapper/SourceToParse.java index 0130207c0a78e..bab288ceac251 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SourceToParse.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SourceToParse.java @@ -29,7 +29,12 @@ public class SourceToParse { public static SourceToParse source(String index, String type, String id, BytesReference source, XContentType contentType) { - return new SourceToParse(index, type, id, source, contentType); + return new SourceToParse(index, type, id, source, contentType, true); + } + + public static SourceToParse source(String index, String type, String id, BytesReference source, + XContentType contentType, boolean mustValidateSource) { + return new SourceToParse(index, type, id, source, contentType, mustValidateSource); } private final BytesReference source; @@ -42,9 +47,12 @@ public static SourceToParse source(String index, String type, String id, BytesRe private String routing; + private final boolean mustValidateSource; + private XContentType xContentType; - private SourceToParse(String index, String type, String id, BytesReference source, XContentType xContentType) { + private SourceToParse(String index, String type, String id, BytesReference source, XContentType xContentType, + boolean mustValidateSource) { this.index = Objects.requireNonNull(index); this.type = Objects.requireNonNull(type); this.id = Objects.requireNonNull(id); @@ -52,6 +60,7 @@ private SourceToParse(String index, String type, String id, BytesReference sourc // so, we might as well do it here, and improve the performance of working with direct byte arrays this.source = new BytesArray(Objects.requireNonNull(source).toBytesRef()); this.xContentType = Objects.requireNonNull(xContentType); + this.mustValidateSource = mustValidateSource; } public BytesReference source() { @@ -62,6 +71,10 @@ public String index() { return this.index; } + public boolean mustValidateSource() { + return mustValidateSource; + } + public String type() { return this.type; } @@ -82,9 +95,4 @@ public SourceToParse routing(String routing) { this.routing = routing; return this; } - - public enum Origin { - PRIMARY, - REPLICA - } } 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 9131055bcd928..757383cc17084 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -674,7 +674,7 @@ public Engine.IndexResult applyIndexOperationOnReplica(long seqNo, long version, Engine.Operation.Origin.REPLICA, sourceToParse); } - private Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, long version, @Nullable VersionType versionType, + public Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, long version, @Nullable VersionType versionType, long autoGeneratedTimeStamp, boolean isRetry, Engine.Operation.Origin origin, SourceToParse sourceToParse) throws IOException { assert opPrimaryTerm <= this.operationPrimaryTerm: "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm @@ -1241,7 +1241,7 @@ public Engine.Result applyTranslogOperation(Translog.Operation operation, Engine result = applyIndexOperation(index.seqNo(), index.primaryTerm(), index.version(), null, index.getAutoGeneratedIdTimestamp(), true, origin, source(shardId.getIndexName(), index.type(), index.id(), index.source(), - XContentHelper.xContentType(index.source())).routing(index.routing())); + XContentHelper.xContentType(index.source()), true).routing(index.routing())); break; case DELETE: final Translog.Delete delete = (Translog.Delete) operation; @@ -1445,7 +1445,7 @@ private void ensureWriteAllowed(Engine.Operation.Origin origin) throws IllegalIn throw new IllegalIndexShardStateException(shardId, state, "operation only allowed when recovering, origin [" + origin + "]"); } } else { - if (origin == Engine.Operation.Origin.PRIMARY) { + if (origin.isPrimary()) { verifyPrimary(); } else { assert origin == Engine.Operation.Origin.REPLICA; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 0e8dbc12c69a7..1a6a353f213da 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -21,17 +21,20 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.env.ShardLock; +import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.fieldvisitor.FieldsVisitor; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.FilterRepository; import org.elasticsearch.repositories.IndexId; @@ -43,6 +46,8 @@ import java.util.Iterator; import java.util.function.Function; +import static org.elasticsearch.index.mapper.SourceToParse.source; + /** *

* This is a filter snapshot repository that only snapshots the minimal required information @@ -177,7 +182,8 @@ protected void closeInternal() { } @Override - public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { + public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, + RecoveryState recoveryState) { super.restoreShard(shard, snapshotId, version, indexId, snapshotShardId, recoveryState); ShardPath shardPath = shard.shardPath(); try { @@ -204,7 +210,7 @@ public void applyPostRestoreOps(IndexShard shard) throws IOException { RecoveryState.Translog state = shard.recoveryState().getTranslog(); assert state.totalOperations() == 0 : "translog state should have 0 total ops but got: " + state.totalOperations(); state.reset(); - long sequenceNumber = 0; + String index = shard.shardId().getIndexName(); try (FSDirectory dir = FSDirectory.open(restoreSourceCopy)) { try (IndexReader reader = DirectoryReader.open(dir)) { state.totalOperationsOnStart(reader.numDocs()); @@ -224,14 +230,16 @@ public void applyPostRestoreOps(IndexShard shard) throws IOException { Uid uid = rootFieldsVisitor.uid(); BytesReference source = rootFieldsVisitor.source(); if (source != null) { // nested fields don't have source. in this case we should be fine. - // TODO we should have a dedicated origin for this LOCAL_TRANSLOG_RECOVERY is misleading. - Engine.Result result = shard.applyTranslogOperation(new Translog.Index(uid.type(), uid.id(), - sequenceNumber++, primaryTerm, 1, source.toBytesRef().bytes, - rootFieldsVisitor.routing(), -1), Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY); + // we can use append-only optimization here since we know there won't be any duplicates! + Engine.Result result = shard.applyIndexOperation(SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm, + Versions.MATCH_ANY, VersionType.INTERNAL, 1, false, Engine.Operation.Origin.LOCAL_REINDEX, + source(index, uid.type(), uid.id(), source, + XContentHelper.xContentType(source), false).routing(rootFieldsVisitor.routing())); if (result.getResultType() != Engine.Result.Type.SUCCESS) { throw new IllegalStateException("failed applying post restore operation result: " + result .getResultType(), result.getFailure()); } + state.incrementRecoveredOperations(); } else { assert restoreMinimal // in this case we don't have nested in the mapping. From 6fc9664b909c53403b5031d97cb02006e9d06c5a Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 16 Aug 2018 10:37:16 +0200 Subject: [PATCH 06/44] remove dead code --- .../java/org/elasticsearch/index/mapper/DocumentParser.java | 3 --- 1 file changed, 3 deletions(-) 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 d1e9e155710f9..62b05c15e8043 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -23,7 +23,6 @@ import org.apache.lucene.index.IndexableField; import org.elasticsearch.Version; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.joda.FormatDateTimeFormatter; @@ -57,10 +56,8 @@ final class DocumentParser { this.docMapper = docMapper; } - private static final BytesArray EMPTY_SOURCE = new BytesArray("{}"); ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { validateType(source); - final Mapping mapping = docMapper.mapping(); final ParseContext.InternalParseContext context; final boolean skipParserValidation = source.mustValidateSource() == false && mapping.root.isEnabled() == false; From ea806bcb0f1bdc0c725154d5b2e3bd85f00c85ce Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 16 Aug 2018 15:23:32 +0200 Subject: [PATCH 07/44] fix imports --- .../elasticsearch/snapshots/SourceOnlySnapshotShardTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index a0cb2b291a8b0..05ebec25bcb10 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -3,7 +3,6 @@ import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; -import org.apache.lucene.store.IndexInput; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.RepositoryMetaData; From 02aecd7e6da981f3e4df05295355b317d5d4aeda Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 20 Aug 2018 13:42:57 +0200 Subject: [PATCH 08/44] status quo --- .../core/internal/io/IOUtils.java | 9 + .../elasticsearch/index/engine/Engine.java | 4 +- .../index/engine/EngineFactory.java | 1 + .../index/engine/EngineSearcher.java | 5 +- .../org/elasticsearch/index/store/Store.java | 1 - .../snapshots/SourceOnlySnapshot.java | 2 +- .../snapshots/SourceOnlySnapshotEngine.java | 477 ++++++++++++++++++ .../SourceOnlySnapshotRepository.java | 38 +- .../elasticsearch/xpack/core/XPackPlugin.java | 22 +- .../snapshots/SourceOnlySnapshotIT.java | 125 +++-- 10 files changed, 621 insertions(+), 63 deletions(-) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java diff --git a/libs/core/src/main/java/org/elasticsearch/core/internal/io/IOUtils.java b/libs/core/src/main/java/org/elasticsearch/core/internal/io/IOUtils.java index 67663516167d5..493d809f9dc33 100644 --- a/libs/core/src/main/java/org/elasticsearch/core/internal/io/IOUtils.java +++ b/libs/core/src/main/java/org/elasticsearch/core/internal/io/IOUtils.java @@ -20,6 +20,7 @@ import java.io.Closeable; import java.io.IOException; import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; import java.nio.file.FileVisitResult; import java.nio.file.FileVisitor; import java.nio.file.Files; @@ -36,6 +37,14 @@ */ public final class IOUtils { + /** + * UTF-8 charset string. + *

Where possible, use {@link StandardCharsets#UTF_8} instead, + * as using the String constant may slow things down. + * @see StandardCharsets#UTF_8 + */ + public static final String UTF_8 = StandardCharsets.UTF_8.name(); + private IOUtils() { // Static utils methods } 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 0ad3d4196c807..749acfcff2e57 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -1126,7 +1126,7 @@ public enum Origin { LOCAL_REINDEX; public boolean isRecovery() { - return this == PEER_RECOVERY || this == LOCAL_TRANSLOG_RECOVERY; + return this == PEER_RECOVERY || this == LOCAL_TRANSLOG_RECOVERY || this == LOCAL_REINDEX; } public boolean skipTranslog() { @@ -1561,7 +1561,7 @@ public static class IndexCommitRef implements Closeable { private final CheckedRunnable onClose; private final IndexCommit indexCommit; - IndexCommitRef(IndexCommit indexCommit, CheckedRunnable onClose) { + public IndexCommitRef(IndexCommit indexCommit, CheckedRunnable onClose) { this.indexCommit = indexCommit; this.onClose = onClose; } diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineFactory.java b/server/src/main/java/org/elasticsearch/index/engine/EngineFactory.java index b477e27b6e150..e50bdd86e7564 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineFactory.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineFactory.java @@ -21,6 +21,7 @@ /** * Simple Engine Factory */ +@FunctionalInterface public interface EngineFactory { Engine newReadWriteEngine(EngineConfig config); diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineSearcher.java b/server/src/main/java/org/elasticsearch/index/engine/EngineSearcher.java index 7fd0fe6cc3904..b888b9682b786 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineSearcher.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineSearcher.java @@ -31,13 +31,14 @@ /** * Searcher for an Engine */ -final class EngineSearcher extends Engine.Searcher { +public final class EngineSearcher extends Engine.Searcher { private final AtomicBoolean released = new AtomicBoolean(false); private final Store store; private final Logger logger; private final ReferenceManager referenceManager; - EngineSearcher(String source, ReferenceManager searcherReferenceManager, Store store, Logger logger) throws IOException { + public EngineSearcher(String source, ReferenceManager searcherReferenceManager, Store store, Logger logger) throws + IOException { super(source, searcherReferenceManager.acquire()); this.store = store; this.logger = logger; 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 ae1492f1a4995..9f9b8834aed8e 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -429,7 +429,6 @@ public final void decRef() { @Override public void close() { - if (isClosed.compareAndSet(false, true)) { // only do this once! decRef(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java index 19a42b15c9408..ef970860da3ca 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java @@ -29,8 +29,8 @@ import org.apache.lucene.store.TrackingDirectoryWrapper; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.IOUtils; import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.core.internal.io.IOUtils; import java.io.ByteArrayOutputStream; import java.io.IOException; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java new file mode 100644 index 0000000000000..d9aa447934518 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java @@ -0,0 +1,477 @@ +/* + * 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.snapshots; + +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.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.SearcherFactory; +import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.store.AlreadyClosedException; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; +import org.elasticsearch.common.util.concurrent.ReleasableLock; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.EngineConfig; +import org.elasticsearch.index.engine.EngineException; +import org.elasticsearch.index.engine.EngineSearcher; +import org.elasticsearch.index.engine.Segment; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.VersionFieldMapper; +import org.elasticsearch.index.seqno.SeqNoStats; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogStats; + +import java.io.Closeable; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.function.BiFunction; +import java.util.stream.Stream; + +public class SourceOnlySnapshotEngine extends Engine { + + private final SegmentInfos lastCommittedSegmentInfos; + private final SeqNoStats seqNoStats; + private final TranslogStats translogStats; + private final SearcherManager searcherManager; + private final IndexCommit indexCommit; + + public SourceOnlySnapshotEngine(EngineConfig config) { + super(config); + try { + Store store = config.getStore(); + store.incRef(); + DirectoryReader reader = null; + boolean success = false; + try { + this.lastCommittedSegmentInfos = Lucene.readSegmentInfos(store.directory()); + this.translogStats = new TranslogStats(0, 0, 0, 0, 0); + final SequenceNumbers.CommitInfo seqNoStats = + SequenceNumbers.loadSeqNoInfoFromLuceneCommit(lastCommittedSegmentInfos.userData.entrySet()); + long maxSeqNo = seqNoStats.maxSeqNo; + long localCheckpoint = seqNoStats.localCheckpoint; + this.seqNoStats = new SeqNoStats(maxSeqNo, localCheckpoint, localCheckpoint); + reader = SeqIdGeneratingDirectoryReader.wrap(ElasticsearchDirectoryReader.wrap(DirectoryReader + .open(store.directory()), config.getShardId()), config.getPrimaryTermSupplier().getAsLong()); + this.indexCommit = reader.getIndexCommit(); + this.searcherManager = new SearcherManager(reader, new SearcherFactory()); + success = true; + } finally { + if (success == false) { + IOUtils.close(reader, store::decRef); + } + } + } catch (IOException e) { + throw new UncheckedIOException(e); // this is stupid + } + } + + @Override + protected void closeNoLock(String reason, CountDownLatch closedLatch) { + if (isClosed.compareAndSet(false, true)) { + try { + IOUtils.close(searcherManager, store::decRef); + } catch (Exception ex) { + logger.warn("failed to close searcher", ex); + } finally { + closedLatch.countDown(); + } + } + } + + @Override + public GetResult get(Get get, BiFunction searcherFactory) throws EngineException { + throw new UnsupportedOperationException(); + } + + @Override + public Searcher acquireSearcher(String source, SearcherScope scope) throws EngineException { + store.incRef(); + Releasable releasable = store::decRef; + try (ReleasableLock ignored = readLock.acquire()) { + final EngineSearcher searcher = new EngineSearcher(source, searcherManager, store, logger); + releasable = null; // hand over the reference to the engine searcher + return searcher; + } catch (AlreadyClosedException ex) { + throw ex; + } catch (Exception ex) { + ensureOpen(ex); // throw AlreadyClosedException if it's closed + throw new EngineException(shardId, "failed to acquire searcher, source " + source, ex); + } finally { + Releasables.close(releasable); + } + } + + @Override + protected SegmentInfos getLastCommittedSegmentInfos() { + return lastCommittedSegmentInfos; + } + + @Override + public String getHistoryUUID() { + return lastCommittedSegmentInfos.userData.get(Engine.HISTORY_UUID_KEY); + } + + @Override + public long getWritingBytes() { + return 0; + } + + @Override + public long getIndexThrottleTimeInMillis() { + return 0; + } + + @Override + public boolean isThrottled() { + return false; + } + + @Override + public IndexResult index(Index index) { + throw new UnsupportedOperationException(); + } + + @Override + public DeleteResult delete(Delete delete) { + throw new UnsupportedOperationException(); + } + + @Override + public NoOpResult noOp(NoOp noOp) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isTranslogSyncNeeded() { + return false; + } + + @Override + public boolean ensureTranslogSynced(Stream locations) { + return false; + } + + @Override + public void syncTranslog() { + } + + @Override + public Closeable acquireTranslogRetentionLock() { + return () -> {}; + } + + @Override + public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) { + return new Translog.Snapshot() { + + @Override + public void close() throws IOException { + } + + @Override + public int totalOperations() { + return 0; + } + + @Override + public Translog.Operation next() throws IOException { + return null; + } + }; + } + + @Override + public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { + return 0; + } + + @Override + public TranslogStats getTranslogStats() { + return translogStats; + } + + @Override + public Translog.Location getTranslogLastWriteLocation() { + throw new UnsupportedOperationException(); + } + + @Override + public long getLocalCheckpoint() { + return seqNoStats.getLocalCheckpoint(); + } + + @Override + public void waitForOpsToComplete(long seqNo) { + } + + @Override + public void resetLocalCheckpoint(long newCheckpoint) { + throw new UnsupportedOperationException(); + } + + @Override + public SeqNoStats getSeqNoStats(long globalCheckpoint) { + return new SeqNoStats(seqNoStats.getMaxSeqNo(), seqNoStats.getLocalCheckpoint(), globalCheckpoint); + } + + @Override + public long getLastSyncedGlobalCheckpoint() { + return seqNoStats.getGlobalCheckpoint(); + } + + @Override + public long getIndexBufferRAMBytesUsed() { + return 0; + } + + @Override + public List segments(boolean verbose) { + return Arrays.asList(getSegmentInfo(lastCommittedSegmentInfos, verbose)); + } + + @Override + public void refresh(String source) throws EngineException { + } + + @Override + public void writeIndexingBuffer() throws EngineException { + + } + + @Override + public boolean shouldPeriodicallyFlush() { + return false; + } + + @Override + public SyncedFlushResult syncFlush(String syncId, CommitId expectedCommitId) throws EngineException { + throw new UnsupportedOperationException(); + } + + @Override + public CommitId flush(boolean force, boolean waitIfOngoing) throws EngineException { + throw new UnsupportedOperationException(); + } + + @Override + public CommitId flush() throws EngineException { + throw new UnsupportedOperationException(); + } + + @Override + public void forceMerge(boolean flush, int maxNumSegments, boolean onlyExpungeDeletes, + boolean upgrade, boolean upgradeOnlyAncientSegments) { + throw new UnsupportedOperationException(); + } + + @Override + public IndexCommitRef acquireLastIndexCommit(boolean flushFirst) { + store.incRef(); + return new IndexCommitRef(indexCommit, store::decRef); + } + + @Override + public IndexCommitRef acquireSafeIndexCommit() { + return acquireLastIndexCommit(false); + } + + @Override + public void activateThrottling() { + } + + @Override + public void deactivateThrottling() { + } + + @Override + public void trimUnreferencedTranslogFiles() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean shouldRollTranslogGeneration() { + return false; + } + + @Override + public void rollTranslogGeneration() throws EngineException { + throw new UnsupportedOperationException(); + } + + @Override + public void restoreLocalCheckpointFromTranslog() { + throw new UnsupportedOperationException(); + } + + @Override + public int fillSeqNoGaps(long primaryTerm) { + return 0; + } + + @Override + public Engine recoverFromTranslog() { + return this; + } + + @Override + public void skipTranslogRecovery() { + } + + @Override + public void trimOperationsFromTranslog(long belowTerm, long aboveSeqNo) throws EngineException { + } + + @Override + public void maybePruneDeletes() { + } + + + + private static final class SeqIdGeneratingDirectoryReader extends FilterDirectoryReader { + private final long primaryTerm; + + public SeqIdGeneratingDirectoryReader(DirectoryReader in, SeqIdGeneratingSubReaderWrapper wrapper) throws IOException { + super(in, wrapper); + primaryTerm = wrapper.primaryTerm; + } + + @Override + protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { + return wrap(in, primaryTerm); + } + + public static DirectoryReader wrap(DirectoryReader in, long primaryTerm) throws IOException { + Map ctxMap = new IdentityHashMap<>(); + for (LeafReaderContext leave : in.leaves()) { + ctxMap.put(leave.reader(), leave); + } + return new SeqIdGeneratingDirectoryReader(in, new SeqIdGeneratingSubReaderWrapper(ctxMap, primaryTerm)); + } + + @Override + public CacheHelper getReaderCacheHelper() { + return in.getReaderCacheHelper(); + } + + private static abstract class FakeNumericDocValues extends NumericDocValues { + private final int maxDoc; + int docID = -1; + + public FakeNumericDocValues(int maxDoc) { + this.maxDoc = maxDoc; + } + + @Override + public int docID() { + return docID; + } + + @Override + public int nextDoc() { + if (docID+1 < maxDoc) { + docID++; + } else { + docID = NO_MORE_DOCS; + } + return docID; + } + + @Override + public int advance(int target) { + if (target >= maxDoc) { + docID = NO_MORE_DOCS; + } else { + docID = target; + } + return docID; + } + + @Override + public long cost() { + return maxDoc; + } + + @Override + public boolean advanceExact(int target) { + advance(target); + return docID != NO_MORE_DOCS; + } + } + + private static class SeqIdGeneratingSubReaderWrapper extends SubReaderWrapper { + private final Map ctxMap; + private final long primaryTerm; + + public SeqIdGeneratingSubReaderWrapper(Map ctxMap, long primaryTerm) { + this.ctxMap = ctxMap; + this.primaryTerm = primaryTerm; + } + + @Override + public LeafReader wrap(LeafReader reader) { + LeafReaderContext leafReaderContext = ctxMap.get(reader); + final int docBase = leafReaderContext.docBase; + return new FilterLeafReader(reader) { + + @Override + public NumericDocValues getNumericDocValues(String field) throws IOException { + if (SeqNoFieldMapper.NAME.equals(field)) { + return new FakeNumericDocValues(maxDoc()) { + @Override + public long longValue() { + return docBase + docID; + } + }; + } else if (SeqNoFieldMapper.PRIMARY_TERM_NAME.equals(field)) { + return new FakeNumericDocValues(maxDoc()) { + @Override + public long longValue() { + return primaryTerm; + } + }; + } else if (VersionFieldMapper.NAME.equals(field)) { + return new FakeNumericDocValues(maxDoc()) { + @Override + public long longValue() { + return 1; + } + }; + } + return super.getNumericDocValues(field); + } + + @Override + public CacheHelper getCoreCacheHelper() { + return reader.getCoreCacheHelper(); + } + + @Override + public CacheHelper getReaderCacheHelper() { + return reader.getReaderCacheHelper(); + } + }; + } + } + } +} + diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 1a6a353f213da..73b62c4c26486 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.env.ShardLock; import org.elasticsearch.index.VersionType; @@ -83,6 +84,9 @@ public final class SourceOnlySnapshotRepository extends FilterRepository { public static final Setting RESTORE_MINIMAL = Setting.boolSetting("restore_minimal", false, Setting.Property.NodeScope); + public static final Setting SOURCE_ONLY_ENGINE = Setting.boolSetting("index.require_source_only_engine", false, Setting + .Property.IndexScope, Setting.Property.InternalIndex, Setting.Property.Final); + public static final String SNAPSHOT_DIR_NAME = "_snapshot"; public static final String RESTORE_DIR_NAME = "_restore"; private final boolean restoreMinimal; @@ -136,9 +140,9 @@ public IndexMetaData getSnapshotIndexMetaData(SnapshotId snapshotId, IndexId ind } else { mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + " } }"; } - System.out.println(mapping); builder.putMapping(next.key, mapping); } + builder.settings(Settings.builder().put(snapshotIndexMetaData.getSettings()).put(SOURCE_ONLY_ENGINE.getKey(), true)); return builder.build(); } else { return snapshotIndexMetaData; @@ -185,26 +189,32 @@ protected void closeInternal() { public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { super.restoreShard(shard, snapshotId, version, indexId, snapshotShardId, recoveryState); - ShardPath shardPath = shard.shardPath(); - try { - Path restoreSourceCopy = shardPath.getDataPath().resolve(RESTORE_DIR_NAME); - try (HardlinkCopyDirectoryWrapper wrapper = new HardlinkCopyDirectoryWrapper(FSDirectory.open(restoreSourceCopy))) { - Lucene.cleanLuceneIndex(wrapper); - SegmentInfos segmentInfos = shard.store().readLastCommittedSegmentsInfo(); - for (String file : segmentInfos.files(true)) { - wrapper.copyFrom(shard.store().directory(), file, file, IOContext.DEFAULT); + if (restoreMinimal == false) { + ShardPath shardPath = shard.shardPath(); + try { + Path restoreSourceCopy = shardPath.getDataPath().resolve(RESTORE_DIR_NAME); + try (HardlinkCopyDirectoryWrapper wrapper = new HardlinkCopyDirectoryWrapper(FSDirectory.open(restoreSourceCopy))) { + Lucene.cleanLuceneIndex(wrapper); + SegmentInfos segmentInfos = shard.store().readLastCommittedSegmentsInfo(); + for (String file : segmentInfos.files(true)) { + wrapper.copyFrom(shard.store().directory(), file, file, IOContext.DEFAULT); + } } + Lucene.cleanLuceneIndex(shard.store().directory()); // wipe the old index + shard.store().createEmpty(); + } catch (IOException ex) { + // why on earth does this super method not declare IOException + throw new UncheckedIOException(ex); } - Lucene.cleanLuceneIndex(shard.store().directory()); // wipe the old index - shard.store().createEmpty(); - } catch (IOException ex) { - // why on earth does this super method not declare IOException - throw new UncheckedIOException(ex); } } @Override public void applyPostRestoreOps(IndexShard shard) throws IOException { + if (restoreMinimal) { + return; + } + ShardPath shardPath = shard.shardPath(); Path restoreSourceCopy = shardPath.getDataPath().resolve(RESTORE_DIR_NAME); RecoveryState.Translog state = shard.recoveryState().getTranslog(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java index e22ee73fa66c6..5c0d11fdd51c3 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java @@ -31,16 +31,20 @@ import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.IndexScopedSettings; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.license.LicenseService; import org.elasticsearch.license.LicensesMetaData; import org.elasticsearch.license.Licensing; import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.persistent.PersistentTaskParams; +import org.elasticsearch.plugins.EnginePlugin; import org.elasticsearch.plugins.ExtensiblePlugin; import org.elasticsearch.plugins.RepositoryPlugin; import org.elasticsearch.plugins.ScriptPlugin; @@ -48,6 +52,7 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.snapshots.SourceOnlySnapshotEngine; import org.elasticsearch.snapshots.SourceOnlySnapshotRepository; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; @@ -78,7 +83,7 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; -public class XPackPlugin extends XPackClientPlugin implements ScriptPlugin, ExtensiblePlugin, RepositoryPlugin { +public class XPackPlugin extends XPackClientPlugin implements ScriptPlugin, ExtensiblePlugin, RepositoryPlugin, EnginePlugin { private static Logger logger = ESLoggerFactory.getLogger(XPackPlugin.class); private static DeprecationLogger deprecationLogger = new DeprecationLogger(logger); @@ -350,4 +355,19 @@ public Map getRepositories(Environment env, NamedXCo return Collections.singletonMap("source", SourceOnlySnapshotRepository.newFactory()); } + @Override + public Optional getEngineFactory(IndexSettings indexSettings) { + if (indexSettings.getValue(SourceOnlySnapshotRepository.SOURCE_ONLY_ENGINE)) { + EngineFactory engineFactory = SourceOnlySnapshotEngine::new; + return Optional.of(engineFactory); + } + return Optional.empty(); + } + + @Override + public List> getSettings() { + List> settings = super.getSettings(); + settings.add(SourceOnlySnapshotRepository.SOURCE_ONLY_ENGINE); + return settings; + } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index a076801b0418d..1e9acdaa59b84 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -11,12 +11,17 @@ import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.env.Environment; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.MockEngineFactoryPlugin; +import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.node.Node; +import org.elasticsearch.plugins.EnginePlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.RepositoryPlugin; import org.elasticsearch.repositories.Repository; @@ -24,12 +29,15 @@ import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.engine.MockEngineFactory; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.ExecutionException; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -44,11 +52,35 @@ protected Collection> nodePlugins() { return classes; } - public static final class MyPlugin extends Plugin implements RepositoryPlugin { + @Override + protected Collection> getMockPlugins() { + Collection> mockPlugins = super.getMockPlugins(); + Collection> classes = new ArrayList<>(super.getMockPlugins()); + classes.remove(MockEngineFactoryPlugin.class); + return classes; + } + + public static final class MyPlugin extends Plugin implements RepositoryPlugin, EnginePlugin { @Override public Map getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry) { return Collections.singletonMap("source", SourceOnlySnapshotRepository.newFactory()); } + @Override + public Optional getEngineFactory(IndexSettings indexSettings) { + if (indexSettings.getValue(SourceOnlySnapshotRepository.SOURCE_ONLY_ENGINE)) { + EngineFactory engineFactory = SourceOnlySnapshotEngine::new; + return Optional.of(engineFactory); + } + return Optional.empty(); + } + + @Override + public List> getSettings() { + List> settings = new ArrayList<>(super.getSettings()); + settings.add(SourceOnlySnapshotRepository.SOURCE_ONLY_ENGINE); + return settings; + } + } /** @@ -82,48 +114,57 @@ public void testSnapshotAndRestore() throws Exception { */ public void testSnapshotAndRestoreMinimal() throws Exception { final String sourceIdx = "test-idx"; - boolean requireRouting = randomBoolean(); - IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting); - - SearchResponse searchResponse = client().prepareSearch(sourceIdx) - .addSort(SeqNoFieldMapper.NAME, SortOrder.ASC) - .setSize(builders.length).get(); - SearchHits hits = searchResponse.getHits(); - assertEquals(builders.length, hits.totalHits); - long i = 0; - for (SearchHit hit : hits) { - String id = hit.getId(); - Map sourceAsMap = hit.getSourceAsMap(); - assertTrue(sourceAsMap.containsKey("field1")); - assertEquals(i++, hit.getSortValues()[0]); - assertEquals("bar " + id, sourceAsMap.get("field1")); - assertEquals("r" + id, hit.field("_routing").getValue()); - } - GetMappingsResponse getMappingsResponse = client().admin().indices().prepareGetMappings(sourceIdx).get(); - ImmutableOpenMap mapping = getMappingsResponse - .getMappings().get(sourceIdx); - assertTrue(mapping.containsKey("_doc")); - if (requireRouting) { - assertEquals("{\"_doc\":{\"enabled\":false," + - "\"_meta\":{\"_doc\":{\"_routing\":{\"required\":true}," + - "\"properties\":{\"field1\":{\"type\":\"text\"," + - "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}}}}," + - "\"_routing\":{\"required\":true}}}", mapping.get("_doc").source().string()); - } else { - assertEquals("{\"_doc\":{\"enabled\":false," + - "\"_meta\":{\"_doc\":{\"properties\":{\"field1\":{\"type\":\"text\"," + - "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}}}}}}", mapping.get("_doc").source().string()); + try { + boolean requireRouting = randomBoolean(); + IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting); + + SearchResponse searchResponse = client().prepareSearch(sourceIdx) + .addSort(SeqNoFieldMapper.NAME, SortOrder.ASC) + .setSize(builders.length).get(); + SearchHits hits = searchResponse.getHits(); + assertEquals(builders.length, hits.totalHits); + long i = 0; + for (SearchHit hit : hits) { + String id = hit.getId(); + Map sourceAsMap = hit.getSourceAsMap(); + assertTrue(sourceAsMap.containsKey("field1")); + assertEquals(i++, hit.getSortValues()[0]); + assertEquals("bar " + id, sourceAsMap.get("field1")); + assertEquals("r" + id, hit.field("_routing").getValue()); + } + GetMappingsResponse getMappingsResponse = client().admin().indices().prepareGetMappings(sourceIdx).get(); + ImmutableOpenMap mapping = getMappingsResponse + .getMappings().get(sourceIdx); + assertTrue(mapping.containsKey("_doc")); + if (requireRouting) { + assertEquals("{\"_doc\":{\"enabled\":false," + + "\"_meta\":{\"_doc\":{\"_routing\":{\"required\":true}," + + "\"properties\":{\"field1\":{\"type\":\"text\"," + + "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}}}}," + + "\"_routing\":{\"required\":true}}}", mapping.get("_doc").source().string()); + } else { + assertEquals("{\"_doc\":{\"enabled\":false," + + "\"_meta\":{\"_doc\":{\"properties\":{\"field1\":{\"type\":\"text\"," + + "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}}}}}}", mapping.get("_doc").source().string()); + } +// assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() +// .addIds("" + randomIntBetween(0, builders.length))).get(), 1); +// // ensure we can not find hits it's a minimal restore +// assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); +// // make sure deletes work +// String idToDelete = "" + randomIntBetween(0, builders.length); +// DeleteResponse deleteResponse = client().prepareDelete(sourceIdx, "_doc", idToDelete).setRouting("r" + idToDelete).get(); +// assertEquals(DocWriteResponse.Result.DELETED, deleteResponse.getResult()); +// refresh(sourceIdx); +// assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery().addIds(idToDelete)).get(), 0); + internalCluster().ensureAtLeastNumDataNodes(2); + client().admin().indices().prepareUpdateSettings(sourceIdx).setSettings(Settings.builder().put("index.number_of_replicas", 1)) + .get(); + ensureGreen(sourceIdx); + } finally { + client().admin().indices().prepareDelete(sourceIdx).get(); } - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() - .addIds("" + randomIntBetween(0, builders.length))).get(), 1); - // ensure we can not find hits it's a minimal restore - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); - // make sure deletes work - String idToDelete = "" + randomIntBetween(0, builders.length); - DeleteResponse deleteResponse = client().prepareDelete(sourceIdx, "_doc", idToDelete).setRouting("r" + idToDelete).get(); - assertEquals(DocWriteResponse.Result.DELETED, deleteResponse.getResult()); - refresh(sourceIdx); - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery().addIds(idToDelete)).get(), 0); + } private IndexRequestBuilder[] snashotAndRestore(String sourceIdx, int numShards, boolean minimal, boolean requireRouting) From 7dcc25dfd3d46233c4a9227af82136d3cec5d23d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 27 Aug 2018 21:24:52 +0200 Subject: [PATCH 09/44] iteration --- .../snapshots/SourceOnlySnapshot.java | 81 +++++++++++++++++-- .../snapshots/SourceOnlySnapshotEngine.java | 2 +- .../snapshots/SourceOnlySnapshotTests.java | 16 +++- 3 files changed, 86 insertions(+), 13 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java index ef970860da3ca..322c3ad8407a2 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java @@ -22,6 +22,12 @@ import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SoftDeletesDirectoryReaderWrapper; import org.apache.lucene.index.StandardDirectoryReader; +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.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexOutput; @@ -29,6 +35,7 @@ import org.apache.lucene.store.TrackingDirectoryWrapper; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.FixedBitSet; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.core.internal.io.IOUtils; @@ -40,6 +47,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.Supplier; import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.FIELDS_EXTENSION; import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.FIELDS_INDEX_EXTENSION; @@ -48,10 +56,16 @@ public final class SourceOnlySnapshot { private final Directory targetDirectory; private final String softDeletesField; private final List createdFiles = new ArrayList<>(); + private final Supplier filterDocsQuerySupplier; - public SourceOnlySnapshot(Directory targetDirectory, String softDeletesField) { + public SourceOnlySnapshot(Directory targetDirectory, String softDeletesField, Supplier filterDocsQuerySupplier) { this.targetDirectory = targetDirectory; this.softDeletesField = softDeletesField; + this.filterDocsQuerySupplier = filterDocsQuerySupplier; + } + + public SourceOnlySnapshot(Directory targetDirectory, String softDeletesField) { + this(targetDirectory, softDeletesField, null); } public List getCreatedFiles() { @@ -79,7 +93,7 @@ public synchronized void syncSnapshot(IndexCommit commit) throws IOException { for (LeafReaderContext ctx : wrapper.leaves()) { SegmentCommitInfo info = segmentInfos.info(ctx.ord); LeafReader leafReader = ctx.reader(); - Bits liveDocs = leafReader.getLiveDocs(); + LiveDocs liveDocs = getLiveDocs(leafReader); SegmentCommitInfo newInfo = syncSegment(info, liveDocs, leafReader.getFieldInfos(), existingSegments); newInfos.add(newInfo); } @@ -100,6 +114,48 @@ public synchronized void syncSnapshot(IndexCommit commit) throws IOException { assert assertCheckIndex(); } + private LiveDocs getLiveDocs(LeafReader reader) throws IOException { + if (filterDocsQuerySupplier != null) { + Query query = filterDocsQuerySupplier.get(); + IndexSearcher s = new IndexSearcher(reader); + s.setQueryCache(null); + Weight weight = s.createWeight(query, false, 1.0f); + Scorer scorer = weight.scorer(reader.getContext()); + if (scorer != null) { + DocIdSetIterator iterator = scorer.iterator(); + if (iterator != null) { + Bits liveDocs = reader.getLiveDocs(); + final FixedBitSet bits; + if (liveDocs != null) { + bits = FixedBitSet.copyOf(liveDocs); + } else { + bits = new FixedBitSet(reader.maxDoc()); + bits.set(0, reader.maxDoc()); + } + int newDeletes = apply(iterator, bits); + if (newDeletes != 0) { + int numDeletes = reader.numDeletedDocs() + newDeletes; + return new LiveDocs(numDeletes, bits); + } + } + } + } + return new LiveDocs(reader.numDeletedDocs(), reader.getLiveDocs()); + } + + private int apply(DocIdSetIterator iterator, FixedBitSet bits) throws IOException { + int docID = -1; + int newDeletes = 0; + while ((docID = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { + if (bits.get(docID)) { + bits.clear(docID); + newDeletes++; + } + } + return newDeletes; + } + + private boolean assertCheckIndex() throws IOException { ByteArrayOutputStream output = new ByteArrayOutputStream(1024); try (CheckIndex checkIndex = new CheckIndex(targetDirectory)) { @@ -118,7 +174,7 @@ DirectoryReader wrapReader(DirectoryReader reader) throws IOException { return softDeletesField == null ? reader : new SoftDeletesDirectoryReaderWrapper(reader, softDeletesField); } - private SegmentCommitInfo syncSegment(SegmentCommitInfo segmentCommitInfo, Bits liveDocs, FieldInfos fieldInfos, + private SegmentCommitInfo syncSegment(SegmentCommitInfo segmentCommitInfo, LiveDocs liveDocs, FieldInfos fieldInfos, Map existingSegments) throws IOException { SegmentInfo si = segmentCommitInfo.info; Codec codec = si.getCodec(); @@ -154,14 +210,13 @@ private SegmentCommitInfo syncSegment(SegmentCommitInfo segmentCommitInfo, Bits newInfo = existingSegments.get(segmentId); assert newInfo.info.getUseCompoundFile() == false; } - int deletes = segmentCommitInfo.getDelCount() + segmentCommitInfo.getSoftDelCount(); - if (liveDocs != null && deletes != 0 && deletes != newInfo.getDelCount()) { + if (liveDocs.bits != null && liveDocs.numDeletes != 0 && liveDocs.numDeletes != newInfo.getDelCount()) { if (newInfo.getDelCount() != 0) { - assert assertLiveDocs(liveDocs, deletes); + assert assertLiveDocs(liveDocs.bits, liveDocs.numDeletes); } - codec.liveDocsFormat().writeLiveDocs(liveDocs, trackingDir, newInfo, deletes - newInfo.getDelCount(), + codec.liveDocsFormat().writeLiveDocs(liveDocs.bits, trackingDir, newInfo, liveDocs.numDeletes - newInfo.getDelCount(), IOContext.DEFAULT); - SegmentCommitInfo info = new SegmentCommitInfo(newInfo.info, deletes, 0, newInfo.getNextDelGen(), -1, -1); + SegmentCommitInfo info = new SegmentCommitInfo(newInfo.info, liveDocs.numDeletes, 0, newInfo.getNextDelGen(), -1, -1); info.setFieldInfosFiles(newInfo.getFieldInfosFiles()); info.info.setFiles(trackingDir.getCreatedFiles()); newInfo = info; @@ -185,4 +240,14 @@ private boolean assertLiveDocs(Bits liveDocs, int deletes) { assert actualDeletes == deletes : " actual: " + actualDeletes + " deletes: " + deletes; return true; } + + private static class LiveDocs { + final int numDeletes; + final Bits bits; + + LiveDocs(int numDeletes, Bits bits) { + this.numDeletes = numDeletes; + this.bits = bits; + } + } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java index d9aa447934518..7df496bde0d8e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java @@ -329,7 +329,7 @@ public int fillSeqNoGaps(long primaryTerm) { } @Override - public Engine recoverFromTranslog() { + public Engine recoverFromTranslog(long upto) { return this; } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java index c03e77f90522a..9a40cb5f32c05 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java @@ -23,8 +23,10 @@ import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SnapshotDeletionPolicy; +import org.apache.lucene.index.SoftDeletesDirectoryReaderWrapper; import org.apache.lucene.index.StandardDirectoryReader; import org.apache.lucene.index.Term; +import org.apache.lucene.search.DocValuesFieldExistsQuery; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; @@ -39,13 +41,15 @@ public void testSourceOnlyRandom() throws IOException { SnapshotDeletionPolicy deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); try (RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig().setIndexDeletionPolicy (deletionPolicy))) { - SourceOnlySnapshot snapshoter = new SourceOnlySnapshot(targetDir, writer.w.getConfig().getSoftDeletesField()); + boolean modifyDeletedDocs = true; randomBoolean(); + SourceOnlySnapshot snapshoter = new SourceOnlySnapshot(targetDir, writer.w.getConfig().getSoftDeletesField(), + modifyDeletedDocs? () -> new DocValuesFieldExistsQuery("some_values") : null); writer.commit(); int numDocs = scaledRandomIntBetween(100, 10000); boolean appendOnly = randomBoolean(); for (int i = 0; i < numDocs; i++) { int docId = appendOnly ? i : randomIntBetween(0, 100); - Document d = newRandomDocument(i); + Document d = newRandomDocument(docId); if (appendOnly) { writer.addDocument(d); } else { @@ -70,7 +74,9 @@ public void testSourceOnlyRandom() throws IOException { try { snapshoter.syncSnapshot(snapshot); try (DirectoryReader snapReader = snapshoter.wrapReader(DirectoryReader.open(targetDir)); - DirectoryReader reader = snapshoter.wrapReader(DirectoryReader.open(snapshot))) { + DirectoryReader wrappedReader = snapshoter.wrapReader(DirectoryReader.open(snapshot))) { + DirectoryReader reader = modifyDeletedDocs ? new SoftDeletesDirectoryReaderWrapper(wrappedReader, "some_value") : + wrappedReader; assertEquals(snapReader.maxDoc(), reader.maxDoc()); assertEquals(snapReader.numDocs(), reader.numDocs()); for (int i = 0; i < snapReader.maxDoc(); i++) { @@ -94,6 +100,9 @@ private Document newRandomDocument(int id) { if (randomBoolean()) { doc.add(new FloatPoint("float_point", 1.3f, 3.4f)); } + if (randomBoolean()) { + doc.add(new NumericDocValuesField("some_value", randomLong())); + } doc.add(new StoredField("_source", randomRealisticUnicodeOfCodepointLengthBetween(5, 10))); return doc; } @@ -216,5 +225,4 @@ public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, reader.close(); } } - } From 81c8127d1801a134d8c8f981ad7f595f841fb367 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 28 Aug 2018 17:51:12 +0200 Subject: [PATCH 10/44] Restore from a soruce only snap by copying only the source --- .../elasticsearch/index/engine/Engine.java | 15 +- .../index/engine/InternalEngine.java | 17 +- .../elasticsearch/index/shard/IndexShard.java | 4 +- .../index/shard/StoreRecovery.java | 1 - .../repositories/FilterRepository.java | 5 - .../repositories/Repository.java | 5 - .../snapshots/SourceOnlySnapshot.java | 30 ++- .../snapshots/SourceOnlySnapshotEngine.java | 3 +- .../SourceOnlySnapshotRepository.java | 144 +++----------- .../snapshots/SourceOnlySnapshotIT.java | 185 +++++++++++------- .../SourceOnlySnapshotShardTests.java | 156 +++++++++++++-- .../snapshots/SourceOnlySnapshotTests.java | 29 +-- .../rest-api-spec/test/snapshot/10_basic.yml | 6 +- 13 files changed, 324 insertions(+), 276 deletions(-) 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 97b2ef1f9cebd..843780059ffc1 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -1122,19 +1122,10 @@ public enum Origin { PRIMARY, REPLICA, PEER_RECOVERY, - LOCAL_TRANSLOG_RECOVERY, - LOCAL_REINDEX; + LOCAL_TRANSLOG_RECOVERY; public boolean isRecovery() { - return this == PEER_RECOVERY || this == LOCAL_TRANSLOG_RECOVERY || this == LOCAL_REINDEX; - } - - public boolean skipTranslog() { - return this == LOCAL_TRANSLOG_RECOVERY || this == LOCAL_REINDEX; - } - - public boolean isPrimary() { - return this == PRIMARY || this == LOCAL_REINDEX; + return this == PEER_RECOVERY || this == LOCAL_TRANSLOG_RECOVERY; } } @@ -1187,7 +1178,7 @@ public static class Index extends Operation { public Index(Term uid, ParsedDocument doc, long seqNo, long primaryTerm, long version, VersionType versionType, Origin origin, long startTime, long autoGeneratedIdTimestamp, boolean isRetry) { super(uid, seqNo, primaryTerm, version, versionType, origin, startTime); - assert (origin.isPrimary()) == (versionType != null) : "invalid version_type=" + versionType + " for origin=" + origin; + assert (origin == Origin.PRIMARY) == (versionType != null) : "invalid version_type=" + versionType + " for origin=" + origin; this.doc = doc; this.isRetry = isRetry; this.autoGeneratedIdTimestamp = autoGeneratedIdTimestamp; 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 dfc26da5c918b..c4c6792bf46a5 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -688,7 +688,6 @@ private boolean canOptimizeAddDocument(Index index) { + index.getAutoGeneratedIdTimestamp(); switch (index.origin()) { case PRIMARY: - case LOCAL_REINDEX: assert (index.version() == Versions.MATCH_ANY && index.versionType() == VersionType.INTERNAL) : "version: " + index.version() + " type: " + index.versionType(); return true; @@ -708,7 +707,7 @@ private boolean canOptimizeAddDocument(Index index) { } private boolean assertIncomingSequenceNumber(final Engine.Operation.Origin origin, final long seqNo) { - if (origin.isPrimary()) { + if (origin == Operation.Origin.PRIMARY) { assert assertOriginPrimarySequenceNumber(seqNo); } else { // sequence number should be set when operation origin is not primary @@ -725,7 +724,7 @@ protected boolean assertOriginPrimarySequenceNumber(final long seqNo) { } private long generateSeqNoForOperation(final Operation operation) { - assert operation.origin().isPrimary(); + assert operation.origin() == Operation.Origin.PRIMARY; return doGenerateSeqNoForOperation(operation); } @@ -780,7 +779,7 @@ public IndexResult index(Index index) throws IOException { */ final IndexingStrategy plan; - if (index.origin().isPrimary()) { + if (index.origin() == Operation.Origin.PRIMARY) { plan = planIndexingAsPrimary(index); } else { // non-primary mode (i.e., replica or recovery) @@ -797,7 +796,7 @@ public IndexResult index(Index index) throws IOException { indexResult = new IndexResult( plan.versionForIndexing, getPrimaryTerm(), plan.seqNoForIndexing, plan.currentNotFoundOrDeleted); } - if (index.origin().skipTranslog() == false) { + if (index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { final Translog.Location location; if (indexResult.getResultType() == Result.Type.SUCCESS) { location = translog.add(new Translog.Index(index, indexResult)); @@ -880,7 +879,7 @@ private IndexingStrategy planIndexingAsNonPrimary(Index index) throws IOExceptio } private IndexingStrategy planIndexingAsPrimary(Index index) throws IOException { - assert index.origin().isPrimary() : "planing as primary but origin isn't. got " + index.origin(); + assert index.origin() == Operation.Origin.PRIMARY : "planing as primary but origin isn't. got " + index.origin(); final IndexingStrategy plan; // resolve an external operation into an internal one which is safe to replay if (canOptimizeAddDocument(index)) { @@ -1092,7 +1091,7 @@ public DeleteResult delete(Delete delete) throws IOException { ensureOpen(); lastWriteNanos = delete.startTime(); final DeletionStrategy plan; - if (delete.origin().isPrimary()) { + if (delete.origin() == Operation.Origin.PRIMARY) { plan = planDeletionAsPrimary(delete); } else { plan = planDeletionAsNonPrimary(delete); @@ -1136,7 +1135,7 @@ public DeleteResult delete(Delete delete) throws IOException { } private DeletionStrategy planDeletionAsNonPrimary(Delete delete) throws IOException { - assert delete.origin().isPrimary() == false : "planing as primary but got " + delete.origin(); + assert delete.origin() != Operation.Origin.PRIMARY : "planing as primary but got " + delete.origin(); maxSeqNoOfNonAppendOnlyOperations.updateAndGet(curr -> Math.max(delete.seqNo(), curr)); assert maxSeqNoOfNonAppendOnlyOperations.get() >= delete.seqNo() : "max_seqno of non-append-only was not updated;" + "max_seqno non-append-only [" + maxSeqNoOfNonAppendOnlyOperations.get() + "], seqno of delete [" + delete.seqNo() + "]"; @@ -1169,7 +1168,7 @@ private DeletionStrategy planDeletionAsNonPrimary(Delete delete) throws IOExcept } private DeletionStrategy planDeletionAsPrimary(Delete delete) throws IOException { - assert delete.origin().isPrimary(): "planing as primary but got " + delete.origin(); + assert delete.origin() == Operation.Origin.PRIMARY : "planing as primary but got " + delete.origin(); // resolve operation from external to internal final VersionValue versionValue = resolveDocVersion(delete); assert incrementVersionLookup(); 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 502bd578a7817..9796b4ab23ca0 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -680,7 +680,7 @@ public Engine.IndexResult applyIndexOperationOnReplica(long seqNo, long version, Engine.Operation.Origin.REPLICA, sourceToParse); } - public Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, long version, @Nullable VersionType versionType, + private Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, long version, @Nullable VersionType versionType, long autoGeneratedTimeStamp, boolean isRetry, Engine.Operation.Origin origin, SourceToParse sourceToParse) throws IOException { assert opPrimaryTerm <= this.operationPrimaryTerm: "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm @@ -1451,7 +1451,7 @@ private void ensureWriteAllowed(Engine.Operation.Origin origin) throws IllegalIn throw new IllegalIndexShardStateException(shardId, state, "operation only allowed when recovering, origin [" + origin + "]"); } } else { - if (origin.isPrimary()) { + if (origin == Engine.Operation.Origin.PRIMARY) { assert assertPrimaryMode(); } else { assert origin == Engine.Operation.Origin.REPLICA; 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 d0030c6e4005d..e9acfe3d8b06f 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -464,7 +464,6 @@ private void restore(final IndexShard indexShard, final Repository repository, f assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; indexShard.openEngineAndRecoverFromTranslog(); indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm()); - repository.applyPostRestoreOps(indexShard); indexShard.finalizeRecovery(); indexShard.postRecovery("restore done"); } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java index c421318f9bc15..4e8e9b6c7f569 100644 --- a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java @@ -135,11 +135,6 @@ public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Ve return in.getShardSnapshotStatus(snapshotId, version, indexId, shardId); } - @Override - public void applyPostRestoreOps(IndexShard shard) throws IOException { - in.applyPostRestoreOps(shard); - } - @Override public Lifecycle.State lifecycleState() { return in.lifecycleState(); diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index 60055d235ad77..9f16d26ac7595 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -218,11 +218,6 @@ void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, IndexId */ void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState); - /** - * This allows a repository to apply operations after the snapshot has been restored as part of the translog recovery phase. - */ - default void applyPostRestoreOps(IndexShard shard) throws IOException {} - /** * Retrieve shard snapshot status for the stored snapshot * diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java index 322c3ad8407a2..3453394ae95a9 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java @@ -23,7 +23,6 @@ import org.apache.lucene.index.SoftDeletesDirectoryReaderWrapper; import org.apache.lucene.index.StandardDirectoryReader; 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; @@ -55,24 +54,19 @@ public final class SourceOnlySnapshot { private final Directory targetDirectory; private final String softDeletesField; - private final List createdFiles = new ArrayList<>(); - private final Supplier filterDocsQuerySupplier; + private final Supplier deleteByQuerySupplier; - public SourceOnlySnapshot(Directory targetDirectory, String softDeletesField, Supplier filterDocsQuerySupplier) { + public SourceOnlySnapshot(Directory targetDirectory, String softDeletesField, Supplier deleteByQuerySupplier) { this.targetDirectory = targetDirectory; this.softDeletesField = softDeletesField; - this.filterDocsQuerySupplier = filterDocsQuerySupplier; + this.deleteByQuerySupplier = deleteByQuerySupplier; } public SourceOnlySnapshot(Directory targetDirectory, String softDeletesField) { this(targetDirectory, softDeletesField, null); } - public List getCreatedFiles() { - return createdFiles; - } - - public synchronized void syncSnapshot(IndexCommit commit) throws IOException { + public synchronized List syncSnapshot(IndexCommit commit) throws IOException { long generation; Map existingSegments = new HashMap<>(); if (Lucene.indexExists(targetDirectory)) { @@ -84,6 +78,7 @@ public synchronized void syncSnapshot(IndexCommit commit) throws IOException { } else { generation = 1; } + List createdFiles = new ArrayList<>(); String segmentFileName; try (Lock writeLock = targetDirectory.obtainLock(IndexWriter.WRITE_LOCK_NAME); StandardDirectoryReader reader = (StandardDirectoryReader) DirectoryReader.open(commit)) { @@ -94,7 +89,7 @@ public synchronized void syncSnapshot(IndexCommit commit) throws IOException { SegmentCommitInfo info = segmentInfos.info(ctx.ord); LeafReader leafReader = ctx.reader(); LiveDocs liveDocs = getLiveDocs(leafReader); - SegmentCommitInfo newInfo = syncSegment(info, liveDocs, leafReader.getFieldInfos(), existingSegments); + SegmentCommitInfo newInfo = syncSegment(info, liveDocs, leafReader.getFieldInfos(), existingSegments, createdFiles); newInfos.add(newInfo); } segmentInfos.clear(); @@ -112,14 +107,18 @@ public synchronized void syncSnapshot(IndexCommit commit) throws IOException { } Lucene.pruneUnreferencedFiles(segmentFileName, targetDirectory); assert assertCheckIndex(); + return Collections.unmodifiableList(createdFiles); } private LiveDocs getLiveDocs(LeafReader reader) throws IOException { - if (filterDocsQuerySupplier != null) { - Query query = filterDocsQuerySupplier.get(); + if (deleteByQuerySupplier != null) { + // we have this additional delete by query functionality to filter out documents before we snapshot them + // we can't filter after the fact since we don't have an index anymore. + Query query = deleteByQuerySupplier.get(); IndexSearcher s = new IndexSearcher(reader); s.setQueryCache(null); - Weight weight = s.createWeight(query, false, 1.0f); + Query rewrite = s.rewrite(query); + Weight weight = s.createWeight(rewrite, false, 1.0f); Scorer scorer = weight.scorer(reader.getContext()); if (scorer != null) { DocIdSetIterator iterator = scorer.iterator(); @@ -175,7 +174,7 @@ DirectoryReader wrapReader(DirectoryReader reader) throws IOException { } private SegmentCommitInfo syncSegment(SegmentCommitInfo segmentCommitInfo, LiveDocs liveDocs, FieldInfos fieldInfos, - Map existingSegments) throws IOException { + Map existingSegments, List createdFiles) throws IOException { SegmentInfo si = segmentCommitInfo.info; Codec codec = si.getCodec(); final String segmentSuffix = ""; @@ -227,7 +226,6 @@ private SegmentCommitInfo syncSegment(SegmentCommitInfo segmentCommitInfo, LiveD } createdFiles.addAll(trackingDir.getCreatedFiles()); return newInfo; - } private boolean assertLiveDocs(Bits liveDocs, int deletes) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java index 7df496bde0d8e..43d973774d000 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java @@ -13,7 +13,6 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.SearcherFactory; import org.apache.lucene.search.SearcherManager; import org.apache.lucene.store.AlreadyClosedException; @@ -47,7 +46,7 @@ import java.util.function.BiFunction; import java.util.stream.Stream; -public class SourceOnlySnapshotEngine extends Engine { +public final class SourceOnlySnapshotEngine extends Engine { private final SegmentInfos lastCommittedSegmentInfos; private final SeqNoStats seqNoStats; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 73b62c4c26486..83f11bbf3e8b7 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -7,6 +7,7 @@ import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.search.Query; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.HardlinkCopyDirectoryWrapper; import org.apache.lucene.store.IOContext; @@ -21,6 +22,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; @@ -46,6 +48,7 @@ import java.nio.file.Path; import java.util.Iterator; import java.util.function.Function; +import java.util.function.Supplier; import static org.elasticsearch.index.mapper.SourceToParse.source; @@ -80,16 +83,10 @@ public final class SourceOnlySnapshotRepository extends FilterRepository { public static final Setting DELEGATE_TYPE = new Setting<>("delegate_type", "", Function.identity(), Setting.Property.NodeScope); - - public static final Setting RESTORE_MINIMAL = Setting.boolSetting("restore_minimal", - false, Setting.Property.NodeScope); - public static final Setting SOURCE_ONLY_ENGINE = Setting.boolSetting("index.require_source_only_engine", false, Setting .Property.IndexScope, Setting.Property.InternalIndex, Setting.Property.Final); public static final String SNAPSHOT_DIR_NAME = "_snapshot"; - public static final String RESTORE_DIR_NAME = "_restore"; - private final boolean restoreMinimal; public static Repository.Factory newFactory() { return new Repository.Factory() { @@ -105,49 +102,44 @@ public Repository create(RepositoryMetaData metaData, Function mappings = snapshotIndexMetaData.getMappings(); - Iterator> iterator = mappings.iterator(); - IndexMetaData.Builder builder = IndexMetaData.builder(snapshotIndexMetaData); - while (iterator.hasNext()) { - ObjectObjectCursor next = iterator.next(); - MappingMetaData.Routing routing = next.value.routing(); - final String mapping; - if (routing.required()) { // we have to respect the routing to be on the safe side so we pass this one on. - mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + ", " + - "\"_routing\" : { \"required\" : true } } }"; - } else { - mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + " } }"; - } - builder.putMapping(next.key, mapping); + // TODO: can we lie about the index.version.created here and produce an index with a new version since we reindex anyway? + + // for a minimal restore we basically disable indexing on all fields and only create an index + // that is fully functional from an operational perspective. ie. it will have all metadata fields like version/ + // seqID etc. and an indexed ID field such that we can potentially perform updates on them or delete documents. + ImmutableOpenMap mappings = snapshotIndexMetaData.getMappings(); + Iterator> iterator = mappings.iterator(); + IndexMetaData.Builder builder = IndexMetaData.builder(snapshotIndexMetaData); + while (iterator.hasNext()) { + ObjectObjectCursor next = iterator.next(); + MappingMetaData.Routing routing = next.value.routing(); + final String mapping; + if (routing.required()) { // we have to respect the routing to be on the safe side so we pass this one on. + mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + ", " + + "\"_routing\" : { \"required\" : true } } }"; + } else { + mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + " } }"; } - builder.settings(Settings.builder().put(snapshotIndexMetaData.getSettings()).put(SOURCE_ONLY_ENGINE.getKey(), true)); - return builder.build(); - } else { - return snapshotIndexMetaData; + builder.putMapping(next.key, mapping); } - + builder.settings(Settings.builder().put(snapshotIndexMetaData.getSettings()) + .put(SOURCE_ONLY_ENGINE.getKey(), true) + .put("index.blocks.write", true)); // read-only! + return builder.build(); } @Override @@ -169,7 +161,8 @@ protected void closeInternal() { // do nothing; } }, Store.OnClose.EMPTY); - SourceOnlySnapshot snapshot = new SourceOnlySnapshot(tempStore.directory(), null); + Supplier querySupplier = shard.mapperService().hasNested() ? () -> Queries.newNestedFilter() : null; + SourceOnlySnapshot snapshot = new SourceOnlySnapshot(tempStore.directory(), null, querySupplier); snapshot.syncSnapshot(snapshotIndexCommit); store.incRef(); try (DirectoryReader reader = DirectoryReader.open(tempStore.directory()); @@ -184,83 +177,4 @@ protected void closeInternal() { throw new UncheckedIOException(e); } } - - @Override - public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, - RecoveryState recoveryState) { - super.restoreShard(shard, snapshotId, version, indexId, snapshotShardId, recoveryState); - if (restoreMinimal == false) { - ShardPath shardPath = shard.shardPath(); - try { - Path restoreSourceCopy = shardPath.getDataPath().resolve(RESTORE_DIR_NAME); - try (HardlinkCopyDirectoryWrapper wrapper = new HardlinkCopyDirectoryWrapper(FSDirectory.open(restoreSourceCopy))) { - Lucene.cleanLuceneIndex(wrapper); - SegmentInfos segmentInfos = shard.store().readLastCommittedSegmentsInfo(); - for (String file : segmentInfos.files(true)) { - wrapper.copyFrom(shard.store().directory(), file, file, IOContext.DEFAULT); - } - } - Lucene.cleanLuceneIndex(shard.store().directory()); // wipe the old index - shard.store().createEmpty(); - } catch (IOException ex) { - // why on earth does this super method not declare IOException - throw new UncheckedIOException(ex); - } - } - } - - @Override - public void applyPostRestoreOps(IndexShard shard) throws IOException { - if (restoreMinimal) { - return; - } - - ShardPath shardPath = shard.shardPath(); - Path restoreSourceCopy = shardPath.getDataPath().resolve(RESTORE_DIR_NAME); - RecoveryState.Translog state = shard.recoveryState().getTranslog(); - assert state.totalOperations() == 0 : "translog state should have 0 total ops but got: " + state.totalOperations(); - state.reset(); - String index = shard.shardId().getIndexName(); - try (FSDirectory dir = FSDirectory.open(restoreSourceCopy)) { - try (IndexReader reader = DirectoryReader.open(dir)) { - state.totalOperationsOnStart(reader.numDocs()); - state.totalOperations(reader.numDocs()); - long primaryTerm = shard.getPendingPrimaryTerm(); - FieldsVisitor rootFieldsVisitor = new FieldsVisitor(true); - for (LeafReaderContext ctx : reader.leaves()) { - LeafReader leafReader = ctx.reader(); - Bits liveDocs = leafReader.getLiveDocs(); - // TODO: we could do this in parallel per segment here or even per docID - // there is a lot of room for doing this multi-threaded but not for the first iteration - for (int i = 0; i < leafReader.maxDoc(); i++) { - if (liveDocs == null || liveDocs.get(i)) { - rootFieldsVisitor.reset(); - leafReader.document(i, rootFieldsVisitor); - rootFieldsVisitor.postProcess(shard.mapperService()); - Uid uid = rootFieldsVisitor.uid(); - BytesReference source = rootFieldsVisitor.source(); - if (source != null) { // nested fields don't have source. in this case we should be fine. - // we can use append-only optimization here since we know there won't be any duplicates! - Engine.Result result = shard.applyIndexOperation(SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm, - Versions.MATCH_ANY, VersionType.INTERNAL, 1, false, Engine.Operation.Origin.LOCAL_REINDEX, - source(index, uid.type(), uid.id(), source, - XContentHelper.xContentType(source), false).routing(rootFieldsVisitor.routing())); - if (result.getResultType() != Engine.Result.Type.SUCCESS) { - throw new IllegalStateException("failed applying post restore operation result: " + result - .getResultType(), result.getFailure()); - } - - state.incrementRecoveredOperations(); - } else { - assert restoreMinimal // in this case we don't have nested in the mapping. - || shard.mapperService().hasNested() : "_source is null but shard has no nested docs"; - } - } - } - } - shard.flush(new FlushRequest()); - } - Lucene.cleanLuceneIndex(dir); // clear the tmp index; - } - } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index 1e9acdaa59b84..5c38b65c51191 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -1,19 +1,20 @@ package org.elasticsearch.snapshots; -import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; -import org.elasticsearch.action.delete.DeleteResponse; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.env.Environment; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.MockEngineFactoryPlugin; @@ -29,10 +30,11 @@ import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.engine.MockEngineFactory; +import org.hamcrest.Matchers; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -40,6 +42,7 @@ import java.util.Optional; import java.util.concurrent.ExecutionException; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; @@ -54,7 +57,6 @@ protected Collection> nodePlugins() { @Override protected Collection> getMockPlugins() { - Collection> mockPlugins = super.getMockPlugins(); Collection> classes = new ArrayList<>(super.getMockPlugins()); classes.remove(MockEngineFactoryPlugin.class); return classes; @@ -80,94 +82,110 @@ public List> getSettings() { settings.add(SourceOnlySnapshotRepository.SOURCE_ONLY_ENGINE); return settings; } - } - /** - * Tests that a source only index snapshot - */ public void testSnapshotAndRestore() throws Exception { final String sourceIdx = "test-idx"; - IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, false, false); - - SearchResponse searchResponse = client().prepareSearch(sourceIdx).setSize(builders.length) - .addSort(SeqNoFieldMapper.NAME, SortOrder.ASC).get(); - SearchHits hits = searchResponse.getHits(); - assertEquals(builders.length, hits.totalHits); - long i = 0; - for (SearchHit hit : hits) { - String id = hit.getId(); - Map sourceAsMap = hit.getSourceAsMap(); - assertTrue(sourceAsMap.containsKey("field1")); - assertEquals(i++, hit.getSortValues()[0]); - assertEquals("bar "+id, sourceAsMap.get("field1")); - assertEquals("r"+id, hit.field("_routing").getValue()); + try { + boolean requireRouting = randomBoolean(); + boolean useNested = randomBoolean(); + IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting, useNested); + assertHits(sourceIdx, builders.length); + assertMappings(sourceIdx, requireRouting, useNested); + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() + .addIds("" + randomIntBetween(0, builders.length))).get(), 0); + // ensure we can not find hits it's a minimal restore + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); + // make sure deletes work + String idToDelete = "" + randomIntBetween(0, builders.length); + expectThrows(ClusterBlockException.class, () -> client().prepareDelete(sourceIdx, "_doc", idToDelete) + .setRouting("r" + idToDelete).get()); + internalCluster().ensureAtLeastNumDataNodes(2); + client().admin().indices().prepareUpdateSettings(sourceIdx) + .setSettings(Settings.builder().put("index.number_of_replicas", 1)).get(); + ensureGreen(sourceIdx); + assertHits(sourceIdx, builders.length); + } finally { + client().admin().indices().prepareDelete(sourceIdx).get(); } - // ensure we can find hits - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() - .addIds("" + randomIntBetween(0, builders.length))).get(), 1); - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), builders.length); } - /** - * Tests that a source only index snapshot - */ - public void testSnapshotAndRestoreMinimal() throws Exception { + public void testSnapshotAndRestoreWithNested() throws Exception { final String sourceIdx = "test-idx"; try { boolean requireRouting = randomBoolean(); - IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting); - - SearchResponse searchResponse = client().prepareSearch(sourceIdx) - .addSort(SeqNoFieldMapper.NAME, SortOrder.ASC) - .setSize(builders.length).get(); - SearchHits hits = searchResponse.getHits(); - assertEquals(builders.length, hits.totalHits); - long i = 0; - for (SearchHit hit : hits) { - String id = hit.getId(); - Map sourceAsMap = hit.getSourceAsMap(); - assertTrue(sourceAsMap.containsKey("field1")); - assertEquals(i++, hit.getSortValues()[0]); - assertEquals("bar " + id, sourceAsMap.get("field1")); - assertEquals("r" + id, hit.field("_routing").getValue()); - } - GetMappingsResponse getMappingsResponse = client().admin().indices().prepareGetMappings(sourceIdx).get(); - ImmutableOpenMap mapping = getMappingsResponse - .getMappings().get(sourceIdx); - assertTrue(mapping.containsKey("_doc")); - if (requireRouting) { - assertEquals("{\"_doc\":{\"enabled\":false," + - "\"_meta\":{\"_doc\":{\"_routing\":{\"required\":true}," + - "\"properties\":{\"field1\":{\"type\":\"text\"," + - "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}}}}," + - "\"_routing\":{\"required\":true}}}", mapping.get("_doc").source().string()); - } else { - assertEquals("{\"_doc\":{\"enabled\":false," + - "\"_meta\":{\"_doc\":{\"properties\":{\"field1\":{\"type\":\"text\"," + - "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}}}}}}", mapping.get("_doc").source().string()); - } -// assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() -// .addIds("" + randomIntBetween(0, builders.length))).get(), 1); -// // ensure we can not find hits it's a minimal restore -// assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); -// // make sure deletes work -// String idToDelete = "" + randomIntBetween(0, builders.length); -// DeleteResponse deleteResponse = client().prepareDelete(sourceIdx, "_doc", idToDelete).setRouting("r" + idToDelete).get(); -// assertEquals(DocWriteResponse.Result.DELETED, deleteResponse.getResult()); -// refresh(sourceIdx); -// assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery().addIds(idToDelete)).get(), 0); + IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting, true); + IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats().clear().setDocs(true).get(); + assertThat(indicesStatsResponse.getTotal().docs.getDeleted(), Matchers.greaterThan(0l)); + assertHits(sourceIdx, builders.length); + assertMappings(sourceIdx, requireRouting, true); + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() + .addIds("" + randomIntBetween(0, builders.length))).get(), 0); + // ensure we can not find hits it's a minimal restore + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); + // make sure deletes work + String idToDelete = "" + randomIntBetween(0, builders.length); + expectThrows(ClusterBlockException.class, () -> client().prepareDelete(sourceIdx, "_doc", idToDelete) + .setRouting("r" + idToDelete).get()); internalCluster().ensureAtLeastNumDataNodes(2); client().admin().indices().prepareUpdateSettings(sourceIdx).setSettings(Settings.builder().put("index.number_of_replicas", 1)) .get(); ensureGreen(sourceIdx); + assertHits(sourceIdx, builders.length); } finally { client().admin().indices().prepareDelete(sourceIdx).get(); } + } + private void assertMappings(String sourceIdx, boolean requireRouting, boolean useNested) throws IOException { + GetMappingsResponse getMappingsResponse = client().admin().indices().prepareGetMappings(sourceIdx).get(); + ImmutableOpenMap mapping = getMappingsResponse + .getMappings().get(sourceIdx); + assertTrue(mapping.containsKey("_doc")); + String nested = useNested ? + ",\"incorrect\":{\"type\":\"object\"},\"nested\":{\"type\":\"nested\",\"properties\":{\"value\":{\"type\":\"long\"}}}" : ""; + if (requireRouting) { + assertEquals("{\"_doc\":{\"enabled\":false," + + "\"_meta\":{\"_doc\":{\"_routing\":{\"required\":true}," + + "\"properties\":{\"field1\":{\"type\":\"text\"," + + "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}" + nested + "}}}," + + "\"_routing\":{\"required\":true}}}", mapping.get("_doc").source().string()); + } else { + assertEquals("{\"_doc\":{\"enabled\":false," + + "\"_meta\":{\"_doc\":{\"properties\":{\"field1\":{\"type\":\"text\"," + + "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}" + nested + "}}}}}", + mapping.get("_doc").source().string()); + } + } + + private void assertHits(String index, int numDocsExpected) { + SearchResponse searchResponse = client().prepareSearch(index) + .addSort(SeqNoFieldMapper.NAME, SortOrder.ASC) + .setSize(numDocsExpected).get(); + SearchHits hits = searchResponse.getHits(); + assertEquals(numDocsExpected, hits.totalHits); + IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats().clear().setDocs(true).get(); + long deleted = indicesStatsResponse.getTotal().docs.getDeleted(); + boolean allowHoles = deleted > 0; // we use indexRandom which might create holes ie. deleted docs + long i = 0; + for (SearchHit hit : hits) { + String id = hit.getId(); + Map sourceAsMap = hit.getSourceAsMap(); + assertTrue(sourceAsMap.containsKey("field1")); + if (allowHoles) { + long seqId = ((Number)hit.getSortValues()[0]).longValue(); + assertThat(i, Matchers.lessThanOrEqualTo(seqId)); + i = seqId + 1; + } else { + assertEquals(i++, hit.getSortValues()[0]); + } + assertEquals("bar " + id, sourceAsMap.get("field1")); + assertEquals("r" + id, hit.field("_routing").getValue()); + } } - private IndexRequestBuilder[] snashotAndRestore(String sourceIdx, int numShards, boolean minimal, boolean requireRouting) + private IndexRequestBuilder[] snashotAndRestore(String sourceIdx, int numShards, boolean minimal, boolean requireRouting, boolean + useNested) throws ExecutionException, InterruptedException, IOException { logger.info("--> starting a master node and a data node"); internalCluster().startMasterOnlyNode(); @@ -186,8 +204,16 @@ private IndexRequestBuilder[] snashotAndRestore(String sourceIdx, int numShards, CreateIndexRequestBuilder createIndexRequestBuilder = prepareCreate(sourceIdx, 0, Settings.builder() .put("number_of_shards", numShards).put("number_of_replicas", 0)); + List mappings = new ArrayList<>(); if (requireRouting) { - createIndexRequestBuilder.addMapping("_doc", "_routing", "required=true"); + mappings.addAll(Arrays.asList("_routing", "required=true")); + } + + if (useNested) { + mappings.addAll(Arrays.asList("nested", "type=nested", "incorrect", "type=object")); + } + if (mappings.isEmpty() == false) { + createIndexRequestBuilder.addMapping("_doc", mappings.toArray()); } assertAcked(createIndexRequestBuilder); ensureGreen(); @@ -195,8 +221,19 @@ private IndexRequestBuilder[] snashotAndRestore(String sourceIdx, int numShards, logger.info("--> indexing some data"); IndexRequestBuilder[] builders = new IndexRequestBuilder[randomIntBetween(10, 100)]; for (int i = 0; i < builders.length; i++) { + XContentBuilder source = jsonBuilder() + .startObject() + .field("field1", "bar " + i); + if (useNested) { + source.startArray("nested"); + for (int j = 0; j < 2; ++j) { + source = source.startObject().field("value", i + 1 + j).endObject(); + } + source.endArray(); + } + source.endObject(); builders[i] = client().prepareIndex(sourceIdx, "_doc", - Integer.toString(i)).setSource("field1", "bar " + i).setRouting("r" + i); + Integer.toString(i)).setSource(source).setRouting("r" + i); } indexRandom(true, builders); flushAndRefresh(); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 05ebec25bcb10..6ed6055b25291 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -1,21 +1,34 @@ package org.elasticsearch.snapshots; import org.apache.lucene.document.Document; +import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.Term; +import org.apache.lucene.util.Bits; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MappingMetaData; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.fieldvisitor.FieldsVisitor; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.shard.IndexShard; @@ -27,9 +40,15 @@ import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.fs.FsRepository; +import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.nio.file.Path; +import java.util.Arrays; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; + +import static org.elasticsearch.index.mapper.SourceToParse.source; public class SourceOnlySnapshotShardTests extends IndexShardTestCase { @@ -54,13 +73,14 @@ public void testSourceIncomplete() throws IOException { } SnapshotId snapshotId = new SnapshotId("test", "test"); IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); - SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository(), false); + SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository()); repository.start(); try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); IllegalStateException illegalStateException = expectThrows(IllegalStateException.class, () -> - repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef.getIndexCommit(), - indexShardSnapshotStatus)); + runAsSnapshot(shard.getThreadPool(), + () -> repository.snapshotShard(shard, shard.store(), snapshotId, indexId, + snapshotRef.getIndexCommit(), indexShardSnapshotStatus))); assertEquals("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled or filters the source" , illegalStateException.getMessage()); } @@ -74,14 +94,15 @@ public void testIncrementalSnapshot() throws IOException { indexDoc(shard, "_doc", id); } - SnapshotId snapshotId = new SnapshotId("test", "test"); IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); - SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository(), false); + SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository()); repository.start(); int totalFileCount = -1; try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef.getIndexCommit(), indexShardSnapshotStatus); + SnapshotId snapshotId = new SnapshotId("test", "test"); + runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef + .getIndexCommit(), indexShardSnapshotStatus)); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); totalFileCount = copy.getTotalFileCount(); @@ -90,10 +111,12 @@ public void testIncrementalSnapshot() throws IOException { indexDoc(shard, "_doc", Integer.toString(10)); indexDoc(shard, "_doc", Integer.toString(11)); - snapshotId = new SnapshotId("test_1", "test_1"); try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + SnapshotId snapshotId = new SnapshotId("test_1", "test_1"); + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef.getIndexCommit(), indexShardSnapshotStatus); + runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef + .getIndexCommit(), indexShardSnapshotStatus)); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1.si, _1.fdx, _1.fnm, _1.fdt assertEquals(5, copy.getIncrementalFileCount()); @@ -102,10 +125,12 @@ public void testIncrementalSnapshot() throws IOException { assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); } deleteDoc(shard, "_doc", Integer.toString(10)); - snapshotId = new SnapshotId("test_2", "test_2"); try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + SnapshotId snapshotId = new SnapshotId("test_2", "test_2"); + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef.getIndexCommit(), indexShardSnapshotStatus); + runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef + .getIndexCommit(), indexShardSnapshotStatus)); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1_1.liv assertEquals(2, copy.getIncrementalFileCount()); @@ -120,7 +145,7 @@ private String randomDoc() { return "{ \"value\" : \"" + randomAlphaOfLength(10) + "\"}"; } - public void testRestoreAndReindex() throws IOException { + public void testRestoreMinmal() throws IOException { IndexShard shard = newStartedShard(true); int numInitialDocs = randomIntBetween(10, 100); for (int i = 0; i < numInitialDocs; i++) { @@ -145,11 +170,16 @@ public void testRestoreAndReindex() throws IOException { } SnapshotId snapshotId = new SnapshotId("test", "test"); IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); - SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository(), false); + SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository()); repository.start(); try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef.getIndexCommit(), indexShardSnapshotStatus); + runAsSnapshot(shard.getThreadPool(), () -> { + repository.initializeSnapshot(snapshotId, Arrays.asList(indexId), + MetaData.builder().put(shard.indexSettings() + .getIndexMetaData(), false).build()); + repository.snapshotShard(shard, shard.store(), snapshotId, indexId, snapshotRef.getIndexCommit(), indexShardSnapshotStatus); + }); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); @@ -158,22 +188,33 @@ public void testRestoreAndReindex() throws IOException { ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), true, ShardRoutingState.INITIALIZING, new RecoverySource.SnapshotRecoverySource(new Snapshot("src_only", snapshotId), Version.CURRENT, indexId.getId())); - IndexShard restoredShard = newShard(shardRouting); + IndexMetaData metaData = runAsSnapshot(threadPool, () -> repository.getSnapshotIndexMetaData(snapshotId, indexId)); + IndexShard restoredShard = newShard(shardRouting, metaData, null, (c) -> new SourceOnlySnapshotEngine(c), () -> {}); restoredShard.mapperService().merge(shard.indexSettings().getIndexMetaData(), MapperService.MergeReason.MAPPING_RECOVERY); DiscoveryNode discoveryNode = new DiscoveryNode("node_g", buildNewFakeTransportAddress(), Version.CURRENT); restoredShard.markAsRecovering("test from snap", new RecoveryState(restoredShard.routingEntry(), discoveryNode, null)); - assertTrue(restoredShard.restoreFromRepository(repository)); + runAsSnapshot(shard.getThreadPool(), () -> + assertTrue(restoredShard.restoreFromRepository(repository))); assertEquals(restoredShard.recoveryState().getStage(), RecoveryState.Stage.DONE); - assertEquals(restoredShard.recoveryState().getTranslog().recoveredOperations(), shard.docStats().getCount()); + assertEquals(restoredShard.recoveryState().getTranslog().recoveredOperations(), 0); assertEquals(IndexShardState.POST_RECOVERY, restoredShard.state()); restoredShard.refresh("test"); assertEquals(restoredShard.docStats().getCount(), shard.docStats().getCount()); - assertEquals(0, restoredShard.docStats().getDeleted()); + expectThrows(UnsupportedOperationException.class, + () -> restoredShard.get(new Engine.Get(false, false, "_doc", Integer.toString(0), + new Term("_id", Uid.encodeId(Integer.toString(0)))))); + final IndexShard targetShard; + try (Engine.Searcher searcher = restoredShard.acquireSearcher("test")) { + targetShard = reindex(searcher.getDirectoryReader(), new MappingMetaData("_doc", + restoredShard.mapperService().documentMapper("_doc").meta())); + } + for (int i = 0; i < numInitialDocs; i++) { Engine.Get get = new Engine.Get(false, false, "_doc", Integer.toString(i), new Term("_id", Uid.encodeId(Integer.toString(i)))); Engine.GetResult original = shard.get(get); - Engine.GetResult restored = restoredShard.get(get); + Engine.GetResult restored = targetShard.get(get); assertEquals(original.exists(), restored.exists()); + if (original.exists()) { Document document = original.docIdAndVersion().reader.document(original.docIdAndVersion().docId); Document restoredDocument = restored.docIdAndVersion().reader.document(restored.docIdAndVersion().docId); @@ -184,7 +225,56 @@ public void testRestoreAndReindex() throws IOException { IOUtils.close(original, restored); } - closeShards(shard, restoredShard); + closeShards(shard, restoredShard, targetShard); + } + + public IndexShard reindex(DirectoryReader reader, MappingMetaData mapping) throws IOException { + ShardRouting targetShardRouting = TestShardRouting.newShardRouting(new ShardId("target", "_na_", 0), randomAlphaOfLength(10), true, + ShardRoutingState.INITIALIZING, RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE); + 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(targetShardRouting.getIndexName()) + .settings(settings) + .primaryTerm(0, primaryTerm); + metaData.putMapping(mapping); + IndexShard targetShard = newShard(targetShardRouting, metaData.build()); + boolean success = false; + try { + recoverShardFromStore(targetShard); + long primaryTerm = targetShard.getPendingPrimaryTerm(); + String index = targetShard.shardId().getIndexName(); + FieldsVisitor rootFieldsVisitor = new FieldsVisitor(true); + for (LeafReaderContext ctx : reader.leaves()) { + LeafReader leafReader = ctx.reader(); + Bits liveDocs = leafReader.getLiveDocs(); + for (int i = 0; i < leafReader.maxDoc(); i++) { + if (liveDocs == null || liveDocs.get(i)) { + rootFieldsVisitor.reset(); + leafReader.document(i, rootFieldsVisitor); + rootFieldsVisitor.postProcess(targetShard.mapperService()); + Uid uid = rootFieldsVisitor.uid(); + BytesReference source = rootFieldsVisitor.source(); + assert source != null : "_source is null but should have been filtered out at snapshot time"; + Engine.Result result = targetShard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, source + (index, uid.type(), uid.id(), source, XContentHelper.xContentType(source), false) + .routing(rootFieldsVisitor.routing()), 1, false); + if (result.getResultType() != Engine.Result.Type.SUCCESS) { + throw new IllegalStateException("failed applying post restore operation result: " + result + .getResultType(), result.getFailure()); + } + } + } + } + targetShard.refresh("test"); + success = true; + } finally { + if (success == false) { + closeShards(targetShard); + } + } + return targetShard; } @@ -204,4 +294,32 @@ private Repository createRepository() throws IOException { return new FsRepository(repositoryMetaData, createEnvironment(), xContentRegistry()); } + private static void runAsSnapshot(ThreadPool pool, Runnable runnable) { + runAsSnapshot(pool, (Callable) () -> { + runnable.run(); + return null; + }); + } + + private static T runAsSnapshot(ThreadPool pool, Callable runnable) { + PlainActionFuture future = new PlainActionFuture<>(); + pool.executor(ThreadPool.Names.SNAPSHOT).execute(() -> { + try { + future.onResponse(runnable.call()); + } catch (Exception e) { + future.onFailure(e); + } + }); + try { + return future.get(); + } catch (ExecutionException e) { + if (e.getCause() instanceof Exception) { + throw ExceptionsHelper.convertToRuntime((Exception) e.getCause()); + } else { + throw new AssertionError(e.getCause()); + } + } catch (InterruptedException e) { + throw new AssertionError(e); + } + } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java index 9a40cb5f32c05..6ba843edf5a53 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.test.ESTestCase; import java.io.IOException; +import java.util.List; public class SourceOnlySnapshotTests extends ESTestCase { public void testSourceOnlyRandom() throws IOException { @@ -41,9 +42,11 @@ public void testSourceOnlyRandom() throws IOException { SnapshotDeletionPolicy deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); try (RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig().setIndexDeletionPolicy (deletionPolicy))) { - boolean modifyDeletedDocs = true; randomBoolean(); - SourceOnlySnapshot snapshoter = new SourceOnlySnapshot(targetDir, writer.w.getConfig().getSoftDeletesField(), - modifyDeletedDocs? () -> new DocValuesFieldExistsQuery("some_values") : null); + String softDeletesField = writer.w.getConfig().getSoftDeletesField(); + // we either use the soft deletes directly or manually delete them to test the additional delete functionality + boolean modifyDeletedDocs = softDeletesField != null && randomBoolean(); + SourceOnlySnapshot snapshoter = new SourceOnlySnapshot(targetDir, modifyDeletedDocs ? null : softDeletesField, + modifyDeletedDocs? () -> new DocValuesFieldExistsQuery(softDeletesField) : null); writer.commit(); int numDocs = scaledRandomIntBetween(100, 10000); boolean appendOnly = randomBoolean(); @@ -75,8 +78,8 @@ public void testSourceOnlyRandom() throws IOException { snapshoter.syncSnapshot(snapshot); try (DirectoryReader snapReader = snapshoter.wrapReader(DirectoryReader.open(targetDir)); DirectoryReader wrappedReader = snapshoter.wrapReader(DirectoryReader.open(snapshot))) { - DirectoryReader reader = modifyDeletedDocs ? new SoftDeletesDirectoryReaderWrapper(wrappedReader, "some_value") : - wrappedReader; + DirectoryReader reader = modifyDeletedDocs + ? new SoftDeletesDirectoryReaderWrapper(wrappedReader, softDeletesField) : wrappedReader; assertEquals(snapReader.maxDoc(), reader.maxDoc()); assertEquals(snapReader.numDocs(), reader.numDocs()); for (int i = 0; i < snapReader.maxDoc(); i++) { @@ -158,8 +161,8 @@ public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, } snapshoter = new SourceOnlySnapshot(targetDir, "id"); - snapshoter.syncSnapshot(snapshot); - assertEquals(0, snapshoter.getCreatedFiles().size()); + List createdFiles = snapshoter.syncSnapshot(snapshot); + assertEquals(0, createdFiles.size()); deletionPolicy.release(snapshot); // now add another doc doc = new Document(); @@ -178,9 +181,9 @@ public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, { snapshot = deletionPolicy.snapshot(); snapshoter = new SourceOnlySnapshot(targetDir, "id"); - snapshoter.syncSnapshot(snapshot); - assertEquals(4, snapshoter.getCreatedFiles().size()); - for (String file : snapshoter.getCreatedFiles()) { + createdFiles = snapshoter.syncSnapshot(snapshot); + assertEquals(4, createdFiles.size()); + for (String file : createdFiles) { String extension = IndexFileNames.getExtension(file); switch (extension) { case "fdt": @@ -203,9 +206,9 @@ public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, { snapshot = deletionPolicy.snapshot(); snapshoter = new SourceOnlySnapshot(targetDir, "id"); - snapshoter.syncSnapshot(snapshot); - assertEquals(1, snapshoter.getCreatedFiles().size()); - for (String file : snapshoter.getCreatedFiles()) { + createdFiles = snapshoter.syncSnapshot(snapshot); + assertEquals(1, createdFiles.size()); + for (String file : createdFiles) { String extension = IndexFileNames.getExtension(file); switch (extension) { case "liv": diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml index bba6d78050fb2..96de08364538b 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml @@ -64,9 +64,9 @@ setup: - match: { test_index.shards.0.type: SNAPSHOT } - match: { test_index.shards.0.stage: DONE } - - match: { test_index.shards.0.translog.recovered: 1} - - match: { test_index.shards.0.translog.total: 1} - - match: { test_index.shards.0.translog.total_on_start: 1} + - match: { test_index.shards.0.translog.recovered: 0} + - match: { test_index.shards.0.translog.total: 0} + - match: { test_index.shards.0.translog.total_on_start: 0} - match: { test_index.shards.0.index.files.recovered: 5} - match: { test_index.shards.0.index.files.reused: 0} - match: { test_index.shards.0.index.size.reused_in_bytes: 0} From ee8a9d547b5af0d165f671423085fdc48c46bf28 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 29 Aug 2018 13:23:25 +0200 Subject: [PATCH 11/44] fix imports and docs --- docs/plugins/repository-source-only.asciidoc | 15 +++++-------- .../snapshots/SourceOnlySnapshotEngine.java | 8 +++---- .../SourceOnlySnapshotRepository.java | 22 ------------------- 3 files changed, 10 insertions(+), 35 deletions(-) diff --git a/docs/plugins/repository-source-only.asciidoc b/docs/plugins/repository-source-only.asciidoc index ea3596edca2df..9475c5aa4538e 100644 --- a/docs/plugins/repository-source-only.asciidoc +++ b/docs/plugins/repository-source-only.asciidoc @@ -26,13 +26,10 @@ PUT _snapshot/my_src_only_repository // CONSOLE Since the `_source` only repository doesn't snapshot any index or doc-values structures but only stored -fields and index metadata, it's required to reindex the data during the restore process. This can either happen -as a full re-index based on the mapping of the original index or in a minimal form were only the internal data-structures -are recreated like the `_id` field in order to update the index. The latter can be configured in the repository settings -by setting `"minimal": true`. This allows updates and get operations but won't allow for aggregations or searches. +fields and index metadata, it's required to reindex the data after the restore process. The restored index is read-only +and can only serve scroll requests in oder to reindex. Individual queries will succeed but won't match or filter any +documents except of a `match_all` query. The restored will also have an empty mapping but allows access to the original +mapping via the types top level `meta` element. -A minimal restore is useful if the data is only needed to be re-indexed into another index or if individual documents should be -modified or deleted. - -During restore the re-indexing progress can be monitored via <> API that shows the per-document progress -under the `translog` recovery phase. +Source only snapshots take approximately 30% to 50% of the size on disk compared to a full snapshot. A source only snapshot +can only be taken if the original document source is maintained in the index and no source-filtering is applied. diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java index 43d973774d000..dca63434a1192 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java @@ -349,7 +349,7 @@ public void maybePruneDeletes() { private static final class SeqIdGeneratingDirectoryReader extends FilterDirectoryReader { private final long primaryTerm; - public SeqIdGeneratingDirectoryReader(DirectoryReader in, SeqIdGeneratingSubReaderWrapper wrapper) throws IOException { + SeqIdGeneratingDirectoryReader(DirectoryReader in, SeqIdGeneratingSubReaderWrapper wrapper) throws IOException { super(in, wrapper); primaryTerm = wrapper.primaryTerm; } @@ -372,11 +372,11 @@ public CacheHelper getReaderCacheHelper() { return in.getReaderCacheHelper(); } - private static abstract class FakeNumericDocValues extends NumericDocValues { + private abstract static class FakeNumericDocValues extends NumericDocValues { private final int maxDoc; int docID = -1; - public FakeNumericDocValues(int maxDoc) { + FakeNumericDocValues(int maxDoc) { this.maxDoc = maxDoc; } @@ -421,7 +421,7 @@ private static class SeqIdGeneratingSubReaderWrapper extends SubReaderWrapper { private final Map ctxMap; private final long primaryTerm; - public SeqIdGeneratingSubReaderWrapper(Map ctxMap, long primaryTerm) { + SeqIdGeneratingSubReaderWrapper(Map ctxMap, long primaryTerm) { this.ctxMap = ctxMap; this.primaryTerm = primaryTerm; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 83f11bbf3e8b7..749b3284abb84 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -3,42 +3,22 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.search.Query; import org.apache.lucene.store.FSDirectory; -import org.apache.lucene.store.HardlinkCopyDirectoryWrapper; -import org.apache.lucene.store.IOContext; import org.apache.lucene.store.SimpleFSDirectory; -import org.apache.lucene.util.Bits; -import org.elasticsearch.Version; -import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.Queries; -import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.env.ShardLock; -import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.fieldvisitor.FieldsVisitor; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.Store; -import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.FilterRepository; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; @@ -50,8 +30,6 @@ import java.util.function.Function; import java.util.function.Supplier; -import static org.elasticsearch.index.mapper.SourceToParse.source; - /** *

* This is a filter snapshot repository that only snapshots the minimal required information From 40cd45d54c6e6f04ef691f7a9d2fcfcfffa6b3f0 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 29 Aug 2018 15:44:33 +0200 Subject: [PATCH 12/44] fix constant --- .../docs/en/rest-api}/repository-source-only.asciidoc | 0 .../java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) rename {docs/plugins => x-pack/docs/en/rest-api}/repository-source-only.asciidoc (100%) diff --git a/docs/plugins/repository-source-only.asciidoc b/x-pack/docs/en/rest-api/repository-source-only.asciidoc similarity index 100% rename from docs/plugins/repository-source-only.asciidoc rename to x-pack/docs/en/rest-api/repository-source-only.asciidoc diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index 5c38b65c51191..c0df71c7acbd5 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -116,7 +116,7 @@ public void testSnapshotAndRestoreWithNested() throws Exception { boolean requireRouting = randomBoolean(); IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting, true); IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats().clear().setDocs(true).get(); - assertThat(indicesStatsResponse.getTotal().docs.getDeleted(), Matchers.greaterThan(0l)); + assertThat(indicesStatsResponse.getTotal().docs.getDeleted(), Matchers.greaterThan(0L)); assertHits(sourceIdx, builders.length); assertMappings(sourceIdx, requireRouting, true); assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() From deff31df42b5c88dd1e256657a6291b64581a36f Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 29 Aug 2018 21:35:13 +0200 Subject: [PATCH 13/44] add license headers --- .../snapshots/SourceOnlySnapshotRepository.java | 5 +++++ .../org/elasticsearch/snapshots/SourceOnlySnapshotIT.java | 5 +++++ .../snapshots/SourceOnlySnapshotShardTests.java | 5 +++++ 3 files changed, 15 insertions(+) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 749b3284abb84..e84a858bd2c81 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -1,3 +1,8 @@ +/* + * 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.snapshots; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index c0df71c7acbd5..fcb8b8ef3825d 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -1,3 +1,8 @@ +/* + * 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.snapshots; import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 6ed6055b25291..634172e8c7e12 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -1,3 +1,8 @@ +/* + * 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.snapshots; import org.apache.lucene.document.Document; From eea9f6feb0e391cff34405e441000225907f2453 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 30 Aug 2018 09:47:19 +0200 Subject: [PATCH 14/44] fix javadocs --- .../SourceOnlySnapshotRepository.java | 28 ++++++------------- 1 file changed, 8 insertions(+), 20 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index e84a858bd2c81..842c462502bc5 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -45,31 +45,19 @@ *

*

* The repository can wrap any other repository delegating the source only snapshot to it to and read - * from it. For instance a file repository of type fs by passing settings.delegate_type=fs + * from it. For instance a file repository of type fs by passing settings.delegate_type=fs * at repository creation time. *

- * The repository supports two distinct restore options: - *
    - *
  • minimal restore: this option re-indexes all documents during restore with an empty mapping. The original mapping is - * stored in the restored indexes _meta mapping field. The minimal restore must be enabled by setting - * settings.restore_minimal=true.
  • - *
  • full restore: this option re-indexes all documents during restore with the original mapping. This option is the - * default. This option has a significant operational overhead compared to the minimal option but recreates a fully functional new - * index
  • - *
- * - * Reindex operations are executed in a single thread and can be monitored via indices recovery stats. Every indexed document will be - * reported as a translog document. - * + * Snapshots restored from source only snapshots are minimal indices that are read-only and only allow + * match_all scroll searches in order to reindex the data. */ -// TODO: as a followup we should rename translog phase to operation phase in the indices _recovery stats public final class SourceOnlySnapshotRepository extends FilterRepository { - public static final Setting DELEGATE_TYPE = - new Setting<>("delegate_type", "", Function.identity(), Setting.Property.NodeScope); + private static final Setting DELEGATE_TYPE = new Setting<>("delegate_type", "", Function.identity(), Setting.Property + .NodeScope); public static final Setting SOURCE_ONLY_ENGINE = Setting.boolSetting("index.require_source_only_engine", false, Setting .Property.IndexScope, Setting.Property.InternalIndex, Setting.Property.Final); - public static final String SNAPSHOT_DIR_NAME = "_snapshot"; + private static final String SNAPSHOT_DIR_NAME = "_snapshot"; public static Repository.Factory newFactory() { return new Repository.Factory() { @@ -92,7 +80,7 @@ public Repository create(RepositoryMetaData metaData, Function querySupplier = shard.mapperService().hasNested() ? () -> Queries.newNestedFilter() : null; + Supplier querySupplier = shard.mapperService().hasNested() ? Queries::newNestedFilter : null; SourceOnlySnapshot snapshot = new SourceOnlySnapshot(tempStore.directory(), null, querySupplier); snapshot.syncSnapshot(snapshotIndexCommit); store.incRef(); From dc679c2c94de2decf2e7b390b22e24b62ea8b770 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 3 Sep 2018 11:22:18 +0200 Subject: [PATCH 15/44] Fix stuff after soft deletes are first class citizen --- .../snapshots/SourceOnlySnapshot.java | 26 ++++++++++++++----- .../snapshots/SourceOnlySnapshotEngine.java | 17 +++++++++--- .../SourceOnlySnapshotRepository.java | 2 +- .../snapshots/SourceOnlySnapshotTests.java | 24 ++++++++++------- 4 files changed, 49 insertions(+), 20 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java index 3453394ae95a9..1f495ec9beb3b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java @@ -51,19 +51,17 @@ import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.FIELDS_EXTENSION; import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.FIELDS_INDEX_EXTENSION; -public final class SourceOnlySnapshot { +public class SourceOnlySnapshot { private final Directory targetDirectory; - private final String softDeletesField; private final Supplier deleteByQuerySupplier; - public SourceOnlySnapshot(Directory targetDirectory, String softDeletesField, Supplier deleteByQuerySupplier) { + public SourceOnlySnapshot(Directory targetDirectory, Supplier deleteByQuerySupplier) { this.targetDirectory = targetDirectory; - this.softDeletesField = softDeletesField; this.deleteByQuerySupplier = deleteByQuerySupplier; } - public SourceOnlySnapshot(Directory targetDirectory, String softDeletesField) { - this(targetDirectory, softDeletesField, null); + public SourceOnlySnapshot(Directory targetDirectory) { + this(targetDirectory, null); } public synchronized List syncSnapshot(IndexCommit commit) throws IOException { @@ -110,6 +108,11 @@ public synchronized List syncSnapshot(IndexCommit commit) throws IOExcep return Collections.unmodifiableList(createdFiles); } + private String getSoftDeletesField(DirectoryReader reader) { + + return null; + } + private LiveDocs getLiveDocs(LeafReader reader) throws IOException { if (deleteByQuerySupplier != null) { // we have this additional delete by query functionality to filter out documents before we snapshot them @@ -170,6 +173,14 @@ private boolean assertCheckIndex() throws IOException { } DirectoryReader wrapReader(DirectoryReader reader) throws IOException { + String softDeletesField = null; + for (LeafReaderContext ctx : reader.leaves()) { + String field = ctx.reader().getFieldInfos().getSoftDeletesField(); + if (field != null) { + softDeletesField = field; + break; + } + } return softDeletesField == null ? reader : new SoftDeletesDirectoryReaderWrapper(reader, softDeletesField); } @@ -189,7 +200,8 @@ private SegmentCommitInfo syncSegment(SegmentCommitInfo segmentCommitInfo, LiveD List fieldInfoCopy = new ArrayList<>(fieldInfos.size()); for (FieldInfo fieldInfo : fieldInfos) { fieldInfoCopy.add(new FieldInfo(fieldInfo.name, fieldInfo.number, - false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, fieldInfo.attributes(), 0, 0, false)); + false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, fieldInfo.attributes(), 0, 0, + fieldInfo.isSoftDeletesField())); } FieldInfos newFieldInfos = new FieldInfos(fieldInfoCopy.toArray(new FieldInfo[0])); codec.fieldInfosFormat().write(trackingDir, newSegmentInfo, segmentSuffix, newFieldInfos, IOContext.DEFAULT); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java index dca63434a1192..6cb014af0a670 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java @@ -27,6 +27,7 @@ import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineSearcher; import org.elasticsearch.index.engine.Segment; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.seqno.SeqNoStats; @@ -175,12 +176,17 @@ public void syncTranslog() { } @Override - public Closeable acquireTranslogRetentionLock() { + public Closeable acquireRetentionLockForPeerRecovery() { return () -> {}; } @Override - public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) { + public Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException { + return readHistoryOperations(source, mapperService, fromSeqNo); + } + + @Override + public Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { return new Translog.Snapshot() { @Override @@ -200,10 +206,15 @@ public Translog.Operation next() throws IOException { } @Override - public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { + public int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { return 0; } + @Override + public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException { + return false; + } + @Override public TranslogStats getTranslogStats() { return translogStats; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 842c462502bc5..08246251886f2 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -133,7 +133,7 @@ protected void closeInternal() { } }, Store.OnClose.EMPTY); Supplier querySupplier = shard.mapperService().hasNested() ? Queries::newNestedFilter : null; - SourceOnlySnapshot snapshot = new SourceOnlySnapshot(tempStore.directory(), null, querySupplier); + SourceOnlySnapshot snapshot = new SourceOnlySnapshot(tempStore.directory(), querySupplier); snapshot.syncSnapshot(snapshotIndexCommit); store.incRef(); try (DirectoryReader reader = DirectoryReader.open(tempStore.directory()); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java index 6ba843edf5a53..7ff1a96808b3d 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java @@ -31,6 +31,7 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; import org.apache.lucene.store.Directory; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -42,11 +43,16 @@ public void testSourceOnlyRandom() throws IOException { SnapshotDeletionPolicy deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); try (RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig().setIndexDeletionPolicy (deletionPolicy))) { - String softDeletesField = writer.w.getConfig().getSoftDeletesField(); + final String softDeletesField = writer.w.getConfig().getSoftDeletesField(); // we either use the soft deletes directly or manually delete them to test the additional delete functionality boolean modifyDeletedDocs = softDeletesField != null && randomBoolean(); - SourceOnlySnapshot snapshoter = new SourceOnlySnapshot(targetDir, modifyDeletedDocs ? null : softDeletesField, - modifyDeletedDocs? () -> new DocValuesFieldExistsQuery(softDeletesField) : null); + SourceOnlySnapshot snapshoter = new SourceOnlySnapshot(targetDir, + modifyDeletedDocs ? () -> new DocValuesFieldExistsQuery(softDeletesField) : null) { + @Override + DirectoryReader wrapReader(DirectoryReader reader) throws IOException { + return modifyDeletedDocs ? reader : super.wrapReader(reader); + } + }; writer.commit(); int numDocs = scaledRandomIntBetween(100, 10000); boolean appendOnly = randomBoolean(); @@ -114,7 +120,7 @@ public void testSrcOnlySnap() throws IOException { try (Directory dir = newDirectory()) { SnapshotDeletionPolicy deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig() - .setSoftDeletesField("id") + .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setIndexDeletionPolicy(deletionPolicy).setMergePolicy(new FilterMergePolicy(NoMergePolicy.INSTANCE) { @Override public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, MergeContext mergeContext) { @@ -140,11 +146,11 @@ public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, doc.add(new TextField("text", "the quick brown fox", Field.Store.NO)); doc.add(new NumericDocValuesField("rank", 3)); doc.add(new StoredField("src", "the quick brown fox")); - writer.softUpdateDocument(new Term("id", "1"), doc, new NumericDocValuesField("id", 1)); + writer.softUpdateDocument(new Term("id", "1"), doc, new NumericDocValuesField(Lucene.SOFT_DELETES_FIELD, 1)); writer.commit(); IndexCommit snapshot = deletionPolicy.snapshot(); Directory targetDir = newDirectory(); - SourceOnlySnapshot snapshoter = new SourceOnlySnapshot(targetDir, "id"); + SourceOnlySnapshot snapshoter = new SourceOnlySnapshot(targetDir); snapshoter.syncSnapshot(snapshot); @@ -160,7 +166,7 @@ public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, assertEquals(0, id.totalHits); } - snapshoter = new SourceOnlySnapshot(targetDir, "id"); + snapshoter = new SourceOnlySnapshot(targetDir); List createdFiles = snapshoter.syncSnapshot(snapshot); assertEquals(0, createdFiles.size()); deletionPolicy.release(snapshot); @@ -180,7 +186,7 @@ public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, writer.commit(); { snapshot = deletionPolicy.snapshot(); - snapshoter = new SourceOnlySnapshot(targetDir, "id"); + snapshoter = new SourceOnlySnapshot(targetDir); createdFiles = snapshoter.syncSnapshot(snapshot); assertEquals(4, createdFiles.size()); for (String file : createdFiles) { @@ -205,7 +211,7 @@ public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, writer.commit(); { snapshot = deletionPolicy.snapshot(); - snapshoter = new SourceOnlySnapshot(targetDir, "id"); + snapshoter = new SourceOnlySnapshot(targetDir); createdFiles = snapshoter.syncSnapshot(snapshot); assertEquals(1, createdFiles.size()); for (String file : createdFiles) { From de30a8f5f8e84a079339bc45f846da52d2a5e0f2 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 3 Sep 2018 15:09:12 +0200 Subject: [PATCH 16/44] move searcher creation to engine adn acquire write lock --- .../elasticsearch/index/engine/Engine.java | 24 +++++++++- .../index/engine/EngineSearcher.java | 5 +-- .../index/engine/InternalEngine.java | 39 ++++------------ .../elasticsearch/index/seqno/SeqNoStats.java | 1 - .../snapshots/SourceOnlySnapshotEngine.java | 45 +++++++------------ 5 files changed, 50 insertions(+), 64 deletions(-) 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 b010a2690edd9..55374f4cdb2cf 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -33,6 +33,7 @@ import org.apache.lucene.index.SegmentReader; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; @@ -569,7 +570,28 @@ public final Searcher acquireSearcher(String source) throws EngineException { * * @see Searcher#close() */ - public abstract Searcher acquireSearcher(String source, SearcherScope scope) throws EngineException; + public Searcher acquireSearcher(String source, SearcherScope scope) throws EngineException { + /* Acquire order here is store -> manager since we need + * to make sure that the store is not closed before + * the searcher is acquired. */ + store.incRef(); + Releasable releasable = store::decRef; + try { + EngineSearcher engineSearcher = new EngineSearcher(source, getReferenceManager(scope), store, logger); + releasable = null; // success - hand over the reference to the engine searcher + return engineSearcher; + } catch (AlreadyClosedException ex) { + throw ex; + } catch (Exception ex) { + ensureOpen(ex); // throw EngineCloseException here if we are already closed + logger.error(() -> new ParameterizedMessage("failed to acquire searcher, source {}", source), ex); + throw new EngineException(shardId, "failed to acquire searcher, source " + source, ex); + } finally { + Releasables.close(releasable); + } + } + + protected abstract ReferenceManager getReferenceManager(SearcherScope scope); public enum SearcherScope { EXTERNAL, INTERNAL diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineSearcher.java b/server/src/main/java/org/elasticsearch/index/engine/EngineSearcher.java index b888b9682b786..7fd0fe6cc3904 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineSearcher.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineSearcher.java @@ -31,14 +31,13 @@ /** * Searcher for an Engine */ -public final class EngineSearcher extends Engine.Searcher { +final class EngineSearcher extends Engine.Searcher { private final AtomicBoolean released = new AtomicBoolean(false); private final Store store; private final Logger logger; private final ReferenceManager referenceManager; - public EngineSearcher(String source, ReferenceManager searcherReferenceManager, Store store, Logger logger) throws - IOException { + EngineSearcher(String source, ReferenceManager searcherReferenceManager, Store store, Logger logger) throws IOException { super(source, searcherReferenceManager.acquire()); this.store = store; this.logger = logger; 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 00d1f67f01bd9..3b93b02fb8871 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -20,7 +20,6 @@ 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; @@ -52,7 +51,6 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.LoggerInfoStream; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; @@ -2010,35 +2008,14 @@ protected final void closeNoLock(String reason, CountDownLatch closedLatch) { } @Override - public Searcher acquireSearcher(String source, SearcherScope scope) { - /* Acquire order here is store -> manager since we need - * to make sure that the store is not closed before - * the searcher is acquired. */ - store.incRef(); - Releasable releasable = store::decRef; - try { - final ReferenceManager referenceManager; - switch (scope) { - case INTERNAL: - referenceManager = internalSearcherManager; - break; - case EXTERNAL: - referenceManager = externalSearcherManager; - break; - default: - throw new IllegalStateException("unknown scope: " + scope); - } - EngineSearcher engineSearcher = new EngineSearcher(source, referenceManager, store, logger); - releasable = null; // success - hand over the reference to the engine searcher - return engineSearcher; - } catch (AlreadyClosedException ex) { - throw ex; - } catch (Exception ex) { - ensureOpen(ex); // throw EngineCloseException here if we are already closed - logger.error(() -> new ParameterizedMessage("failed to acquire searcher, source {}", source), ex); - throw new EngineException(shardId, "failed to acquire searcher, source " + source, ex); - } finally { - Releasables.close(releasable); + protected ReferenceManager getReferenceManager(SearcherScope scope) { + switch (scope) { + case INTERNAL: + return internalSearcherManager; + case EXTERNAL: + return externalSearcherManager; + default: + throw new IllegalStateException("unknown scope: " + scope); } } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java b/server/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java index 9c1795d654ccc..c711fb429366e 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java @@ -91,5 +91,4 @@ public String toString() { ", globalCheckpoint=" + globalCheckpoint + '}'; } - } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java index 6cb014af0a670..902ce866d5f33 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java @@ -9,23 +9,22 @@ import org.apache.lucene.index.FilterDirectoryReader; import org.apache.lucene.index.FilterLeafReader; import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.search.SearcherFactory; import org.apache.lucene.search.SearcherManager; -import org.apache.lucene.store.AlreadyClosedException; -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.lease.Releasables; +import org.apache.lucene.store.Lock; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; -import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineException; -import org.elasticsearch.index.engine.EngineSearcher; import org.elasticsearch.index.engine.Segment; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -54,6 +53,7 @@ public final class SourceOnlySnapshotEngine extends Engine { private final TranslogStats translogStats; private final SearcherManager searcherManager; private final IndexCommit indexCommit; + private final Lock indexWriterLock; public SourceOnlySnapshotEngine(EngineConfig config) { super(config); @@ -61,8 +61,10 @@ public SourceOnlySnapshotEngine(EngineConfig config) { Store store = config.getStore(); store.incRef(); DirectoryReader reader = null; + Lock indexWriterLock = null; boolean success = false; try { + indexWriterLock = store.directory().obtainLock(IndexWriter.WRITE_LOCK_NAME); this.lastCommittedSegmentInfos = Lucene.readSegmentInfos(store.directory()); this.translogStats = new TranslogStats(0, 0, 0, 0, 0); final SequenceNumbers.CommitInfo seqNoStats = @@ -74,10 +76,11 @@ public SourceOnlySnapshotEngine(EngineConfig config) { .open(store.directory()), config.getShardId()), config.getPrimaryTermSupplier().getAsLong()); this.indexCommit = reader.getIndexCommit(); this.searcherManager = new SearcherManager(reader, new SearcherFactory()); + this.indexWriterLock = indexWriterLock; success = true; } finally { if (success == false) { - IOUtils.close(reader, store::decRef); + IOUtils.close(reader, indexWriterLock, store::decRef); } } } catch (IOException e) { @@ -89,7 +92,7 @@ public SourceOnlySnapshotEngine(EngineConfig config) { protected void closeNoLock(String reason, CountDownLatch closedLatch) { if (isClosed.compareAndSet(false, true)) { try { - IOUtils.close(searcherManager, store::decRef); + IOUtils.close(searcherManager, indexWriterLock, store::decRef); } catch (Exception ex) { logger.warn("failed to close searcher", ex); } finally { @@ -104,21 +107,8 @@ public GetResult get(Get get, BiFunction search } @Override - public Searcher acquireSearcher(String source, SearcherScope scope) throws EngineException { - store.incRef(); - Releasable releasable = store::decRef; - try (ReleasableLock ignored = readLock.acquire()) { - final EngineSearcher searcher = new EngineSearcher(source, searcherManager, store, logger); - releasable = null; // hand over the reference to the engine searcher - return searcher; - } catch (AlreadyClosedException ex) { - throw ex; - } catch (Exception ex) { - ensureOpen(ex); // throw AlreadyClosedException if it's closed - throw new EngineException(shardId, "failed to acquire searcher, source " + source, ex); - } finally { - Releasables.close(releasable); - } + protected ReferenceManager getReferenceManager(SearcherScope scope) { + return searcherManager; } @Override @@ -181,7 +171,8 @@ public Closeable acquireRetentionLockForPeerRecovery() { } @Override - public Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException { + public Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, long fromSeqNo, long toSeqNo, + boolean requiredFullRange) throws IOException { return readHistoryOperations(source, mapperService, fromSeqNo); } @@ -190,8 +181,7 @@ public Translog.Snapshot readHistoryOperations(String source, MapperService mapp return new Translog.Snapshot() { @Override - public void close() throws IOException { - } + public void close() { } @Override public int totalOperations() { @@ -199,7 +189,7 @@ public int totalOperations() { } @Override - public Translog.Operation next() throws IOException { + public Translog.Operation next() { return null; } }; @@ -231,8 +221,7 @@ public long getLocalCheckpoint() { } @Override - public void waitForOpsToComplete(long seqNo) { - } + public void waitForOpsToComplete(long seqNo) { } @Override public void resetLocalCheckpoint(long newCheckpoint) { From 6d63b4171421cb3f0f3b65575161d54cb0f033c7 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 3 Sep 2018 19:45:14 +0200 Subject: [PATCH 17/44] fix tests --- .../test/InternalTestCluster.java | 11 ++- .../snapshots/SourceOnlySnapshotEngine.java | 8 +- .../snapshots/SourceOnlySnapshotIT.java | 82 +++++++++---------- 3 files changed, 49 insertions(+), 52 deletions(-) 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 4c813372fae31..276b9417a2865 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -75,6 +75,7 @@ import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.CommitStats; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.shard.ShardId; @@ -1152,10 +1153,12 @@ private void assertOpenTranslogReferences() throws Exception { IndicesService indexServices = getInstance(IndicesService.class, nodeAndClient.name); for (IndexService indexService : indexServices) { for (IndexShard indexShard : indexService) { - try { - IndexShardTestCase.getTranslog(indexShard).getDeletionPolicy().assertNoOpenTranslogRefs(); - } catch (AlreadyClosedException ok) { - // all good + if (IndexShardTestCase.getEngine(indexShard) instanceof InternalEngine) { + try { + IndexShardTestCase.getTranslog(indexShard).getDeletionPolicy().assertNoOpenTranslogRefs(); + } catch (AlreadyClosedException ok) { + // all good + } } } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java index 902ce866d5f33..32597efb1969f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java @@ -18,6 +18,7 @@ import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.search.SearcherFactory; import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.store.Directory; import org.apache.lucene.store.Lock; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; @@ -61,11 +62,12 @@ public SourceOnlySnapshotEngine(EngineConfig config) { Store store = config.getStore(); store.incRef(); DirectoryReader reader = null; + Directory directory = store.directory(); Lock indexWriterLock = null; boolean success = false; try { - indexWriterLock = store.directory().obtainLock(IndexWriter.WRITE_LOCK_NAME); - this.lastCommittedSegmentInfos = Lucene.readSegmentInfos(store.directory()); + indexWriterLock = directory.obtainLock(IndexWriter.WRITE_LOCK_NAME); + this.lastCommittedSegmentInfos = Lucene.readSegmentInfos(directory); this.translogStats = new TranslogStats(0, 0, 0, 0, 0); final SequenceNumbers.CommitInfo seqNoStats = SequenceNumbers.loadSeqNoInfoFromLuceneCommit(lastCommittedSegmentInfos.userData.entrySet()); @@ -73,7 +75,7 @@ public SourceOnlySnapshotEngine(EngineConfig config) { long localCheckpoint = seqNoStats.localCheckpoint; this.seqNoStats = new SeqNoStats(maxSeqNo, localCheckpoint, localCheckpoint); reader = SeqIdGeneratingDirectoryReader.wrap(ElasticsearchDirectoryReader.wrap(DirectoryReader - .open(store.directory()), config.getShardId()), config.getPrimaryTermSupplier().getAsLong()); + .open(directory), config.getShardId()), config.getPrimaryTermSupplier().getAsLong()); this.indexCommit = reader.getIndexCommit(); this.searcherManager = new SearcherManager(reader, new SearcherFactory()); this.indexWriterLock = indexWriterLock; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index fcb8b8ef3825d..401e47fcb8a81 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -91,55 +91,47 @@ public List> getSettings() { public void testSnapshotAndRestore() throws Exception { final String sourceIdx = "test-idx"; - try { - boolean requireRouting = randomBoolean(); - boolean useNested = randomBoolean(); - IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting, useNested); - assertHits(sourceIdx, builders.length); - assertMappings(sourceIdx, requireRouting, useNested); - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() - .addIds("" + randomIntBetween(0, builders.length))).get(), 0); - // ensure we can not find hits it's a minimal restore - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); - // make sure deletes work - String idToDelete = "" + randomIntBetween(0, builders.length); - expectThrows(ClusterBlockException.class, () -> client().prepareDelete(sourceIdx, "_doc", idToDelete) - .setRouting("r" + idToDelete).get()); - internalCluster().ensureAtLeastNumDataNodes(2); - client().admin().indices().prepareUpdateSettings(sourceIdx) - .setSettings(Settings.builder().put("index.number_of_replicas", 1)).get(); - ensureGreen(sourceIdx); - assertHits(sourceIdx, builders.length); - } finally { - client().admin().indices().prepareDelete(sourceIdx).get(); - } + boolean requireRouting = randomBoolean(); + boolean useNested = randomBoolean(); + IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting, useNested); + assertHits(sourceIdx, builders.length); + assertMappings(sourceIdx, requireRouting, useNested); + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() + .addIds("" + randomIntBetween(0, builders.length))).get(), 0); + // ensure we can not find hits it's a minimal restore + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); + // make sure deletes work + String idToDelete = "" + randomIntBetween(0, builders.length); + expectThrows(ClusterBlockException.class, () -> client().prepareDelete(sourceIdx, "_doc", idToDelete) + .setRouting("r" + idToDelete).get()); + internalCluster().ensureAtLeastNumDataNodes(2); + client().admin().indices().prepareUpdateSettings(sourceIdx) + .setSettings(Settings.builder().put("index.number_of_replicas", 1)).get(); + ensureGreen(sourceIdx); + assertHits(sourceIdx, builders.length); } public void testSnapshotAndRestoreWithNested() throws Exception { final String sourceIdx = "test-idx"; - try { - boolean requireRouting = randomBoolean(); - IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting, true); - IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats().clear().setDocs(true).get(); - assertThat(indicesStatsResponse.getTotal().docs.getDeleted(), Matchers.greaterThan(0L)); - assertHits(sourceIdx, builders.length); - assertMappings(sourceIdx, requireRouting, true); - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() - .addIds("" + randomIntBetween(0, builders.length))).get(), 0); - // ensure we can not find hits it's a minimal restore - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); - // make sure deletes work - String idToDelete = "" + randomIntBetween(0, builders.length); - expectThrows(ClusterBlockException.class, () -> client().prepareDelete(sourceIdx, "_doc", idToDelete) - .setRouting("r" + idToDelete).get()); - internalCluster().ensureAtLeastNumDataNodes(2); - client().admin().indices().prepareUpdateSettings(sourceIdx).setSettings(Settings.builder().put("index.number_of_replicas", 1)) - .get(); - ensureGreen(sourceIdx); - assertHits(sourceIdx, builders.length); - } finally { - client().admin().indices().prepareDelete(sourceIdx).get(); - } + boolean requireRouting = randomBoolean(); + IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting, true); + IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats().clear().setDocs(true).get(); + assertThat(indicesStatsResponse.getTotal().docs.getDeleted(), Matchers.greaterThan(0L)); + assertHits(sourceIdx, builders.length); + assertMappings(sourceIdx, requireRouting, true); + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() + .addIds("" + randomIntBetween(0, builders.length))).get(), 0); + // ensure we can not find hits it's a minimal restore + assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); + // make sure deletes work + String idToDelete = "" + randomIntBetween(0, builders.length); + expectThrows(ClusterBlockException.class, () -> client().prepareDelete(sourceIdx, "_doc", idToDelete) + .setRouting("r" + idToDelete).get()); + internalCluster().ensureAtLeastNumDataNodes(2); + client().admin().indices().prepareUpdateSettings(sourceIdx).setSettings(Settings.builder().put("index.number_of_replicas", 1)) + .get(); + ensureGreen(sourceIdx); + assertHits(sourceIdx, builders.length); } private void assertMappings(String sourceIdx, boolean requireRouting, boolean useNested) throws IOException { From f5e7f702856cc76915e788a00a4a85be08290848 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 3 Sep 2018 20:10:36 +0200 Subject: [PATCH 18/44] fix tests --- .../elasticsearch/index/engine/EngineTestCase.java | 3 ++- .../elasticsearch/snapshots/SourceOnlySnapshot.java | 11 ++++------- 2 files changed, 6 insertions(+), 8 deletions(-) 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 b558cd1ba9000..f6bf1aa8b2949 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 @@ -821,7 +821,8 @@ public static List readAllOperationsInLucene(Engine engine, * 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) { + if (mapper.documentMapper() == null || engine.config().getIndexSettings().isSoftDeleteEnabled() == false + || (engine instanceof InternalEngine) == false) { return; } final long maxSeqNo = ((InternalEngine) engine).getLocalCheckpointTracker().getMaxSeqNo(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java index 1f495ec9beb3b..6ebee5cdaf88b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java @@ -87,8 +87,10 @@ public synchronized List syncSnapshot(IndexCommit commit) throws IOExcep SegmentCommitInfo info = segmentInfos.info(ctx.ord); LeafReader leafReader = ctx.reader(); LiveDocs liveDocs = getLiveDocs(leafReader); - SegmentCommitInfo newInfo = syncSegment(info, liveDocs, leafReader.getFieldInfos(), existingSegments, createdFiles); - newInfos.add(newInfo); + if (leafReader.numDocs() != 0) { // fully deleted segments don't need to be processed + SegmentCommitInfo newInfo = syncSegment(info, liveDocs, leafReader.getFieldInfos(), existingSegments, createdFiles); + newInfos.add(newInfo); + } } segmentInfos.clear(); segmentInfos.addAll(newInfos); @@ -108,11 +110,6 @@ public synchronized List syncSnapshot(IndexCommit commit) throws IOExcep return Collections.unmodifiableList(createdFiles); } - private String getSoftDeletesField(DirectoryReader reader) { - - return null; - } - private LiveDocs getLiveDocs(LeafReader reader) throws IOException { if (deleteByQuerySupplier != null) { // we have this additional delete by query functionality to filter out documents before we snapshot them From 6975c82fd4ecdbddb25ecf12c4d93ae1f47da068 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 4 Sep 2018 09:36:28 +0200 Subject: [PATCH 19/44] fix test again --- .../java/org/elasticsearch/test/InternalTestCluster.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 276b9417a2865..631e2ecb8719b 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -1153,12 +1153,12 @@ private void assertOpenTranslogReferences() throws Exception { IndicesService indexServices = getInstance(IndicesService.class, nodeAndClient.name); for (IndexService indexService : indexServices) { for (IndexShard indexShard : indexService) { - if (IndexShardTestCase.getEngine(indexShard) instanceof InternalEngine) { - try { + try { + if (IndexShardTestCase.getEngine(indexShard) instanceof InternalEngine) { IndexShardTestCase.getTranslog(indexShard).getDeletionPolicy().assertNoOpenTranslogRefs(); - } catch (AlreadyClosedException ok) { - // all good } + } catch (AlreadyClosedException ok) { + // all good } } } From 70395be8f1c7862ea275560ef99cadb231eb3eb4 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 5 Sep 2018 14:48:35 +0200 Subject: [PATCH 20/44] fix settings --- .../snapshots/SourceOnlySnapshotRepository.java | 6 +++--- .../main/java/org/elasticsearch/xpack/core/XPackPlugin.java | 4 ++-- .../org/elasticsearch/snapshots/SourceOnlySnapshotIT.java | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 08246251886f2..c4e047d5fff50 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -54,8 +54,8 @@ public final class SourceOnlySnapshotRepository extends FilterRepository { private static final Setting DELEGATE_TYPE = new Setting<>("delegate_type", "", Function.identity(), Setting.Property .NodeScope); - public static final Setting SOURCE_ONLY_ENGINE = Setting.boolSetting("index.require_source_only_engine", false, Setting - .Property.IndexScope, Setting.Property.InternalIndex, Setting.Property.Final); + public static final Setting SOURCE_ONLY = Setting.boolSetting("index.source_only", false, Setting + .Property.IndexScope, Setting.Property.Final, Setting.Property.PrivateIndex); private static final String SNAPSHOT_DIR_NAME = "_snapshot"; @@ -108,7 +108,7 @@ public IndexMetaData getSnapshotIndexMetaData(SnapshotId snapshotId, IndexId ind builder.putMapping(next.key, mapping); } builder.settings(Settings.builder().put(snapshotIndexMetaData.getSettings()) - .put(SOURCE_ONLY_ENGINE.getKey(), true) + .put(SOURCE_ONLY.getKey(), true) .put("index.blocks.write", true)); // read-only! return builder.build(); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java index 5c0d11fdd51c3..c1360b7cc5170 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java @@ -357,7 +357,7 @@ public Map getRepositories(Environment env, NamedXCo @Override public Optional getEngineFactory(IndexSettings indexSettings) { - if (indexSettings.getValue(SourceOnlySnapshotRepository.SOURCE_ONLY_ENGINE)) { + if (indexSettings.getValue(SourceOnlySnapshotRepository.SOURCE_ONLY)) { EngineFactory engineFactory = SourceOnlySnapshotEngine::new; return Optional.of(engineFactory); } @@ -367,7 +367,7 @@ public Optional getEngineFactory(IndexSettings indexSettings) { @Override public List> getSettings() { List> settings = super.getSettings(); - settings.add(SourceOnlySnapshotRepository.SOURCE_ONLY_ENGINE); + settings.add(SourceOnlySnapshotRepository.SOURCE_ONLY); return settings; } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index 401e47fcb8a81..67053ad6ba21c 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -74,7 +74,7 @@ public Map getRepositories(Environment env, NamedXCo } @Override public Optional getEngineFactory(IndexSettings indexSettings) { - if (indexSettings.getValue(SourceOnlySnapshotRepository.SOURCE_ONLY_ENGINE)) { + if (indexSettings.getValue(SourceOnlySnapshotRepository.SOURCE_ONLY)) { EngineFactory engineFactory = SourceOnlySnapshotEngine::new; return Optional.of(engineFactory); } @@ -84,7 +84,7 @@ public Optional getEngineFactory(IndexSettings indexSettings) { @Override public List> getSettings() { List> settings = new ArrayList<>(super.getSettings()); - settings.add(SourceOnlySnapshotRepository.SOURCE_ONLY_ENGINE); + settings.add(SourceOnlySnapshotRepository.SOURCE_ONLY); return settings; } } From 134bcf59a6c49f4e85247922fc456118f43b4a25 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 7 Sep 2018 18:07:30 +0200 Subject: [PATCH 21/44] fix compilation --- .../src/main/java/org/elasticsearch/index/engine/Engine.java | 2 +- .../java/org/elasticsearch/snapshots/SourceOnlySnapshot.java | 4 ++-- .../org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) 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 fe27aea805eef..84b4b7328b177 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -1593,7 +1593,7 @@ public static class IndexCommitRef implements Closeable { private final CheckedRunnable onClose; private final IndexCommit indexCommit; - IndexCommitRef(IndexCommit indexCommit, CheckedRunnable onClose) { + public IndexCommitRef(IndexCommit indexCommit, CheckedRunnable onClose) { this.indexCommit = indexCommit; this.onClose = onClose; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java index 6ebee5cdaf88b..31a15bc4f8e39 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java @@ -25,6 +25,7 @@ import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; import org.apache.lucene.store.Directory; @@ -118,7 +119,7 @@ private LiveDocs getLiveDocs(LeafReader reader) throws IOException { IndexSearcher s = new IndexSearcher(reader); s.setQueryCache(null); Query rewrite = s.rewrite(query); - Weight weight = s.createWeight(rewrite, false, 1.0f); + Weight weight = s.createWeight(rewrite, ScoreMode.COMPLETE_NO_SCORES, 1.0f); Scorer scorer = weight.scorer(reader.getContext()); if (scorer != null) { DocIdSetIterator iterator = scorer.iterator(); @@ -159,7 +160,6 @@ private boolean assertCheckIndex() throws IOException { ByteArrayOutputStream output = new ByteArrayOutputStream(1024); try (CheckIndex checkIndex = new CheckIndex(targetDirectory)) { checkIndex.setFailFast(true); - checkIndex.setCrossCheckTermVectors(false); checkIndex.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false); CheckIndex.Status status = checkIndex.checkIndex(); if (status == null || status.clean == false) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java index 32597efb1969f..fee9d9aab0c65 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java @@ -330,7 +330,7 @@ public int fillSeqNoGaps(long primaryTerm) { } @Override - public Engine recoverFromTranslog(long upto) { + public Engine recoverFromTranslog(TranslogRecoveryRunner translogRecoveryRunner, long recoverUpToSeqNo) throws IOException { return this; } From efb930375a1edf70c12752ea4c1c7853d465f2f9 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 10 Sep 2018 11:29:40 +0200 Subject: [PATCH 22/44] fix test after lucene upgrade --- .../snapshots/SourceOnlySnapshotTests.java | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java index 7ff1a96808b3d..5f841ded37980 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java @@ -17,6 +17,7 @@ import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.RandomIndexWriter; @@ -41,8 +42,9 @@ public class SourceOnlySnapshotTests extends ESTestCase { public void testSourceOnlyRandom() throws IOException { try (Directory dir = newDirectory(); Directory targetDir = newDirectory()) { SnapshotDeletionPolicy deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); - try (RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig().setIndexDeletionPolicy - (deletionPolicy))) { + IndexWriterConfig indexWriterConfig = newIndexWriterConfig().setIndexDeletionPolicy + (deletionPolicy).setSoftDeletesField(random().nextBoolean() ? null : Lucene.SOFT_DELETES_FIELD); + try (RandomIndexWriter writer = new RandomIndexWriter(random(), dir, indexWriterConfig, false)) { final String softDeletesField = writer.w.getConfig().getSoftDeletesField(); // we either use the soft deletes directly or manually delete them to test the additional delete functionality boolean modifyDeletedDocs = softDeletesField != null && randomBoolean(); @@ -148,14 +150,13 @@ public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, doc.add(new StoredField("src", "the quick brown fox")); writer.softUpdateDocument(new Term("id", "1"), doc, new NumericDocValuesField(Lucene.SOFT_DELETES_FIELD, 1)); writer.commit(); - IndexCommit snapshot = deletionPolicy.snapshot(); Directory targetDir = newDirectory(); + IndexCommit snapshot = deletionPolicy.snapshot(); SourceOnlySnapshot snapshoter = new SourceOnlySnapshot(targetDir); snapshoter.syncSnapshot(snapshot); - - StandardDirectoryReader reader = (StandardDirectoryReader) DirectoryReader.open(snapshot); - try(DirectoryReader snapReader = DirectoryReader.open(targetDir)) { + StandardDirectoryReader reader = (StandardDirectoryReader) DirectoryReader.open(snapshot); + try (DirectoryReader snapReader = DirectoryReader.open(targetDir)) { assertEquals(snapReader.maxDoc(), 3); assertEquals(snapReader.numDocs(), 2); for (int i = 0; i < 3; i++) { @@ -163,7 +164,7 @@ public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, } IndexSearcher searcher = new IndexSearcher(snapReader); TopDocs id = searcher.search(new TermQuery(new Term("id", "1")), 10); - assertEquals(0, id.totalHits); + assertEquals(0, id.totalHits.value); } snapshoter = new SourceOnlySnapshot(targetDir); From d68254856b568fbdde9689c2e9f47f6ed99a09f3 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 10 Sep 2018 11:36:03 +0200 Subject: [PATCH 23/44] remove routing invariant --- .../snapshots/SourceOnlySnapshotRepository.java | 10 ++-------- .../snapshots/SourceOnlySnapshotIT.java | 16 ++++------------ 2 files changed, 6 insertions(+), 20 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index c4e047d5fff50..63a4b5d3c6382 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -97,14 +97,8 @@ public IndexMetaData getSnapshotIndexMetaData(SnapshotId snapshotId, IndexId ind IndexMetaData.Builder builder = IndexMetaData.builder(snapshotIndexMetaData); while (iterator.hasNext()) { ObjectObjectCursor next = iterator.next(); - MappingMetaData.Routing routing = next.value.routing(); - final String mapping; - if (routing.required()) { // we have to respect the routing to be on the safe side so we pass this one on. - mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + ", " + - "\"_routing\" : { \"required\" : true } } }"; - } else { - mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + " } }"; - } + // we don't need to obey any routing here stuff is read-only anyway and get is disabled + final String mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + " } }"; builder.putMapping(next.key, mapping); } builder.settings(Settings.builder().put(snapshotIndexMetaData.getSettings()) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index 67053ad6ba21c..e4e070bf559c1 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -141,18 +141,10 @@ private void assertMappings(String sourceIdx, boolean requireRouting, boolean us assertTrue(mapping.containsKey("_doc")); String nested = useNested ? ",\"incorrect\":{\"type\":\"object\"},\"nested\":{\"type\":\"nested\",\"properties\":{\"value\":{\"type\":\"long\"}}}" : ""; - if (requireRouting) { - assertEquals("{\"_doc\":{\"enabled\":false," + - "\"_meta\":{\"_doc\":{\"_routing\":{\"required\":true}," + - "\"properties\":{\"field1\":{\"type\":\"text\"," + - "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}" + nested + "}}}," + - "\"_routing\":{\"required\":true}}}", mapping.get("_doc").source().string()); - } else { - assertEquals("{\"_doc\":{\"enabled\":false," + - "\"_meta\":{\"_doc\":{\"properties\":{\"field1\":{\"type\":\"text\"," + - "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}" + nested + "}}}}}", - mapping.get("_doc").source().string()); - } + assertEquals("{\"_doc\":{\"enabled\":false," + + "\"_meta\":{\"_doc\":{\"properties\":{\"field1\":{\"type\":\"text\"," + + "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}" + nested + "}}}}}", + mapping.get("_doc").source().string()); } private void assertHits(String index, int numDocsExpected) { From 1ae023f6a1962899e689544718231b52510c83e7 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 10 Sep 2018 11:37:16 +0200 Subject: [PATCH 24/44] add comment about soft deletes --- .../elasticsearch/snapshots/SourceOnlySnapshotRepository.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 63a4b5d3c6382..54163b0858b39 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -127,6 +127,7 @@ protected void closeInternal() { } }, Store.OnClose.EMPTY); Supplier querySupplier = shard.mapperService().hasNested() ? Queries::newNestedFilter : null; + // SourceOnlySnapshot will take care of soft- and hard-deletes no special casing needed here SourceOnlySnapshot snapshot = new SourceOnlySnapshot(tempStore.directory(), querySupplier); snapshot.syncSnapshot(snapshotIndexCommit); store.incRef(); From 506c68bb53f638f10c9aefcac9f9e0837c8b762d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 10 Sep 2018 11:40:47 +0200 Subject: [PATCH 25/44] remove TODO --- .../org/elasticsearch/snapshots/SourceOnlySnapshot.java | 3 ++- .../elasticsearch/snapshots/SourceOnlySnapshotTests.java | 7 +++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java index 31a15bc4f8e39..b7d6a51f45a8a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshot.java @@ -192,7 +192,8 @@ private SegmentCommitInfo syncSegment(SegmentCommitInfo segmentCommitInfo, LiveD boolean exists = existingSegments.containsKey(segmentId); if (exists == false) { SegmentInfo newSegmentInfo = new SegmentInfo(si.dir, si.getVersion(), si.getMinVersion(), si.name, si.maxDoc(), false, - si.getCodec(), si.getDiagnostics(), si.getId(), si.getAttributes(), si.getIndexSort()); // TODO should we drop the sort? + si.getCodec(), si.getDiagnostics(), si.getId(), si.getAttributes(), null); + // we drop the sort on purpose since the field we sorted on doesn't exist in the target index anymore. newInfo = new SegmentCommitInfo(newSegmentInfo, 0, 0, -1, -1, -1); List fieldInfoCopy = new ArrayList<>(fieldInfos.size()); for (FieldInfo fieldInfo : fieldInfos) { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java index 5f841ded37980..e7d731739de02 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotTests.java @@ -19,10 +19,12 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.SegmentReader; import org.apache.lucene.index.SnapshotDeletionPolicy; import org.apache.lucene.index.SoftDeletesDirectoryReaderWrapper; import org.apache.lucene.index.StandardDirectoryReader; @@ -93,6 +95,11 @@ DirectoryReader wrapReader(DirectoryReader reader) throws IOException { for (int i = 0; i < snapReader.maxDoc(); i++) { assertEquals(snapReader.document(i).get("_source"), reader.document(i).get("_source")); } + for (LeafReaderContext ctx : snapReader.leaves()) { + if (ctx.reader() instanceof SegmentReader) { + assertNull(((SegmentReader) ctx.reader()).getSegmentInfo().info.getIndexSort()); + } + } } } finally { deletionPolicy.release(snapshot); From eec9f185961b48424c0ce53a136bc6670ef0d808 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 10 Sep 2018 11:47:20 +0200 Subject: [PATCH 26/44] make flush a no-op --- .../snapshots/SourceOnlySnapshotEngine.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java index fee9d9aab0c65..91c3cf5040a2e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java @@ -251,13 +251,10 @@ public List segments(boolean verbose) { } @Override - public void refresh(String source) throws EngineException { - } + public void refresh(String source) throws EngineException {} @Override - public void writeIndexingBuffer() throws EngineException { - - } + public void writeIndexingBuffer() throws EngineException {} @Override public boolean shouldPeriodicallyFlush() { @@ -266,17 +263,21 @@ public boolean shouldPeriodicallyFlush() { @Override public SyncedFlushResult syncFlush(String syncId, CommitId expectedCommitId) throws EngineException { - throw new UnsupportedOperationException(); + CommitId commitId = new CommitId(lastCommittedSegmentInfos.getId()); + if (commitId.equals(expectedCommitId)) { + return SyncedFlushResult.SUCCESS; + } + return SyncedFlushResult.COMMIT_MISMATCH; } @Override public CommitId flush(boolean force, boolean waitIfOngoing) throws EngineException { - throw new UnsupportedOperationException(); + return new CommitId(lastCommittedSegmentInfos.getId()); } @Override public CommitId flush() throws EngineException { - throw new UnsupportedOperationException(); + return flush(false, false); } @Override From df925d708b18794f12eadfb33ee69994055b23cd Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Sep 2018 14:45:02 +0200 Subject: [PATCH 27/44] integrate read-only engine --- .../SeqIdGeneratingFilterReader.java | 150 ++++++ .../snapshots/SourceOnlySnapshotEngine.java | 479 ------------------ .../SourceOnlySnapshotRepository.java | 14 + .../elasticsearch/xpack/core/XPackPlugin.java | 4 +- 4 files changed, 165 insertions(+), 482 deletions(-) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SeqIdGeneratingFilterReader.java delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SeqIdGeneratingFilterReader.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SeqIdGeneratingFilterReader.java new file mode 100644 index 0000000000000..e5c5a93d92980 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SeqIdGeneratingFilterReader.java @@ -0,0 +1,150 @@ +/* + * 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.snapshots; + +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FilterDirectoryReader; +import org.apache.lucene.index.FilterLeafReader; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.VersionFieldMapper; + +import java.io.IOException; +import java.util.IdentityHashMap; +import java.util.Map; + +/** + * This filter reader fakes sequence ID, primary term and version + * for a source only index. + */ +final class SeqIdGeneratingFilterReader extends FilterDirectoryReader { + private final long primaryTerm; + + private SeqIdGeneratingFilterReader(DirectoryReader in, SeqIdGeneratingSubReaderWrapper wrapper) throws IOException { + super(in, wrapper); + primaryTerm = wrapper.primaryTerm; + } + + @Override + protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { + return wrap(in, primaryTerm); + } + + static DirectoryReader wrap(DirectoryReader in, long primaryTerm) throws IOException { + Map ctxMap = new IdentityHashMap<>(); + for (LeafReaderContext leave : in.leaves()) { + ctxMap.put(leave.reader(), leave); + } + return new SeqIdGeneratingFilterReader(in, new SeqIdGeneratingSubReaderWrapper(ctxMap, primaryTerm)); + } + + @Override + public CacheHelper getReaderCacheHelper() { + return in.getReaderCacheHelper(); + } + + private abstract static class FakeNumericDocValues extends NumericDocValues { + private final int maxDoc; + int docID = -1; + + FakeNumericDocValues(int maxDoc) { + this.maxDoc = maxDoc; + } + + @Override + public int docID() { + return docID; + } + + @Override + public int nextDoc() { + if (docID+1 < maxDoc) { + docID++; + } else { + docID = NO_MORE_DOCS; + } + return docID; + } + + @Override + public int advance(int target) { + if (target >= maxDoc) { + docID = NO_MORE_DOCS; + } else { + docID = target; + } + return docID; + } + + @Override + public long cost() { + return maxDoc; + } + + @Override + public boolean advanceExact(int target) { + advance(target); + return docID != NO_MORE_DOCS; + } + } + + private static class SeqIdGeneratingSubReaderWrapper extends SubReaderWrapper { + private final Map ctxMap; + private final long primaryTerm; + + SeqIdGeneratingSubReaderWrapper(Map ctxMap, long primaryTerm) { + this.ctxMap = ctxMap; + this.primaryTerm = primaryTerm; + } + + @Override + public LeafReader wrap(LeafReader reader) { + LeafReaderContext leafReaderContext = ctxMap.get(reader); + final int docBase = leafReaderContext.docBase; + return new FilterLeafReader(reader) { + + @Override + public NumericDocValues getNumericDocValues(String field) throws IOException { + if (SeqNoFieldMapper.NAME.equals(field)) { + return new FakeNumericDocValues(maxDoc()) { + @Override + public long longValue() { + return docBase + docID; + } + }; + } else if (SeqNoFieldMapper.PRIMARY_TERM_NAME.equals(field)) { + return new FakeNumericDocValues(maxDoc()) { + @Override + public long longValue() { + return primaryTerm; + } + }; + } else if (VersionFieldMapper.NAME.equals(field)) { + return new FakeNumericDocValues(maxDoc()) { + @Override + public long longValue() { + return 1; + } + }; + } + return super.getNumericDocValues(field); + } + + @Override + public CacheHelper getCoreCacheHelper() { + return reader.getCoreCacheHelper(); + } + + @Override + public CacheHelper getReaderCacheHelper() { + return reader.getReaderCacheHelper(); + } + }; + } + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java deleted file mode 100644 index 91c3cf5040a2e..0000000000000 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotEngine.java +++ /dev/null @@ -1,479 +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.snapshots; - -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.IndexWriter; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.ReferenceManager; -import org.apache.lucene.search.SearcherFactory; -import org.apache.lucene.search.SearcherManager; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.Lock; -import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.engine.EngineConfig; -import org.elasticsearch.index.engine.EngineException; -import org.elasticsearch.index.engine.Segment; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.VersionFieldMapper; -import org.elasticsearch.index.seqno.SeqNoStats; -import org.elasticsearch.index.seqno.SequenceNumbers; -import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.index.translog.TranslogStats; - -import java.io.Closeable; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Arrays; -import java.util.IdentityHashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.function.BiFunction; -import java.util.stream.Stream; - -public final class SourceOnlySnapshotEngine extends Engine { - - private final SegmentInfos lastCommittedSegmentInfos; - private final SeqNoStats seqNoStats; - private final TranslogStats translogStats; - private final SearcherManager searcherManager; - private final IndexCommit indexCommit; - private final Lock indexWriterLock; - - public SourceOnlySnapshotEngine(EngineConfig config) { - super(config); - try { - Store store = config.getStore(); - store.incRef(); - DirectoryReader reader = null; - Directory directory = store.directory(); - Lock indexWriterLock = null; - boolean success = false; - try { - indexWriterLock = directory.obtainLock(IndexWriter.WRITE_LOCK_NAME); - this.lastCommittedSegmentInfos = Lucene.readSegmentInfos(directory); - this.translogStats = new TranslogStats(0, 0, 0, 0, 0); - final SequenceNumbers.CommitInfo seqNoStats = - SequenceNumbers.loadSeqNoInfoFromLuceneCommit(lastCommittedSegmentInfos.userData.entrySet()); - long maxSeqNo = seqNoStats.maxSeqNo; - long localCheckpoint = seqNoStats.localCheckpoint; - this.seqNoStats = new SeqNoStats(maxSeqNo, localCheckpoint, localCheckpoint); - reader = SeqIdGeneratingDirectoryReader.wrap(ElasticsearchDirectoryReader.wrap(DirectoryReader - .open(directory), config.getShardId()), config.getPrimaryTermSupplier().getAsLong()); - this.indexCommit = reader.getIndexCommit(); - this.searcherManager = new SearcherManager(reader, new SearcherFactory()); - this.indexWriterLock = indexWriterLock; - success = true; - } finally { - if (success == false) { - IOUtils.close(reader, indexWriterLock, store::decRef); - } - } - } catch (IOException e) { - throw new UncheckedIOException(e); // this is stupid - } - } - - @Override - protected void closeNoLock(String reason, CountDownLatch closedLatch) { - if (isClosed.compareAndSet(false, true)) { - try { - IOUtils.close(searcherManager, indexWriterLock, store::decRef); - } catch (Exception ex) { - logger.warn("failed to close searcher", ex); - } finally { - closedLatch.countDown(); - } - } - } - - @Override - public GetResult get(Get get, BiFunction searcherFactory) throws EngineException { - throw new UnsupportedOperationException(); - } - - @Override - protected ReferenceManager getReferenceManager(SearcherScope scope) { - return searcherManager; - } - - @Override - protected SegmentInfos getLastCommittedSegmentInfos() { - return lastCommittedSegmentInfos; - } - - @Override - public String getHistoryUUID() { - return lastCommittedSegmentInfos.userData.get(Engine.HISTORY_UUID_KEY); - } - - @Override - public long getWritingBytes() { - return 0; - } - - @Override - public long getIndexThrottleTimeInMillis() { - return 0; - } - - @Override - public boolean isThrottled() { - return false; - } - - @Override - public IndexResult index(Index index) { - throw new UnsupportedOperationException(); - } - - @Override - public DeleteResult delete(Delete delete) { - throw new UnsupportedOperationException(); - } - - @Override - public NoOpResult noOp(NoOp noOp) { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isTranslogSyncNeeded() { - return false; - } - - @Override - public boolean ensureTranslogSynced(Stream locations) { - return false; - } - - @Override - public void syncTranslog() { - } - - @Override - public Closeable acquireRetentionLockForPeerRecovery() { - return () -> {}; - } - - @Override - public Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, long fromSeqNo, long toSeqNo, - boolean requiredFullRange) throws IOException { - return readHistoryOperations(source, mapperService, fromSeqNo); - } - - @Override - public Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { - return new Translog.Snapshot() { - - @Override - public void close() { } - - @Override - public int totalOperations() { - return 0; - } - - @Override - public Translog.Operation next() { - return null; - } - }; - } - - @Override - public int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { - return 0; - } - - @Override - public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException { - return false; - } - - @Override - public TranslogStats getTranslogStats() { - return translogStats; - } - - @Override - public Translog.Location getTranslogLastWriteLocation() { - throw new UnsupportedOperationException(); - } - - @Override - public long getLocalCheckpoint() { - return seqNoStats.getLocalCheckpoint(); - } - - @Override - public void waitForOpsToComplete(long seqNo) { } - - @Override - public void resetLocalCheckpoint(long newCheckpoint) { - throw new UnsupportedOperationException(); - } - - @Override - public SeqNoStats getSeqNoStats(long globalCheckpoint) { - return new SeqNoStats(seqNoStats.getMaxSeqNo(), seqNoStats.getLocalCheckpoint(), globalCheckpoint); - } - - @Override - public long getLastSyncedGlobalCheckpoint() { - return seqNoStats.getGlobalCheckpoint(); - } - - @Override - public long getIndexBufferRAMBytesUsed() { - return 0; - } - - @Override - public List segments(boolean verbose) { - return Arrays.asList(getSegmentInfo(lastCommittedSegmentInfos, verbose)); - } - - @Override - public void refresh(String source) throws EngineException {} - - @Override - public void writeIndexingBuffer() throws EngineException {} - - @Override - public boolean shouldPeriodicallyFlush() { - return false; - } - - @Override - public SyncedFlushResult syncFlush(String syncId, CommitId expectedCommitId) throws EngineException { - CommitId commitId = new CommitId(lastCommittedSegmentInfos.getId()); - if (commitId.equals(expectedCommitId)) { - return SyncedFlushResult.SUCCESS; - } - return SyncedFlushResult.COMMIT_MISMATCH; - } - - @Override - public CommitId flush(boolean force, boolean waitIfOngoing) throws EngineException { - return new CommitId(lastCommittedSegmentInfos.getId()); - } - - @Override - public CommitId flush() throws EngineException { - return flush(false, false); - } - - @Override - public void forceMerge(boolean flush, int maxNumSegments, boolean onlyExpungeDeletes, - boolean upgrade, boolean upgradeOnlyAncientSegments) { - throw new UnsupportedOperationException(); - } - - @Override - public IndexCommitRef acquireLastIndexCommit(boolean flushFirst) { - store.incRef(); - return new IndexCommitRef(indexCommit, store::decRef); - } - - @Override - public IndexCommitRef acquireSafeIndexCommit() { - return acquireLastIndexCommit(false); - } - - @Override - public void activateThrottling() { - } - - @Override - public void deactivateThrottling() { - } - - @Override - public void trimUnreferencedTranslogFiles() { - throw new UnsupportedOperationException(); - } - - @Override - public boolean shouldRollTranslogGeneration() { - return false; - } - - @Override - public void rollTranslogGeneration() throws EngineException { - throw new UnsupportedOperationException(); - } - - @Override - public void restoreLocalCheckpointFromTranslog() { - throw new UnsupportedOperationException(); - } - - @Override - public int fillSeqNoGaps(long primaryTerm) { - return 0; - } - - @Override - public Engine recoverFromTranslog(TranslogRecoveryRunner translogRecoveryRunner, long recoverUpToSeqNo) throws IOException { - return this; - } - - @Override - public void skipTranslogRecovery() { - } - - @Override - public void trimOperationsFromTranslog(long belowTerm, long aboveSeqNo) throws EngineException { - } - - @Override - public void maybePruneDeletes() { - } - - - - private static final class SeqIdGeneratingDirectoryReader extends FilterDirectoryReader { - private final long primaryTerm; - - SeqIdGeneratingDirectoryReader(DirectoryReader in, SeqIdGeneratingSubReaderWrapper wrapper) throws IOException { - super(in, wrapper); - primaryTerm = wrapper.primaryTerm; - } - - @Override - protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { - return wrap(in, primaryTerm); - } - - public static DirectoryReader wrap(DirectoryReader in, long primaryTerm) throws IOException { - Map ctxMap = new IdentityHashMap<>(); - for (LeafReaderContext leave : in.leaves()) { - ctxMap.put(leave.reader(), leave); - } - return new SeqIdGeneratingDirectoryReader(in, new SeqIdGeneratingSubReaderWrapper(ctxMap, primaryTerm)); - } - - @Override - public CacheHelper getReaderCacheHelper() { - return in.getReaderCacheHelper(); - } - - private abstract static class FakeNumericDocValues extends NumericDocValues { - private final int maxDoc; - int docID = -1; - - FakeNumericDocValues(int maxDoc) { - this.maxDoc = maxDoc; - } - - @Override - public int docID() { - return docID; - } - - @Override - public int nextDoc() { - if (docID+1 < maxDoc) { - docID++; - } else { - docID = NO_MORE_DOCS; - } - return docID; - } - - @Override - public int advance(int target) { - if (target >= maxDoc) { - docID = NO_MORE_DOCS; - } else { - docID = target; - } - return docID; - } - - @Override - public long cost() { - return maxDoc; - } - - @Override - public boolean advanceExact(int target) { - advance(target); - return docID != NO_MORE_DOCS; - } - } - - private static class SeqIdGeneratingSubReaderWrapper extends SubReaderWrapper { - private final Map ctxMap; - private final long primaryTerm; - - SeqIdGeneratingSubReaderWrapper(Map ctxMap, long primaryTerm) { - this.ctxMap = ctxMap; - this.primaryTerm = primaryTerm; - } - - @Override - public LeafReader wrap(LeafReader reader) { - LeafReaderContext leafReaderContext = ctxMap.get(reader); - final int docBase = leafReaderContext.docBase; - return new FilterLeafReader(reader) { - - @Override - public NumericDocValues getNumericDocValues(String field) throws IOException { - if (SeqNoFieldMapper.NAME.equals(field)) { - return new FakeNumericDocValues(maxDoc()) { - @Override - public long longValue() { - return docBase + docID; - } - }; - } else if (SeqNoFieldMapper.PRIMARY_TERM_NAME.equals(field)) { - return new FakeNumericDocValues(maxDoc()) { - @Override - public long longValue() { - return primaryTerm; - } - }; - } else if (VersionFieldMapper.NAME.equals(field)) { - return new FakeNumericDocValues(maxDoc()) { - @Override - public long longValue() { - return 1; - } - }; - } - return super.getNumericDocValues(field); - } - - @Override - public CacheHelper getCoreCacheHelper() { - return reader.getCoreCacheHelper(); - } - - @Override - public CacheHelper getReaderCacheHelper() { - return reader.getReaderCacheHelper(); - } - }; - } - } - } -} - diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 54163b0858b39..50601d4f7a2e3 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -20,10 +20,13 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.ShardLock; +import org.elasticsearch.index.engine.EngineFactory; +import org.elasticsearch.index.engine.ReadOnlyEngine; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.TranslogStats; import org.elasticsearch.repositories.FilterRepository; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; @@ -143,4 +146,15 @@ protected void closeInternal() { throw new UncheckedIOException(e); } } + + public static EngineFactory getEngineFactory() { + return config -> new ReadOnlyEngine(config, null, new TranslogStats(0, 0, 0, 0, 0), true, + reader -> { + try { + return SeqIdGeneratingFilterReader.wrap(reader, config.getPrimaryTermSupplier().getAsLong()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java index c1360b7cc5170..1e66b0d9f47af 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java @@ -52,7 +52,6 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; import org.elasticsearch.script.ScriptService; -import org.elasticsearch.snapshots.SourceOnlySnapshotEngine; import org.elasticsearch.snapshots.SourceOnlySnapshotRepository; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; @@ -358,8 +357,7 @@ public Map getRepositories(Environment env, NamedXCo @Override public Optional getEngineFactory(IndexSettings indexSettings) { if (indexSettings.getValue(SourceOnlySnapshotRepository.SOURCE_ONLY)) { - EngineFactory engineFactory = SourceOnlySnapshotEngine::new; - return Optional.of(engineFactory); + return Optional.of(SourceOnlySnapshotRepository.getEngineFactory()); } return Optional.empty(); } From afba90e60d7f7c69fb58a826399ecbff38ef92b6 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Sep 2018 15:47:27 +0200 Subject: [PATCH 28/44] fix compilation --- .../src/main/java/org/elasticsearch/indices/IndicesService.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index 1c83a880511cd..e9f674e14a501 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -396,7 +396,6 @@ public boolean hasIndex(Index index) { public IndexService indexService(Index index) { return indices.get(index.getUUID()); } - /** * Returns an IndexService for the specified index if exists otherwise a {@link IndexNotFoundException} is thrown. */ From b615020902cb4827402c07a26e2d47299076a274 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Sep 2018 16:13:17 +0200 Subject: [PATCH 29/44] bootstrap source only shards with new UUID and localCheckpoint == maxDoc --- .../index/engine/ReadOnlyEngine.java | 2 +- .../shard/AbstractIndexShardComponent.java | 2 -- .../org/elasticsearch/index/store/Store.java | 17 ++++++++-- .../SourceOnlySnapshotRepository.java | 34 ++++++++++++++++++- .../snapshots/SourceOnlySnapshotIT.java | 19 +++++++---- .../SourceOnlySnapshotShardTests.java | 6 ++-- 6 files changed, 65 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java index a55987d0a0082..41038af389bed 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java @@ -55,7 +55,7 @@ * * @see #ReadOnlyEngine(EngineConfig, SeqNoStats, TranslogStats, boolean, Function) */ -public final class ReadOnlyEngine extends Engine { +public class ReadOnlyEngine extends Engine { private final SegmentInfos lastCommittedSegmentInfos; private final SeqNoStats seqNoStats; diff --git a/server/src/main/java/org/elasticsearch/index/shard/AbstractIndexShardComponent.java b/server/src/main/java/org/elasticsearch/index/shard/AbstractIndexShardComponent.java index c56b0d740e71e..c967e94f7dae7 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/AbstractIndexShardComponent.java +++ b/server/src/main/java/org/elasticsearch/index/shard/AbstractIndexShardComponent.java @@ -51,6 +51,4 @@ public IndexSettings indexSettings() { public String nodeName() { return indexSettings.getNodeName(); } - - } 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 b892c5c01fefc..a0650feaa4255 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -1439,9 +1439,22 @@ public void createEmpty() throws IOException { */ public void bootstrapNewHistory() throws IOException { metadataLock.writeLock().lock(); - try (IndexWriter writer = newIndexWriter(IndexWriterConfig.OpenMode.APPEND, directory, null)) { - final Map userData = getUserData(writer); + try { + Map userData = readLastCommittedSegmentsInfo().getUserData(); final long maxSeqNo = Long.parseLong(userData.get(SequenceNumbers.MAX_SEQ_NO)); + bootstrapNewHistoryWithLocalCheckpoint(maxSeqNo); + } finally { + metadataLock.writeLock().unlock(); + } + } + + /** + * Marks an existing lucene index with a new history uuid and sets the given seqNo as the local checkpoint + * This is used to make sure no existing shard will recovery from this index using ops based recovery. + */ + public void bootstrapNewHistoryWithLocalCheckpoint(long maxSeqNo) throws IOException { + metadataLock.writeLock().lock(); + try (IndexWriter writer = newIndexWriter(IndexWriterConfig.OpenMode.APPEND, directory, null)) { final Map map = new HashMap<>(); map.put(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID()); map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(maxSeqNo)); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 50601d4f7a2e3..aed4d2612713b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -8,25 +8,36 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.SegmentCommitInfo; +import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.search.Query; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.SimpleFSDirectory; +import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.ShardLock; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.ReadOnlyEngine; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.TranslogStats; +import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.FilterRepository; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; @@ -34,7 +45,10 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; +import java.util.HashMap; import java.util.Iterator; +import java.util.Map; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; @@ -147,6 +161,19 @@ protected void closeInternal() { } } + @Override + public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { + super.restoreShard(shard, snapshotId, version, indexId, snapshotShardId, recoveryState); + Store store = shard.store(); + try { + // we will use the lucene doc ID as the seq ID so we set the local checkpoint to maxDoc with a new index UUID + SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); + store.bootstrapNewHistoryWithLocalCheckpoint(segmentInfos.totalMaxDoc()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + public static EngineFactory getEngineFactory() { return config -> new ReadOnlyEngine(config, null, new TranslogStats(0, 0, 0, 0, 0), true, reader -> { @@ -155,6 +182,11 @@ public static EngineFactory getEngineFactory() { } catch (IOException e) { throw new UncheckedIOException(e); } - }); + }) { + @Override + public GetResult get(Get get, BiFunction searcherFactory) throws EngineException { + throw new UnsupportedOperationException("_source only shards don't support _get calls"); + } + }; } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index e4e070bf559c1..e30af2230c383 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -75,8 +75,7 @@ public Map getRepositories(Environment env, NamedXCo @Override public Optional getEngineFactory(IndexSettings indexSettings) { if (indexSettings.getValue(SourceOnlySnapshotRepository.SOURCE_ONLY)) { - EngineFactory engineFactory = SourceOnlySnapshotEngine::new; - return Optional.of(engineFactory); + return Optional.of(SourceOnlySnapshotRepository.getEngineFactory()); } return Optional.empty(); } @@ -141,10 +140,18 @@ private void assertMappings(String sourceIdx, boolean requireRouting, boolean us assertTrue(mapping.containsKey("_doc")); String nested = useNested ? ",\"incorrect\":{\"type\":\"object\"},\"nested\":{\"type\":\"nested\",\"properties\":{\"value\":{\"type\":\"long\"}}}" : ""; - assertEquals("{\"_doc\":{\"enabled\":false," + - "\"_meta\":{\"_doc\":{\"properties\":{\"field1\":{\"type\":\"text\"," + - "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}" + nested + "}}}}}", - mapping.get("_doc").source().string()); + if (requireRouting) { + assertEquals("{\"_doc\":{\"enabled\":false," + + "\"_meta\":{\"_doc\":{\"_routing\":{\"required\":true}," + + "\"properties\":{\"field1\":{\"type\":\"text\"," + + "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}" + nested + + "}}}}}", mapping.get("_doc").source().string()); + } else { + assertEquals("{\"_doc\":{\"enabled\":false," + + "\"_meta\":{\"_doc\":{\"properties\":{\"field1\":{\"type\":\"text\"," + + "\"fields\":{\"keyword\":{\"type\":\"keyword\",\"ignore_above\":256}}}" + nested + "}}}}}", + mapping.get("_doc").source().string()); + } } private void assertHits(String index, int numDocsExpected) { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index d0fb6cfeaa9e3..ac55dbcfc69be 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -60,7 +60,7 @@ public class SourceOnlySnapshotShardTests extends IndexShardTestCase { public void testSourceIncomplete() throws IOException { ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), true, - ShardRoutingState.INITIALIZING, RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE); + ShardRoutingState.INITIALIZING, RecoverySource.EmptyStoreRecoverySource.INSTANCE); 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) @@ -195,7 +195,7 @@ public void testRestoreMinmal() throws IOException { ShardRoutingState.INITIALIZING, new RecoverySource.SnapshotRecoverySource(new Snapshot("src_only", snapshotId), Version.CURRENT, indexId.getId())); IndexMetaData metaData = runAsSnapshot(threadPool, () -> repository.getSnapshotIndexMetaData(snapshotId, indexId)); - IndexShard restoredShard = newShard(shardRouting, metaData, null, (c) -> new SourceOnlySnapshotEngine(c), () -> {}); + IndexShard restoredShard = newShard(shardRouting, metaData, null, SourceOnlySnapshotRepository.getEngineFactory(), () -> {}); restoredShard.mapperService().merge(shard.indexSettings().getIndexMetaData(), MapperService.MergeReason.MAPPING_RECOVERY); DiscoveryNode discoveryNode = new DiscoveryNode("node_g", buildNewFakeTransportAddress(), Version.CURRENT); restoredShard.markAsRecovering("test from snap", new RecoveryState(restoredShard.routingEntry(), discoveryNode, null)); @@ -236,7 +236,7 @@ public void testRestoreMinmal() throws IOException { public IndexShard reindex(DirectoryReader reader, MappingMetaData mapping) throws IOException { ShardRouting targetShardRouting = TestShardRouting.newShardRouting(new ShardId("target", "_na_", 0), randomAlphaOfLength(10), true, - ShardRoutingState.INITIALIZING, RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE); + ShardRoutingState.INITIALIZING, RecoverySource.EmptyStoreRecoverySource.INSTANCE); 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) From 2625996a55e71cfc94aee57faeba366ca74ccab2 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Sep 2018 16:58:21 +0200 Subject: [PATCH 30/44] fix imports --- .../snapshots/SourceOnlySnapshotRepository.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index aed4d2612713b..bf12ee49b4972 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -8,9 +8,6 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.search.Query; import org.apache.lucene.store.FSDirectory; @@ -20,17 +17,14 @@ import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.ShardLock; -import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.ReadOnlyEngine; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; @@ -45,9 +39,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; -import java.util.HashMap; import java.util.Iterator; -import java.util.Map; import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; @@ -162,7 +154,8 @@ protected void closeInternal() { } @Override - public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { + public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, + RecoveryState recoveryState) { super.restoreShard(shard, snapshotId, version, indexId, snapshotShardId, recoveryState); Store store = shard.store(); try { From 140eb50ca6d953f0e08939c8145ff46c6dfbef48 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Sep 2018 20:49:29 +0200 Subject: [PATCH 31/44] move all actions to the snapshot creation side --- .../SourceOnlySnapshotRepository.java | 121 +++++++++--------- .../elasticsearch/xpack/core/XPackPlugin.java | 2 +- .../snapshots/SourceOnlySnapshotIT.java | 2 +- 3 files changed, 64 insertions(+), 61 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index bf12ee49b4972..c72dfa1563326 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -12,9 +12,9 @@ import org.apache.lucene.search.Query; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.SimpleFSDirectory; -import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; @@ -26,12 +26,10 @@ import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.ReadOnlyEngine; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.TranslogStats; -import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.FilterRepository; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; @@ -40,6 +38,7 @@ import java.io.UncheckedIOException; import java.nio.file.Path; import java.util.Iterator; +import java.util.List; import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; @@ -68,52 +67,40 @@ public final class SourceOnlySnapshotRepository extends FilterRepository { private static final String SNAPSHOT_DIR_NAME = "_snapshot"; - public static Repository.Factory newFactory() { - return new Repository.Factory() { - - @Override - public Repository create(RepositoryMetaData metadata) { - throw new UnsupportedOperationException(); - } - - @Override - public Repository create(RepositoryMetaData metaData, Function typeLookup) throws Exception { - String delegateType = DELEGATE_TYPE.get(metaData.settings()); - if (Strings.hasLength(delegateType) == false) { - throw new IllegalArgumentException(DELEGATE_TYPE.getKey() + " must be set"); - } - Repository.Factory factory = typeLookup.apply(delegateType); - return new SourceOnlySnapshotRepository(factory.create(new RepositoryMetaData(metaData.name(), - delegateType, metaData.settings()), typeLookup)); - } - }; - } - SourceOnlySnapshotRepository(Repository in) { super(in); } @Override - public IndexMetaData getSnapshotIndexMetaData(SnapshotId snapshotId, IndexId index) throws IOException { - IndexMetaData snapshotIndexMetaData = super.getSnapshotIndexMetaData(snapshotId, index); - // TODO: can we lie about the index.version.created here and produce an index with a new version since we reindex anyway? - - // for a minimal restore we basically disable indexing on all fields and only create an index - // that is fully functional from an operational perspective. ie. it will have all metadata fields like version/ - // seqID etc. and an indexed ID field such that we can potentially perform updates on them or delete documents. - ImmutableOpenMap mappings = snapshotIndexMetaData.getMappings(); - Iterator> iterator = mappings.iterator(); - IndexMetaData.Builder builder = IndexMetaData.builder(snapshotIndexMetaData); - while (iterator.hasNext()) { - ObjectObjectCursor next = iterator.next(); - // we don't need to obey any routing here stuff is read-only anyway and get is disabled - final String mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + " } }"; - builder.putMapping(next.key, mapping); + public void initializeSnapshot(SnapshotId snapshotId, List indices, MetaData metaData) { + // we process the index metadata at snapshot time. This means if somebody tries to restore + // a _source only snapshot with a plain repository it will be just fine since we already set the + // required engine, that the index is read-only and the mapping to a default mapping + try { + MetaData.Builder builder = MetaData.builder(metaData); + for (IndexId indexId : indices) { + IndexMetaData index = metaData.index(indexId.getName()); + IndexMetaData.Builder indexMetadataBuilder = IndexMetaData.builder(index); + // for a minimal restore we basically disable indexing on all fields and only create an index + // that is fully functional from an operational perspective. ie. it will have all metadata fields like version/ + // seqID etc. and an indexed ID field such that we can potentially perform updates on them or delete documents. + ImmutableOpenMap mappings = index.getMappings(); + Iterator> iterator = mappings.iterator(); + while (iterator.hasNext()) { + ObjectObjectCursor next = iterator.next(); + // we don't need to obey any routing here stuff is read-only anyway and get is disabled + final String mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + " } }"; + indexMetadataBuilder.putMapping(next.key, mapping); + } + indexMetadataBuilder.settings(Settings.builder().put(index.getSettings()) + .put(SOURCE_ONLY.getKey(), true) + .put("index.blocks.write", true)); // read-only! + builder.put(indexMetadataBuilder); + } + super.initializeSnapshot(snapshotId, indices, builder.build()); + } catch (IOException ex) { + throw new UncheckedIOException(ex); } - builder.settings(Settings.builder().put(snapshotIndexMetaData.getSettings()) - .put(SOURCE_ONLY.getKey(), true) - .put("index.blocks.write", true)); // read-only! - return builder.build(); } @Override @@ -139,9 +126,11 @@ protected void closeInternal() { // SourceOnlySnapshot will take care of soft- and hard-deletes no special casing needed here SourceOnlySnapshot snapshot = new SourceOnlySnapshot(tempStore.directory(), querySupplier); snapshot.syncSnapshot(snapshotIndexCommit); + // we will use the lucene doc ID as the seq ID so we set the local checkpoint to maxDoc with a new index UUID + SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); + tempStore.bootstrapNewHistoryWithLocalCheckpoint(segmentInfos.totalMaxDoc()); store.incRef(); - try (DirectoryReader reader = DirectoryReader.open(tempStore.directory()); - ) { + try (DirectoryReader reader = DirectoryReader.open(tempStore.directory())) { IndexCommit indexCommit = reader.getIndexCommit(); super.snapshotShard(shard, tempStore, snapshotId, indexId, indexCommit, snapshotStatus); } finally { @@ -153,20 +142,9 @@ protected void closeInternal() { } } - @Override - public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, - RecoveryState recoveryState) { - super.restoreShard(shard, snapshotId, version, indexId, snapshotShardId, recoveryState); - Store store = shard.store(); - try { - // we will use the lucene doc ID as the seq ID so we set the local checkpoint to maxDoc with a new index UUID - SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); - store.bootstrapNewHistoryWithLocalCheckpoint(segmentInfos.totalMaxDoc()); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - + /** + * Returns an {@link EngineFactory} for the source only snapshots. + */ public static EngineFactory getEngineFactory() { return config -> new ReadOnlyEngine(config, null, new TranslogStats(0, 0, 0, 0, 0), true, reader -> { @@ -176,10 +154,35 @@ public static EngineFactory getEngineFactory() { throw new UncheckedIOException(e); } }) { + @Override public GetResult get(Get get, BiFunction searcherFactory) throws EngineException { throw new UnsupportedOperationException("_source only shards don't support _get calls"); } }; } + + /** + * Returns a new source only repository factory + */ + public static Repository.Factory newRepositoryFactory() { + return new Repository.Factory() { + + @Override + public Repository create(RepositoryMetaData metadata) { + throw new UnsupportedOperationException(); + } + + @Override + public Repository create(RepositoryMetaData metaData, Function typeLookup) throws Exception { + String delegateType = DELEGATE_TYPE.get(metaData.settings()); + if (Strings.hasLength(delegateType) == false) { + throw new IllegalArgumentException(DELEGATE_TYPE.getKey() + " must be set"); + } + Repository.Factory factory = typeLookup.apply(delegateType); + return new SourceOnlySnapshotRepository(factory.create(new RepositoryMetaData(metaData.name(), + delegateType, metaData.settings()), typeLookup)); + } + }; + } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java index 1e66b0d9f47af..ca76e71e0529c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java @@ -351,7 +351,7 @@ default Optional getRequiredFeature() { @Override public Map getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry) { - return Collections.singletonMap("source", SourceOnlySnapshotRepository.newFactory()); + return Collections.singletonMap("source", SourceOnlySnapshotRepository.newRepositoryFactory()); } @Override diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index e30af2230c383..1db35e1cbf206 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -70,7 +70,7 @@ protected Collection> getMockPlugins() { public static final class MyPlugin extends Plugin implements RepositoryPlugin, EnginePlugin { @Override public Map getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry) { - return Collections.singletonMap("source", SourceOnlySnapshotRepository.newFactory()); + return Collections.singletonMap("source", SourceOnlySnapshotRepository.newRepositoryFactory()); } @Override public Optional getEngineFactory(IndexSettings indexSettings) { From 14f7caad4fc11c7e15b0f823e30124503960323c Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Sep 2018 21:04:47 +0200 Subject: [PATCH 32/44] rework docs --- .../rest-api/repository-source-only.asciidoc | 37 ++++++++++++------- 1 file changed, 24 insertions(+), 13 deletions(-) diff --git a/x-pack/docs/en/rest-api/repository-source-only.asciidoc b/x-pack/docs/en/rest-api/repository-source-only.asciidoc index 9475c5aa4538e..3e7bb854c4c7a 100644 --- a/x-pack/docs/en/rest-api/repository-source-only.asciidoc +++ b/x-pack/docs/en/rest-api/repository-source-only.asciidoc @@ -1,15 +1,34 @@ [[repository-src-only]] === Source Only Repository -The Source Only repository adds support for creating `_source` only snapshots using any other +The `_source`-only repository adds support for creating minimal snapshots of an index using any other available repository as it's storage backend. This allows using {ref}/modules-snapshots.html[Snapshot/Restore] -to create incremental, storage optimized, and minimal snapshots of an index. +to create incremental, storage optimized, and minimal snapshots of an index where only the `_source` and some metadata +is maintained in the snapshot. `_source`-only snapshots take approximately 30% to 50% of the size on disk compared +to a full snapshot. +[WARNING] +.`_source`-only snapshots are minimal and not searchable +================================================== + +Since the `_source`-only repository doesn't snapshot any index or doc-values structures but only stored +fields and index metadata, it's required to <> the data into a separate index +after the snapshot is restored. Be aware of the following limitations: + + * The restored index will be read-only and can only serve match-all scroll requests in order to reindex + + * Individual queries will succeed but won't match or filter any documents except of a `match_all` query. + + * The restored index has an empty mapping but allows access to the original mapping via the types top + level `meta` element. + + * `_source`-only snapshots are only supported if the `_source` field is enabled and no source-filtering is applied. +================================================== [[repository-src-only-usage]] ==== Configuration -The `_source` only repository always requires a delegate repository to be used as it's storage backend. +The `_source`-only repository always requires a delegate repository to be used as it's storage backend. In order to use the `fs` repository: [source,js] @@ -18,18 +37,10 @@ PUT _snapshot/my_src_only_repository { "type": "source", "settings": { - "delegate_type": "fs", + "delegate_type": "fs", <1> "location": "my_backup_location" } } ----------------------------------- // CONSOLE - -Since the `_source` only repository doesn't snapshot any index or doc-values structures but only stored -fields and index metadata, it's required to reindex the data after the restore process. The restored index is read-only -and can only serve scroll requests in oder to reindex. Individual queries will succeed but won't match or filter any -documents except of a `match_all` query. The restored will also have an empty mapping but allows access to the original -mapping via the types top level `meta` element. - -Source only snapshots take approximately 30% to 50% of the size on disk compared to a full snapshot. A source only snapshot -can only be taken if the original document source is maintained in the index and no source-filtering is applied. +<1> Specifies the delegate repository type to use as a backend. \ No newline at end of file From d69f675ba3a9e63a6641589cfd1b51226a65a2a2 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Sep 2018 21:13:09 +0200 Subject: [PATCH 33/44] fix line len --- .../elasticsearch/snapshots/SourceOnlySnapshotRepository.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index c72dfa1563326..516d6d5bdd454 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -89,7 +89,8 @@ public void initializeSnapshot(SnapshotId snapshotId, List indices, Met while (iterator.hasNext()) { ObjectObjectCursor next = iterator.next(); // we don't need to obey any routing here stuff is read-only anyway and get is disabled - final String mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + " } }"; + final String mapping = "{ \"" + next.key + "\": { \"enabled\": false, \"_meta\": " + next.value.source().string() + + " } }"; indexMetadataBuilder.putMapping(next.key, mapping); } indexMetadataBuilder.settings(Settings.builder().put(index.getSettings()) From 7c9b5eba9dfda8d6d47f0d469911e75c5300f0c5 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Sep 2018 22:03:05 +0200 Subject: [PATCH 34/44] add source only snap docs --- docs/reference/modules/snapshots.asciidoc | 44 ++++++++++++++++++ .../rest-api/repository-source-only.asciidoc | 46 ------------------- 2 files changed, 44 insertions(+), 46 deletions(-) delete mode 100644 x-pack/docs/en/rest-api/repository-source-only.asciidoc diff --git a/docs/reference/modules/snapshots.asciidoc b/docs/reference/modules/snapshots.asciidoc index 0562a677a8d3f..e42601c967d86 100644 --- a/docs/reference/modules/snapshots.asciidoc +++ b/docs/reference/modules/snapshots.asciidoc @@ -207,6 +207,50 @@ repositories.url.allowed_urls: ["http://www.example.org/root/*", "https://*.mydo URL repositories with `file:` URLs can only point to locations registered in the `path.repo` setting similar to shared file system repository. +[float] +===== Source Only Repository + +The `_source`-only repository adds support for creating minimal snapshots of an index using any other +available repository as it's storage backend. This allows using {ref}/modules-snapshots.html[Snapshot/Restore] +to create incremental, storage optimized, and minimal snapshots of an index where only the `_source` and some metadata +is maintained in the snapshot. `_source`-only snapshots take approximately 30% to 50% of the size on disk compared +to a full snapshot. + +[WARNING] +.`_source`-only snapshots are minimal and not searchable +================================================== + +Since the `_source`-only repository doesn't snapshot any index or doc-values structures but only stored +fields and index metadata, it's required to <> the data into a separate index +after the snapshot is restored. Be aware of the following limitations: + + * The restored index will be read-only and can only serve match-all scroll requests in order to reindex + + * Individual queries will succeed but won't match or filter any documents except of a `match_all` query. + + * The restored index has an empty mapping but allows access to the original mapping via the types top + level `meta` element. + + * `_source`-only snapshots are only supported if the `_source` field is enabled and no source-filtering is applied. +================================================== + +The `_source`-only repository always requires a delegate repository to be used as it's storage backend. +In order to use the `fs` repository: + +[source,js] +----------------------------------- +PUT _snapshot/my_src_only_repository +{ + "type": "source", + "settings": { + "delegate_type": "fs", <1> + "location": "my_backup_location" + } +} +----------------------------------- +// CONSOLE +<1> Specifies the delegate repository type to use as a backend. + [float] ===== Repository plugins diff --git a/x-pack/docs/en/rest-api/repository-source-only.asciidoc b/x-pack/docs/en/rest-api/repository-source-only.asciidoc deleted file mode 100644 index 3e7bb854c4c7a..0000000000000 --- a/x-pack/docs/en/rest-api/repository-source-only.asciidoc +++ /dev/null @@ -1,46 +0,0 @@ -[[repository-src-only]] -=== Source Only Repository - -The `_source`-only repository adds support for creating minimal snapshots of an index using any other -available repository as it's storage backend. This allows using {ref}/modules-snapshots.html[Snapshot/Restore] -to create incremental, storage optimized, and minimal snapshots of an index where only the `_source` and some metadata -is maintained in the snapshot. `_source`-only snapshots take approximately 30% to 50% of the size on disk compared -to a full snapshot. - -[WARNING] -.`_source`-only snapshots are minimal and not searchable -================================================== - -Since the `_source`-only repository doesn't snapshot any index or doc-values structures but only stored -fields and index metadata, it's required to <> the data into a separate index -after the snapshot is restored. Be aware of the following limitations: - - * The restored index will be read-only and can only serve match-all scroll requests in order to reindex - - * Individual queries will succeed but won't match or filter any documents except of a `match_all` query. - - * The restored index has an empty mapping but allows access to the original mapping via the types top - level `meta` element. - - * `_source`-only snapshots are only supported if the `_source` field is enabled and no source-filtering is applied. -================================================== - -[[repository-src-only-usage]] -==== Configuration - -The `_source`-only repository always requires a delegate repository to be used as it's storage backend. -In order to use the `fs` repository: - -[source,js] ------------------------------------ -PUT _snapshot/my_src_only_repository -{ - "type": "source", - "settings": { - "delegate_type": "fs", <1> - "location": "my_backup_location" - } -} ------------------------------------ -// CONSOLE -<1> Specifies the delegate repository type to use as a backend. \ No newline at end of file From fe817cafcc7ed20397205febf7d3b30278fe6d58 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Sep 2018 22:08:07 +0200 Subject: [PATCH 35/44] add xpack role to docs --- docs/reference/modules/snapshots.asciidoc | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/reference/modules/snapshots.asciidoc b/docs/reference/modules/snapshots.asciidoc index e42601c967d86..6402bf32eb532 100644 --- a/docs/reference/modules/snapshots.asciidoc +++ b/docs/reference/modules/snapshots.asciidoc @@ -208,6 +208,7 @@ URL repositories with `file:` URLs can only point to locations registered in the shared file system repository. [float] +[role="xpack"] ===== Source Only Repository The `_source`-only repository adds support for creating minimal snapshots of an index using any other From 59e5dbfb89b6cbb8434712237663091dda797597 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 11 Sep 2018 22:40:41 +0200 Subject: [PATCH 36/44] fix docs tests --- docs/reference/modules/snapshots.asciidoc | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/reference/modules/snapshots.asciidoc b/docs/reference/modules/snapshots.asciidoc index 6402bf32eb532..2da2cefc0240d 100644 --- a/docs/reference/modules/snapshots.asciidoc +++ b/docs/reference/modules/snapshots.asciidoc @@ -209,6 +209,7 @@ shared file system repository. [float] [role="xpack"] +[testenv="basic"] ===== Source Only Repository The `_source`-only repository adds support for creating minimal snapshots of an index using any other @@ -250,6 +251,8 @@ PUT _snapshot/my_src_only_repository } ----------------------------------- // CONSOLE +// TEST[continued] + <1> Specifies the delegate repository type to use as a backend. [float] From 12473d53dd51da4ef5b1b78bf932d468cfa406ab Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Sep 2018 09:26:03 +0200 Subject: [PATCH 37/44] reword docs --- docs/reference/modules/snapshots.asciidoc | 39 +++++++++++------------ 1 file changed, 19 insertions(+), 20 deletions(-) diff --git a/docs/reference/modules/snapshots.asciidoc b/docs/reference/modules/snapshots.asciidoc index 2da2cefc0240d..a66542a724e03 100644 --- a/docs/reference/modules/snapshots.asciidoc +++ b/docs/reference/modules/snapshots.asciidoc @@ -212,32 +212,33 @@ shared file system repository. [testenv="basic"] ===== Source Only Repository -The `_source`-only repository adds support for creating minimal snapshots of an index using any other -available repository as it's storage backend. This allows using {ref}/modules-snapshots.html[Snapshot/Restore] -to create incremental, storage optimized, and minimal snapshots of an index where only the `_source` and some metadata -is maintained in the snapshot. `_source`-only snapshots take approximately 30% to 50% of the size on disk compared -to a full snapshot. - -[WARNING] -.`_source`-only snapshots are minimal and not searchable +A source repository enables you to create minimal, source-only snapshots that take up to 50% less space on disk. +Source only snapshots contain stored fields and index metadata. They do not include index or doc values structures +and are not searchable when restored. After restoring a source-only snapshot, you must <> +the data into a new index. + +Source repositories delegate to another snapshot repository for storage. + + +[IMPORTANT] ================================================== -Since the `_source`-only repository doesn't snapshot any index or doc-values structures but only stored -fields and index metadata, it's required to <> the data into a separate index -after the snapshot is restored. Be aware of the following limitations: +Source only snapshots are only supported if the `_source` field is enabled and no source-filtering is applied. +When you restore a source only snapshot: - * The restored index will be read-only and can only serve match-all scroll requests in order to reindex + * The restored index is read-only and can only serve `match_all` search or scroll requests to enable reindexing. - * Individual queries will succeed but won't match or filter any documents except of a `match_all` query. + * Queries other than `match_all` will return no results. - * The restored index has an empty mapping but allows access to the original mapping via the types top + * `_get` requests are not supported. + + * The mapping of the restored index is empty, but the original mapping is available from the types top level `meta` element. - * `_source`-only snapshots are only supported if the `_source` field is enabled and no source-filtering is applied. ================================================== -The `_source`-only repository always requires a delegate repository to be used as it's storage backend. -In order to use the `fs` repository: +When you create a source repository, you must specify the type and name of the delegate repository +where the snapshots will be stored: [source,js] ----------------------------------- @@ -245,7 +246,7 @@ PUT _snapshot/my_src_only_repository { "type": "source", "settings": { - "delegate_type": "fs", <1> + "delegate_type": "fs", "location": "my_backup_location" } } @@ -253,8 +254,6 @@ PUT _snapshot/my_src_only_repository // CONSOLE // TEST[continued] -<1> Specifies the delegate repository type to use as a backend. - [float] ===== Repository plugins From 71d2e58fc84d8a2420a4f8dfddb261ffd3f434cb Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Sep 2018 09:27:04 +0200 Subject: [PATCH 38/44] fix nit --- server/src/main/java/org/elasticsearch/index/store/Store.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 a0650feaa4255..106f792433661 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -1452,12 +1452,12 @@ public void bootstrapNewHistory() throws IOException { * Marks an existing lucene index with a new history uuid and sets the given seqNo as the local checkpoint * This is used to make sure no existing shard will recovery from this index using ops based recovery. */ - public void bootstrapNewHistoryWithLocalCheckpoint(long maxSeqNo) throws IOException { + public void bootstrapNewHistoryWithLocalCheckpoint(long localCheckpoint) throws IOException { metadataLock.writeLock().lock(); try (IndexWriter writer = newIndexWriter(IndexWriterConfig.OpenMode.APPEND, directory, null)) { final Map map = new HashMap<>(); map.put(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID()); - map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(maxSeqNo)); + map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(localCheckpoint)); updateCommitData(writer, map); } finally { metadataLock.writeLock().unlock(); From 5be884449c55b5a423ec100fe72dfc3bdf099aa6 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Sep 2018 09:45:59 +0200 Subject: [PATCH 39/44] apply feedback --- .../java/org/elasticsearch/index/store/Store.java | 12 ++++++++---- .../snapshots/SourceOnlySnapshotRepository.java | 4 ++-- .../snapshots/SourceOnlySnapshotIT.java | 2 +- .../rest-api-spec/test/snapshot/10_basic.yml | 11 +++++++++++ 4 files changed, 22 insertions(+), 7 deletions(-) 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 106f792433661..8e57caad3b4a5 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -1442,22 +1442,26 @@ public void bootstrapNewHistory() throws IOException { try { Map userData = readLastCommittedSegmentsInfo().getUserData(); final long maxSeqNo = Long.parseLong(userData.get(SequenceNumbers.MAX_SEQ_NO)); - bootstrapNewHistoryWithLocalCheckpoint(maxSeqNo); + bootstrapNewHistory(maxSeqNo); } finally { metadataLock.writeLock().unlock(); } } /** - * Marks an existing lucene index with a new history uuid and sets the given seqNo as the local checkpoint + * Marks an existing lucene index with a new history uuid and sets the given maxSeqNo as the local checkpoint + * as well as the maximum sequence number. * This is used to make sure no existing shard will recovery from this index using ops based recovery. + * @see SequenceNumbers#LOCAL_CHECKPOINT_KEY + * @see SequenceNumbers#MAX_SEQ_NO */ - public void bootstrapNewHistoryWithLocalCheckpoint(long localCheckpoint) throws IOException { + public void bootstrapNewHistory(long maxSeqNo) throws IOException { metadataLock.writeLock().lock(); try (IndexWriter writer = newIndexWriter(IndexWriterConfig.OpenMode.APPEND, directory, null)) { final Map map = new HashMap<>(); map.put(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID()); - map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(localCheckpoint)); + map.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); + map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(maxSeqNo)); updateCommitData(writer, map); } finally { metadataLock.writeLock().unlock(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 516d6d5bdd454..d7823085a3c47 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -82,7 +82,7 @@ public void initializeSnapshot(SnapshotId snapshotId, List indices, Met IndexMetaData index = metaData.index(indexId.getName()); IndexMetaData.Builder indexMetadataBuilder = IndexMetaData.builder(index); // for a minimal restore we basically disable indexing on all fields and only create an index - // that is fully functional from an operational perspective. ie. it will have all metadata fields like version/ + // that is valid from an operational perspective. ie. it will have all metadata fields like version/ // seqID etc. and an indexed ID field such that we can potentially perform updates on them or delete documents. ImmutableOpenMap mappings = index.getMappings(); Iterator> iterator = mappings.iterator(); @@ -129,7 +129,7 @@ protected void closeInternal() { snapshot.syncSnapshot(snapshotIndexCommit); // we will use the lucene doc ID as the seq ID so we set the local checkpoint to maxDoc with a new index UUID SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); - tempStore.bootstrapNewHistoryWithLocalCheckpoint(segmentInfos.totalMaxDoc()); + tempStore.bootstrapNewHistory(segmentInfos.totalMaxDoc()); store.incRef(); try (DirectoryReader reader = DirectoryReader.open(tempStore.directory())) { IndexCommit indexCommit = reader.getIndexCommit(); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index 1db35e1cbf206..f3f1c9991ba6d 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -99,7 +99,7 @@ public void testSnapshotAndRestore() throws Exception { .addIds("" + randomIntBetween(0, builders.length))).get(), 0); // ensure we can not find hits it's a minimal restore assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); - // make sure deletes work + // make sure deletes do not work String idToDelete = "" + randomIntBetween(0, builders.length); expectThrows(ClusterBlockException.class, () -> client().prepareDelete(sourceIdx, "_doc", idToDelete) .setRouting("r" + idToDelete).get()); diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml index 96de08364538b..c0f161472b75c 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/snapshot/10_basic.yml @@ -71,3 +71,14 @@ setup: - match: { test_index.shards.0.index.files.reused: 0} - match: { test_index.shards.0.index.size.reused_in_bytes: 0} - gt: { test_index.shards.0.index.size.recovered_in_bytes: 0} + + - do: + search: + index: test_index + body: + query: + match_all: {} + + - match: {hits.total: 1 } + - length: {hits.hits: 1 } + - match: {hits.hits.0._id: "1" } From 18c468086d45a3b60e8e41baea7bb298b6d03393 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Sep 2018 10:08:05 +0200 Subject: [PATCH 40/44] Make sure all queries and get requests other than match_all fail --- docs/reference/modules/snapshots.asciidoc | 4 +-- .../index/engine/ReadOnlyEngine.java | 2 +- .../SeqIdGeneratingFilterReader.java | 12 +++++++ .../SourceOnlySnapshotRepository.java | 10 +----- .../SourceOnlySnapshotShardTests.java | 34 +++++++++++++++++-- 5 files changed, 46 insertions(+), 16 deletions(-) diff --git a/docs/reference/modules/snapshots.asciidoc b/docs/reference/modules/snapshots.asciidoc index a66542a724e03..ba6adf1d35fb8 100644 --- a/docs/reference/modules/snapshots.asciidoc +++ b/docs/reference/modules/snapshots.asciidoc @@ -228,9 +228,7 @@ When you restore a source only snapshot: * The restored index is read-only and can only serve `match_all` search or scroll requests to enable reindexing. - * Queries other than `match_all` will return no results. - - * `_get` requests are not supported. + * Queries other than `match_all` and `_get` requests are not supported. * The mapping of the restored index is empty, but the original mapping is available from the types top level `meta` element. diff --git a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java index 41038af389bed..a55987d0a0082 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java @@ -55,7 +55,7 @@ * * @see #ReadOnlyEngine(EngineConfig, SeqNoStats, TranslogStats, boolean, Function) */ -public class ReadOnlyEngine extends Engine { +public final class ReadOnlyEngine extends Engine { private final SegmentInfos lastCommittedSegmentInfos; private final SeqNoStats seqNoStats; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SeqIdGeneratingFilterReader.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SeqIdGeneratingFilterReader.java index e5c5a93d92980..8dd5d9d98ca8f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SeqIdGeneratingFilterReader.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SeqIdGeneratingFilterReader.java @@ -11,6 +11,8 @@ import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.PointValues; +import org.apache.lucene.index.Terms; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.VersionFieldMapper; @@ -144,6 +146,16 @@ public CacheHelper getCoreCacheHelper() { public CacheHelper getReaderCacheHelper() { return reader.getReaderCacheHelper(); } + + @Override + public Terms terms(String field) { + throw new UnsupportedOperationException("_source only indices can't be searched or filtered"); + } + + @Override + public PointValues getPointValues(String field) { + throw new UnsupportedOperationException("_source only indices can't be searched or filtered"); + } }; } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index d7823085a3c47..a75d5f488ee2b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -22,7 +22,6 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.ShardLock; -import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.ReadOnlyEngine; import org.elasticsearch.index.shard.IndexShard; @@ -39,7 +38,6 @@ import java.nio.file.Path; import java.util.Iterator; import java.util.List; -import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; @@ -154,13 +152,7 @@ public static EngineFactory getEngineFactory() { } catch (IOException e) { throw new UncheckedIOException(e); } - }) { - - @Override - public GetResult get(Get get, BiFunction searcherFactory) throws EngineException { - throw new UnsupportedOperationException("_source only shards don't support _get calls"); - } - }; + }); } /** diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index ac55dbcfc69be..7058724ecf0bc 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -11,6 +11,13 @@ import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.Term; +import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.TopDocs; import org.apache.lucene.util.Bits; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; @@ -33,10 +40,13 @@ import org.elasticsearch.env.TestEnvironment; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.fieldvisitor.FieldsVisitor; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.IndexShardTestCase; @@ -47,6 +57,7 @@ import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.threadpool.ThreadPool; +import org.hamcrest.Matchers; import java.io.IOException; import java.nio.file.Path; @@ -206,11 +217,28 @@ public void testRestoreMinmal() throws IOException { assertEquals(IndexShardState.POST_RECOVERY, restoredShard.state()); restoredShard.refresh("test"); assertEquals(restoredShard.docStats().getCount(), shard.docStats().getCount()); - expectThrows(UnsupportedOperationException.class, - () -> restoredShard.get(new Engine.Get(false, false, "_doc", Integer.toString(0), - new Term("_id", Uid.encodeId(Integer.toString(0)))))); + EngineException engineException = expectThrows(EngineException.class, () -> restoredShard.get( + new Engine.Get(false, false, "_doc", Integer.toString(0), new Term("_id", Uid.encodeId(Integer.toString(0)))))); + assertEquals(engineException.getCause().getMessage(), "_source only indices can't be searched or filtered"); + SeqNoStats seqNoStats = restoredShard.seqNoStats(); + assertEquals(seqNoStats.getMaxSeqNo(), seqNoStats.getLocalCheckpoint()); final IndexShard targetShard; try (Engine.Searcher searcher = restoredShard.acquireSearcher("test")) { + assertEquals(searcher.reader().maxDoc(), seqNoStats.getLocalCheckpoint()); + TopDocs search = searcher.searcher().search(new MatchAllDocsQuery(), Integer.MAX_VALUE); + assertEquals(searcher.reader().numDocs(), search.totalHits.value); + search = searcher.searcher().search(new MatchAllDocsQuery(), Integer.MAX_VALUE, + new Sort(new SortField(SeqNoFieldMapper.NAME, SortField.Type.LONG)), false); + assertEquals(searcher.reader().numDocs(), search.totalHits.value); + long previous = -1; + for (ScoreDoc doc : search.scoreDocs) { + FieldDoc fieldDoc = (FieldDoc) doc; + assertEquals(1, fieldDoc.fields.length); + long current = (Long)fieldDoc.fields[0]; + assertThat(previous, Matchers.lessThan(current)); + previous = current; + } + expectThrows(UnsupportedOperationException.class, () -> searcher.searcher().search(new TermQuery(new Term("boom", "boom")), 1)); targetShard = reindex(searcher.getDirectoryReader(), new MappingMetaData("_doc", restoredShard.mapperService().documentMapper("_doc").meta())); } From f38402895f090b222f7798195ca97f967b98051c Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Sep 2018 10:10:15 +0200 Subject: [PATCH 41/44] fix comment --- .../java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index f3f1c9991ba6d..b4f852082e867 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -122,7 +122,7 @@ public void testSnapshotAndRestoreWithNested() throws Exception { .addIds("" + randomIntBetween(0, builders.length))).get(), 0); // ensure we can not find hits it's a minimal restore assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); - // make sure deletes work + // make sure deletes do not work String idToDelete = "" + randomIntBetween(0, builders.length); expectThrows(ClusterBlockException.class, () -> client().prepareDelete(sourceIdx, "_doc", idToDelete) .setRouting("r" + idToDelete).get()); From 88774976ba30f99a53dc1522964fd65cbaf392ea Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Sep 2018 13:20:37 +0200 Subject: [PATCH 42/44] fix tests to expect exception on query --- .../snapshots/SourceOnlySnapshotIT.java | 26 +++++++++++++------ 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index b4f852082e867..581fe8782ac76 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -11,6 +11,7 @@ import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.block.ClusterBlockException; @@ -33,8 +34,10 @@ import org.elasticsearch.repositories.Repository; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; +import org.elasticsearch.search.query.QueryPhaseExecutionException; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.transport.RemoteTransportException; import org.hamcrest.Matchers; import java.io.IOException; @@ -95,10 +98,15 @@ public void testSnapshotAndRestore() throws Exception { IndexRequestBuilder[] builders = snashotAndRestore(sourceIdx, 1, true, requireRouting, useNested); assertHits(sourceIdx, builders.length); assertMappings(sourceIdx, requireRouting, useNested); - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() - .addIds("" + randomIntBetween(0, builders.length))).get(), 0); - // ensure we can not find hits it's a minimal restore - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); + SearchPhaseExecutionException e = expectThrows(SearchPhaseExecutionException.class, () -> { + client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() + .addIds("" + randomIntBetween(0, builders.length))).get(); + }); + assertTrue(e.toString().contains("_source only indices can't be searched or filtered")); + + e = expectThrows(SearchPhaseExecutionException.class, () -> + client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get()); + assertTrue(e.toString().contains("_source only indices can't be searched or filtered")); // make sure deletes do not work String idToDelete = "" + randomIntBetween(0, builders.length); expectThrows(ClusterBlockException.class, () -> client().prepareDelete(sourceIdx, "_doc", idToDelete) @@ -118,10 +126,12 @@ public void testSnapshotAndRestoreWithNested() throws Exception { assertThat(indicesStatsResponse.getTotal().docs.getDeleted(), Matchers.greaterThan(0L)); assertHits(sourceIdx, builders.length); assertMappings(sourceIdx, requireRouting, true); - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery() - .addIds("" + randomIntBetween(0, builders.length))).get(), 0); - // ensure we can not find hits it's a minimal restore - assertHitCount(client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get(), 0); + SearchPhaseExecutionException e = expectThrows(SearchPhaseExecutionException.class, () -> + client().prepareSearch(sourceIdx).setQuery(QueryBuilders.idsQuery().addIds("" + randomIntBetween(0, builders.length))).get()); + assertTrue(e.toString().contains("_source only indices can't be searched or filtered")); + e = expectThrows(SearchPhaseExecutionException.class, () -> + client().prepareSearch(sourceIdx).setQuery(QueryBuilders.termQuery("field1", "bar")).get()); + assertTrue(e.toString().contains("_source only indices can't be searched or filtered")); // make sure deletes do not work String idToDelete = "" + randomIntBetween(0, builders.length); expectThrows(ClusterBlockException.class, () -> client().prepareDelete(sourceIdx, "_doc", idToDelete) From 78de6b5e63954fd6d26fe051bca0653a101fd550 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Sep 2018 13:38:11 +0200 Subject: [PATCH 43/44] fix imports --- .../java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index 581fe8782ac76..d622a7218f1ff 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -34,10 +34,8 @@ import org.elasticsearch.repositories.Repository; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; -import org.elasticsearch.search.query.QueryPhaseExecutionException; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.transport.RemoteTransportException; import org.hamcrest.Matchers; import java.io.IOException; From 5f6529f31e9a776812099024d9127d118c7a3d95 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 12 Sep 2018 14:16:51 +0200 Subject: [PATCH 44/44] add test that slices work too --- .../snapshots/SourceOnlySnapshotIT.java | 55 ++++++++++++------- 1 file changed, 36 insertions(+), 19 deletions(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index d622a7218f1ff..6d3a17e3ebfd3 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -19,6 +19,7 @@ import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.env.Environment; @@ -34,6 +35,7 @@ import org.elasticsearch.repositories.Repository; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; +import org.elasticsearch.search.slice.SliceBuilder; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; import org.hamcrest.Matchers; @@ -47,6 +49,7 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.ExecutionException; +import java.util.function.Consumer; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -166,26 +169,40 @@ private void assertHits(String index, int numDocsExpected) { SearchResponse searchResponse = client().prepareSearch(index) .addSort(SeqNoFieldMapper.NAME, SortOrder.ASC) .setSize(numDocsExpected).get(); - SearchHits hits = searchResponse.getHits(); - assertEquals(numDocsExpected, hits.totalHits); - IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats().clear().setDocs(true).get(); - long deleted = indicesStatsResponse.getTotal().docs.getDeleted(); - boolean allowHoles = deleted > 0; // we use indexRandom which might create holes ie. deleted docs - long i = 0; - for (SearchHit hit : hits) { - String id = hit.getId(); - Map sourceAsMap = hit.getSourceAsMap(); - assertTrue(sourceAsMap.containsKey("field1")); - if (allowHoles) { - long seqId = ((Number)hit.getSortValues()[0]).longValue(); - assertThat(i, Matchers.lessThanOrEqualTo(seqId)); - i = seqId + 1; - } else { - assertEquals(i++, hit.getSortValues()[0]); + Consumer assertConsumer = res -> { + SearchHits hits = res.getHits(); + IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats().clear().setDocs(true).get(); + long deleted = indicesStatsResponse.getTotal().docs.getDeleted(); + boolean allowHoles = deleted > 0; // we use indexRandom which might create holes ie. deleted docs + long i = 0; + for (SearchHit hit : hits) { + String id = hit.getId(); + Map sourceAsMap = hit.getSourceAsMap(); + assertTrue(sourceAsMap.containsKey("field1")); + if (allowHoles) { + long seqId = ((Number) hit.getSortValues()[0]).longValue(); + assertThat(i, Matchers.lessThanOrEqualTo(seqId)); + i = seqId + 1; + } else { + assertEquals(i++, hit.getSortValues()[0]); + } + assertEquals("bar " + id, sourceAsMap.get("field1")); + assertEquals("r" + id, hit.field("_routing").getValue()); } - assertEquals("bar " + id, sourceAsMap.get("field1")); - assertEquals("r" + id, hit.field("_routing").getValue()); - } + }; + assertConsumer.accept(searchResponse); + assertEquals(numDocsExpected, searchResponse.getHits().totalHits); + searchResponse = client().prepareSearch(index) + .addSort(SeqNoFieldMapper.NAME, SortOrder.ASC) + .setScroll("1m") + .slice(new SliceBuilder(SeqNoFieldMapper.NAME, randomIntBetween(0,1), 2)) + .setSize(randomIntBetween(1, 10)).get(); + do { + // now do a scroll with a slice + assertConsumer.accept(searchResponse); + searchResponse = client().prepareSearchScroll(searchResponse.getScrollId()).setScroll(TimeValue.timeValueMinutes(1)).get(); + } while (searchResponse.getHits().getHits().length > 0); + } private IndexRequestBuilder[] snashotAndRestore(String sourceIdx, int numShards, boolean minimal, boolean requireRouting, boolean