From d996d4e66a8c2d73e31cfbee4e1399d474a53133 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 2 Jan 2019 13:19:46 -0700 Subject: [PATCH 01/30] WIP --- .../GetCcrRestoreFileChunkAction.java | 92 +++++++++++++++++++ .../GetCcrRestoreFileChunkRequest.java | 81 ++++++++++++++++ .../PutCcrRestoreSessionRequest.java | 10 +- .../xpack/ccr/repository/CcrRepository.java | 2 +- .../repository/CcrRestoreSourceService.java | 11 +++ 5 files changed, 186 insertions(+), 10 deletions(-) create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java new file mode 100644 index 0000000000000..1493605a5cae1 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -0,0 +1,92 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.ccr.action.repositories; + +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.ccr.repository.CcrRestoreSourceService; + +import java.io.IOException; + +public class GetCcrRestoreFileChunkAction extends Action { + + public static final GetCcrRestoreFileChunkAction INSTANCE = new GetCcrRestoreFileChunkAction(); + public static final String NAME = "internal:admin/ccr/restore/file_chunk/get"; + + private GetCcrRestoreFileChunkAction() { + super(NAME); + } + + @Override + public GetCcrRestoreFileChunkResponse newResponse() { + throw new UnsupportedOperationException(); + } + + @Override + public Writeable.Reader getResponseReader() { + return GetCcrRestoreFileChunkResponse::new; + } + + + public static class TransportGetCcrRestoreFileChunkAction + extends HandledTransportAction { + + private final CcrRestoreSourceService restoreSourceService; + + @Inject + public TransportGetCcrRestoreFileChunkAction(ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, + CcrRestoreSourceService restoreSourceService) { + super(NAME, transportService, actionFilters, GetCcrRestoreFileChunkRequest::new); + this.restoreSourceService = restoreSourceService; + } + + @Override + protected void doExecute(Task task, GetCcrRestoreFileChunkRequest request, ActionListener listener) { + Engine.IndexCommitRef snapshot = restoreSourceService.getSession(request.getSessionUUID()); + try (IndexInput in = snapshot.getIndexCommit().getDirectory().openInput(request.getFileName(), IOContext.READONCE)) { + byte[] chunk = new byte[request.getSize()]; + in.seek(request.getOffset()); + in.readBytes(chunk, 0, request.getSize()); + listener.onResponse(new GetCcrRestoreFileChunkResponse(new BytesArray(chunk))); + } catch (IOException e) { + throw new ElasticsearchException(e); + } + } + } + + public static class GetCcrRestoreFileChunkResponse extends ActionResponse { + + private BytesReference chunk; + + GetCcrRestoreFileChunkResponse(StreamInput streamInput) throws IOException { + super(streamInput); + } + + GetCcrRestoreFileChunkResponse(BytesReference chunk) { + this.chunk = chunk; + } + + public BytesReference getChunk() { + return chunk; + } + } +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java new file mode 100644 index 0000000000000..917a764cc89c8 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java @@ -0,0 +1,81 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.ccr.action.repositories; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.transport.RemoteClusterAwareRequest; + +import java.io.IOException; + +public class GetCcrRestoreFileChunkRequest extends ActionRequest implements RemoteClusterAwareRequest { + + private DiscoveryNode node; + private String sessionUUID; + private String fileName; + private long offset; + private int size; + + @Override + public ActionRequestValidationException validate() { + return null; + } + + GetCcrRestoreFileChunkRequest(DiscoveryNode node, String sessionUUID, String fileName, long offset, int size) { + this.sessionUUID = sessionUUID; + this.fileName = fileName; + this.offset = offset; + this.size = size; + this.node = node; + } + + GetCcrRestoreFileChunkRequest(StreamInput in) throws IOException { + super.readFrom(in); + sessionUUID = in.readString(); + fileName = in.readString(); + offset = in.readVLong(); + size = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(sessionUUID); + out.writeString(fileName); + out.writeVLong(offset); + out.writeVInt(size); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + throw new UnsupportedOperationException(); + } + + public String getSessionUUID() { + return sessionUUID; + } + + public String getFileName() { + return fileName; + } + + public long getOffset() { + return offset; + } + + public int getSize() { + return size; + } + + @Override + public DiscoveryNode getPreferredTargetNode() { + return node; + } +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java index 2b94193f674af..6c72840d5d9f5 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java @@ -19,16 +19,14 @@ public class PutCcrRestoreSessionRequest extends SingleShardRequest Date: Thu, 3 Jan 2019 12:22:11 -0700 Subject: [PATCH 02/30] WIP --- .../PutCcrRestoreSessionAction.java | 16 ++++-- .../xpack/ccr/CcrRepositoryIT.java | 49 +++++++++++++++++++ 2 files changed, 62 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionAction.java index 2a1b354f5d8ea..07ee076135a1b 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionAction.java @@ -22,6 +22,7 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardNotFoundException; +import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -70,8 +71,8 @@ protected PutCcrRestoreSessionResponse shardOperation(PutCcrRestoreSessionReques if (indexShard == null) { throw new ShardNotFoundException(shardId); } - ccrRestoreService.openSession(request.getSessionUUID(), indexShard); - return new PutCcrRestoreSessionResponse(clusterService.localNode()); + Store.MetadataSnapshot storeFileMetaData = ccrRestoreService.openSession(request.getSessionUUID(), indexShard); + return new PutCcrRestoreSessionResponse(clusterService.localNode(), storeFileMetaData); } @Override @@ -95,33 +96,42 @@ protected ShardsIterator shards(ClusterState state, InternalRequest request) { public static class PutCcrRestoreSessionResponse extends ActionResponse { private DiscoveryNode node; + private Store.MetadataSnapshot storeFileMetaData; PutCcrRestoreSessionResponse() { } - PutCcrRestoreSessionResponse(DiscoveryNode node) { + PutCcrRestoreSessionResponse(DiscoveryNode node, Store.MetadataSnapshot storeFileMetaData) { this.node = node; + this.storeFileMetaData = storeFileMetaData; } PutCcrRestoreSessionResponse(StreamInput in) throws IOException { super(in); node = new DiscoveryNode(in); + storeFileMetaData = new Store.MetadataSnapshot(in); } @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); node = new DiscoveryNode(in); + storeFileMetaData = new Store.MetadataSnapshot(in); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); node.writeTo(out); + storeFileMetaData.writeTo(out); } public DiscoveryNode getNode() { return node; } + + public Store.MetadataSnapshot getStoreFileMetaData() { + return storeFileMetaData; + } } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java index 2d3ca857ff848..610850748773e 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; +import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.ClusterChangedEvent; @@ -200,6 +201,47 @@ public void testThatSessionIsRegisteredWithPrimaryShard() throws IOException { assertEquals(0, restoreInfo.failedShards()); } + public void testDocsAreRecovered() throws IOException { + String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster"; + String leaderIndex = "index1"; + String followerIndex = "index2"; + + final int numberOfPrimaryShards = randomIntBetween(1, 3); + final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, between(0, 1), + singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + assertAcked(leaderClient().admin().indices().prepareCreate(leaderIndex).setSource(leaderIndexSettings, XContentType.JSON)); + ensureLeaderGreen(leaderIndex); + + final RestoreService restoreService = getFollowerCluster().getCurrentMasterNodeInstance(RestoreService.class); + final ClusterService clusterService = getFollowerCluster().getCurrentMasterNodeInstance(ClusterService.class); + + final int firstBatchNumDocs = randomIntBetween(64, 128); + logger.info("Indexing [{}] docs as first batch", firstBatchNumDocs); + for (int i = 0; i < firstBatchNumDocs; i++) { + final String source = String.format(Locale.ROOT, "{\"f\":%d}", i); + leaderClient().prepareIndex("index1", "doc", Integer.toString(i)).setSource(source, XContentType.JSON).get(); + } + + leaderClient().admin().indices().prepareFlush(leaderIndex).setForce(true).setWaitIfOngoing(true).get(); + + Settings.Builder settingsBuilder = Settings.builder() + .put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, followerIndex) + .put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true); + RestoreService.RestoreRequest restoreRequest = new RestoreService.RestoreRequest(leaderClusterRepoName, + CcrRepository.LATEST, new String[]{leaderIndex}, indicesOptions, + "^(.*)$", followerIndex, Settings.EMPTY, new TimeValue(1, TimeUnit.HOURS), false, + false, true, settingsBuilder.build(), new String[0], + "restore_snapshot[" + leaderClusterRepoName + ":" + leaderIndex + "]"); + + PlainActionFuture future = PlainActionFuture.newFuture(); + restoreService.restoreSnapshot(restoreRequest, waitForRestore(clusterService, future)); + RestoreInfo restoreInfo = future.actionGet(); + + assertEquals(restoreInfo.totalShards(), restoreInfo.successfulShards()); + assertEquals(0, restoreInfo.failedShards()); + assertExpectedDocument(followerIndex, firstBatchNumDocs); + } + public void testFollowerMappingIsUpdated() throws IOException { String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster"; String leaderIndex = "index1"; @@ -254,6 +296,13 @@ public void testFollowerMappingIsUpdated() throws IOException { assertThat(XContentMapValues.extractValue("properties.k.type", mappingMetaData.sourceAsMap()), equalTo("long")); } + private void assertExpectedDocument(String followerIndex, final int value) { + final GetResponse getResponse = followerClient().prepareGet(followerIndex, "doc", Integer.toString(value)).get(); + assertTrue("Doc with id [" + value + "] is missing", getResponse.isExists()); + assertTrue((getResponse.getSource().containsKey("f"))); + assertThat(getResponse.getSource().get("f"), equalTo(value)); + } + private ActionListener waitForRestore(ClusterService clusterService, ActionListener listener) { return new ActionListener() { From ee37b25f2f152ac528b919ba0edc36d6c48f7700 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 3 Jan 2019 14:04:23 -0700 Subject: [PATCH 03/30] Work on restore process --- .../GetCcrRestoreFileChunkRequest.java | 2 +- .../xpack/ccr/repository/CcrRepository.java | 238 ++++++++++++++++-- .../xpack/ccr/CcrRepositoryIT.java | 2 +- 3 files changed, 223 insertions(+), 19 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java index 917a764cc89c8..043efc4c8076e 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java @@ -28,7 +28,7 @@ public ActionRequestValidationException validate() { return null; } - GetCcrRestoreFileChunkRequest(DiscoveryNode node, String sessionUUID, String fileName, long offset, int size) { + public GetCcrRestoreFileChunkRequest(DiscoveryNode node, String sessionUUID, String fileName, long offset, int size) { this.sessionUUID = sessionUUID; this.fileName = fileName; this.offset = offset; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index ef29aac822e7a..dfc2127c6e5a3 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -6,7 +6,16 @@ package org.elasticsearch.xpack.ccr.repository; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexFormatTooNewException; +import org.apache.lucene.index.IndexFormatTooOldException; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; @@ -20,17 +29,21 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardRecoveryException; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.IndexShardRestoreFailedException; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; @@ -44,11 +57,17 @@ import org.elasticsearch.xpack.ccr.action.CcrRequests; import org.elasticsearch.xpack.ccr.action.repositories.ClearCcrRestoreSessionAction; import org.elasticsearch.xpack.ccr.action.repositories.ClearCcrRestoreSessionRequest; +import org.elasticsearch.xpack.ccr.action.repositories.GetCcrRestoreFileChunkAction; +import org.elasticsearch.xpack.ccr.action.repositories.GetCcrRestoreFileChunkRequest; import org.elasticsearch.xpack.ccr.action.repositories.PutCcrRestoreSessionAction; import org.elasticsearch.xpack.ccr.action.repositories.PutCcrRestoreSessionRequest; +import java.io.Closeable; import java.io.IOException; +import java.io.InputStream; +import java.io.UncheckedIOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -66,6 +85,8 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit public static final String NAME_PREFIX = "_ccr_"; private static final SnapshotId SNAPSHOT_ID = new SnapshotId(LATEST, LATEST); + private static final Logger logger = LogManager.getLogger(CcrRepository.class); + private final RepositoryMetaData metadata; private final String remoteClusterAlias; private final Client client; @@ -237,25 +258,17 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v store.decRef(); } - Store.MetadataSnapshot recoveryMetadata; - try { - recoveryMetadata = indexShard.snapshotStoreMetadata(); - } catch (IOException e) { - throw new IndexShardRecoveryException(shardId, "failed access store metadata", e); - } - Map ccrMetaData = indexShard.indexSettings().getIndexMetaData().getCustomData(Ccr.CCR_CUSTOM_METADATA_KEY); String leaderUUID = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_UUID_KEY); Index leaderIndex = new Index(shardId.getIndexName(), leaderUUID); ShardId leaderShardId = new ShardId(leaderIndex, shardId.getId()); Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); - String sessionUUID = UUIDs.randomBase64UUID(); - PutCcrRestoreSessionAction.PutCcrRestoreSessionResponse response = remoteClient.execute(PutCcrRestoreSessionAction.INSTANCE, - new PutCcrRestoreSessionRequest(sessionUUID, leaderShardId)).actionGet(); - DiscoveryNode node = response.getNode(); - // TODO: Implement file restore - closeSession(remoteClient, node, sessionUUID); + try (RestoreSession restoreSession = RestoreSession.openSession(remoteClient, leaderShardId, indexShard, shardId, recoveryState)) { + // TODO: Implement file restore + restoreSession.closeRemoteSession(); + } + maybeUpdateMappings(client, remoteClient, leaderIndex, indexShard.indexSettings()); } @@ -278,9 +291,200 @@ private void maybeUpdateMappings(Client localClient, Client remoteClient, Index } } - private void closeSession(Client remoteClient, DiscoveryNode node, String sessionUUID) { - ClearCcrRestoreSessionRequest clearRequest = new ClearCcrRestoreSessionRequest(sessionUUID, node); - ClearCcrRestoreSessionAction.ClearCcrRestoreSessionResponse response = - remoteClient.execute(ClearCcrRestoreSessionAction.INSTANCE, clearRequest).actionGet(); + private static class RestoreSession implements Closeable { + + private final int BUFFER_SIZE = 1 << 16; + + private final Client remoteClient; + private final String sessionUUID; + private final DiscoveryNode node; + private final IndexShard indexShard; + private final ShardId shardId; + private final Store store; + private final RecoveryState recoveryState; + private final Store.MetadataSnapshot sourceMetaData; + + RestoreSession(Client remoteClient, String sessionUUID, DiscoveryNode node, IndexShard indexShard, ShardId shardId, + RecoveryState recoveryState, Store.MetadataSnapshot sourceMetaData) { + this.remoteClient = remoteClient; + this.sessionUUID = sessionUUID; + this.node = node; + this.indexShard = indexShard; + this.shardId = shardId; + this.store = indexShard.store(); + this.store.incRef(); + this.recoveryState = recoveryState; + this.sourceMetaData = sourceMetaData; + } + + static RestoreSession openSession(Client remoteClient, ShardId leaderShardId, IndexShard indexShard, ShardId shardId, + RecoveryState recoveryState) { + String sessionUUID = UUIDs.randomBase64UUID(); + PutCcrRestoreSessionAction.PutCcrRestoreSessionResponse response = remoteClient.execute(PutCcrRestoreSessionAction.INSTANCE, + new PutCcrRestoreSessionRequest(sessionUUID, leaderShardId)).actionGet(); + Store.MetadataSnapshot sourceFileMetaData = response.getStoreFileMetaData(); + return new RestoreSession(remoteClient, sessionUUID, response.getNode(), indexShard, shardId, recoveryState, + sourceFileMetaData); + } + + void closeRemoteSession() { + ClearCcrRestoreSessionRequest clearRequest = new ClearCcrRestoreSessionRequest(sessionUUID, node); + ClearCcrRestoreSessionAction.ClearCcrRestoreSessionResponse response = + remoteClient.execute(ClearCcrRestoreSessionAction.INSTANCE, clearRequest).actionGet(); + } + + void restoreFiles() { + Store.MetadataSnapshot recoveryMetadata; + try { + recoveryMetadata = indexShard.snapshotStoreMetadata(); + } catch (IOException e) { + throw new IndexShardRecoveryException(shardId, "failed access store metadata", e); + } + + final StoreFileMetaData restoredSegmentsFile = sourceMetaData.getSegmentsFile(); + if (restoredSegmentsFile == null) { + throw new IndexShardRestoreFailedException(shardId, "Ccr leader index has no segments file"); + } + + final Store.RecoveryDiff diff = sourceMetaData.recoveryDiff(recoveryMetadata); + for (StoreFileMetaData fileMetaData : diff.identical) { + recoveryState.getIndex().addFileDetail(fileMetaData.name(), fileMetaData.length(), true); + logger.trace("shard [{}] not recovering file [{}], exists in local store and is identical", shardId, fileMetaData.name()); + } + + List filesToRecover = new ArrayList<>(); + for (StoreFileMetaData fileMetaData : Iterables.concat(diff.different, diff.missing)) { + filesToRecover.add(fileMetaData); + recoveryState.getIndex().addFileDetail(fileMetaData.name(), fileMetaData.length(), false); + logger.trace("shard [{}] recovering file [{}]", shardId, fileMetaData.name()); + } + + if (filesToRecover.isEmpty()) { + logger.trace("shard [{}] no files to recover, all exist within the local store"); + } + + final List deleteIfExistFiles = Arrays.asList(getDirectoryFiles()); + for (final StoreFileMetaData fileToRecover : filesToRecover) { + // if a file with a same physical name already exist in the store we need to delete it + // before restoring it from the snapshot. We could be lenient and try to reuse the existing + // store files (and compare their names/length/checksum again with the snapshot files) but to + // avoid extra complexity we simply delete them and restore them again like StoreRecovery + // does with dangling indices. Any existing store file that is not restored from the snapshot + // will be clean up by RecoveryTarget.cleanFiles(). + final String name = fileToRecover.name(); + if (deleteIfExistFiles.contains(name)) { + logger.trace("shard [{}] deleting pre-existing file [{}]", shardId, name); + deleteFile(name); + } + + logger.trace("[{}] restoring file [{}]", shardId, fileToRecover.name()); + try { + restoreFile(fileToRecover, store); + } catch (IOException e) { + // TODO: Handle + } + } + } + + private String[] getDirectoryFiles() { + try { + return store.directory().listAll(); + } catch (IOException e) { + throw new IndexShardRecoveryException(shardId, "failed to list directory files", e); + } + } + + private void deleteFile(String name) { + try { + store.directory().deleteFile(name); + } catch (IOException e) { + throw new IndexShardRecoveryException(shardId, "failed to delete file", e); + } + } + + private void restoreFile(StoreFileMetaData fileToRecover, Store store) throws IOException { + boolean success = false; + + try (InputStream stream = new RestoreFileInputStream(remoteClient, sessionUUID, node, fileToRecover)) { + try (IndexOutput indexOutput = store.createVerifyingOutput(fileToRecover.name(), fileToRecover, IOContext.DEFAULT)) { + final byte[] buffer = new byte[BUFFER_SIZE]; + int length; + while ((length = stream.read(buffer)) > 0) { + indexOutput.writeBytes(buffer, 0, length); + recoveryState.getIndex().addRecoveredBytesToFile(fileToRecover.name(), length); + } + Store.verify(indexOutput); + indexOutput.close(); + store.directory().sync(Collections.singleton(fileToRecover.name())); + success = true; + } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { + try { + store.markStoreCorrupted(ex); + } catch (IOException e) { + logger.warn("store cannot be marked as corrupted", e); + } + throw ex; + } finally { + if (success == false) { + store.deleteQuiet(fileToRecover.name()); + } + } + } + } + + @Override + public void close() { + this.store.decRef(); + } + } + + private static class RestoreFileInputStream extends InputStream { + + private final DiscoveryNode node; + private final Client remoteClient; + private final String sessionUUID; + private final StoreFileMetaData fileInfo; + + private long pos = 0; + + private RestoreFileInputStream(Client remoteClient, String sessionUUID, DiscoveryNode node, StoreFileMetaData fileInfo) { + this.remoteClient = remoteClient; + this.sessionUUID = sessionUUID; + this.node = node; + this.fileInfo = fileInfo; + } + + + @Override + public int read() { + throw new UnsupportedOperationException(); + } + + @Override + public int read(byte[] bytes, int off, int len) throws IOException { + if (pos >= fileInfo.length()) { + return 0; + } + + GetCcrRestoreFileChunkRequest request = new GetCcrRestoreFileChunkRequest(node, sessionUUID, + fileInfo.name(), pos, (int) Math.min(fileInfo.length() - pos, len)); + BytesReference fileChunk = remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request).actionGet().getChunk(); + + + BytesRefIterator iterator = fileChunk.iterator(); + BytesRef ref; + int bytesCopied = 0; + while((ref = iterator.next()) != null) { + byte[] refBytes = ref.bytes; + System.arraycopy(refBytes, 0, bytes, off + bytesCopied, ref.length); + bytesCopied += ref.length; + } + + int chunkLength = fileChunk.length(); + assert bytesCopied == chunkLength : "Did not copy [" + bytesCopied + "] the expected [" + chunkLength + "] number of bytes"; + + pos += chunkLength; + return chunkLength; + } } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java index 610850748773e..f1fb734df0432 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java @@ -239,7 +239,7 @@ public void testDocsAreRecovered() throws IOException { assertEquals(restoreInfo.totalShards(), restoreInfo.successfulShards()); assertEquals(0, restoreInfo.failedShards()); - assertExpectedDocument(followerIndex, firstBatchNumDocs); +// assertExpectedDocument(followerIndex, firstBatchNumDocs); } public void testFollowerMappingIsUpdated() throws IOException { From cce5c9936387a36a0531a779aca7bf70c00a4989 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 3 Jan 2019 14:36:39 -0700 Subject: [PATCH 04/30] WIP --- .../java/org/elasticsearch/xpack/ccr/Ccr.java | 3 ++ .../GetCcrRestoreFileChunkAction.java | 10 +++++++ .../xpack/ccr/repository/CcrRepository.java | 28 +++++++++++++++---- .../xpack/ccr/CcrRepositoryIT.java | 6 ++-- 4 files changed, 40 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java index b3d2d05048211..370d017a4bde7 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java @@ -61,6 +61,7 @@ import org.elasticsearch.xpack.ccr.action.bulk.TransportBulkShardOperationsAction; import org.elasticsearch.xpack.ccr.action.repositories.ClearCcrRestoreSessionAction; import org.elasticsearch.xpack.ccr.action.repositories.DeleteInternalCcrRepositoryAction; +import org.elasticsearch.xpack.ccr.action.repositories.GetCcrRestoreFileChunkAction; import org.elasticsearch.xpack.ccr.action.repositories.PutCcrRestoreSessionAction; import org.elasticsearch.xpack.ccr.action.repositories.PutInternalCcrRepositoryAction; import org.elasticsearch.xpack.ccr.index.engine.FollowingEngineFactory; @@ -193,6 +194,8 @@ public List> getPersistentTasksExecutor(ClusterServic PutCcrRestoreSessionAction.TransportPutCcrRestoreSessionAction.class), new ActionHandler<>(ClearCcrRestoreSessionAction.INSTANCE, ClearCcrRestoreSessionAction.TransportDeleteCcrRestoreSessionAction.class), + new ActionHandler<>(GetCcrRestoreFileChunkAction.INSTANCE, + GetCcrRestoreFileChunkAction.TransportGetCcrRestoreFileChunkAction.class), // stats action new ActionHandler<>(FollowStatsAction.INSTANCE, TransportFollowStatsAction.class), new ActionHandler<>(CcrStatsAction.INSTANCE, TransportCcrStatsAction.class), diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index 1493605a5cae1..d5c8f1a0e436b 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -18,10 +18,12 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportActionProxy; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.ccr.repository.CcrRestoreSourceService; @@ -56,6 +58,7 @@ public static class TransportGetCcrRestoreFileChunkAction public TransportGetCcrRestoreFileChunkAction(ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, CcrRestoreSourceService restoreSourceService) { super(NAME, transportService, actionFilters, GetCcrRestoreFileChunkRequest::new); + TransportActionProxy.registerProxyAction(transportService, NAME, GetCcrRestoreFileChunkResponse::new); this.restoreSourceService = restoreSourceService; } @@ -79,6 +82,7 @@ public static class GetCcrRestoreFileChunkResponse extends ActionResponse { GetCcrRestoreFileChunkResponse(StreamInput streamInput) throws IOException { super(streamInput); + chunk = streamInput.readBytesReference(); } GetCcrRestoreFileChunkResponse(BytesReference chunk) { @@ -88,5 +92,11 @@ public static class GetCcrRestoreFileChunkResponse extends ActionResponse { public BytesReference getChunk() { return chunk; } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeBytesReference(chunk); + } } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index dfc2127c6e5a3..5d31e2bbc1541 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -8,10 +8,12 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; +import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.BytesRef; @@ -32,6 +34,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.index.Index; @@ -65,7 +68,6 @@ import java.io.Closeable; import java.io.IOException; import java.io.InputStream; -import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -265,7 +267,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); try (RestoreSession restoreSession = RestoreSession.openSession(remoteClient, leaderShardId, indexShard, shardId, recoveryState)) { - // TODO: Implement file restore + restoreSession.restoreFiles(); restoreSession.closeRemoteSession(); } @@ -377,13 +379,29 @@ void restoreFiles() { deleteFile(name); } - logger.trace("[{}] restoring file [{}]", shardId, fileToRecover.name()); + logger.trace("[{}] restoring file [{}]", shardId, name); try { restoreFile(fileToRecover, store); - } catch (IOException e) { - // TODO: Handle + } catch (Exception e) { + logger.info(() -> new ParameterizedMessage("shard [{}] failed to restore file [{}]", shardId, name), e); + throw new IndexShardRecoveryException(shardId, "failed to restore file", e); } } + + final SegmentInfos segmentCommitInfos; + try { + segmentCommitInfos = Lucene.pruneUnreferencedFiles(restoredSegmentsFile.name(), store.directory()); + } catch (IOException e) { + throw new IndexShardRestoreFailedException(shardId, "Failed to fetch index version after copying it over", e); + } + recoveryState.getIndex().updateVersion(segmentCommitInfos.getVersion()); + + // TODO: Do we need to update mappings prior to doing this? + try { + store.cleanupAndVerify("restore complete from remote", sourceMetaData); + } catch (IOException e) { + throw new IndexShardRecoveryException(shardId, "failed to cleanup and verify the store", e); + } } private String[] getDirectoryFiles() { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java index f1fb734df0432..aa7cc9536fe29 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java @@ -201,7 +201,7 @@ public void testThatSessionIsRegisteredWithPrimaryShard() throws IOException { assertEquals(0, restoreInfo.failedShards()); } - public void testDocsAreRecovered() throws IOException { + public void testDocsAreRecovered() throws Exception { String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster"; String leaderIndex = "index1"; String followerIndex = "index2"; @@ -239,7 +239,9 @@ public void testDocsAreRecovered() throws IOException { assertEquals(restoreInfo.totalShards(), restoreInfo.successfulShards()); assertEquals(0, restoreInfo.failedShards()); -// assertExpectedDocument(followerIndex, firstBatchNumDocs); + for (int i = 0; i < firstBatchNumDocs; ++i) { + assertExpectedDocument(followerIndex, i); + } } public void testFollowerMappingIsUpdated() throws IOException { From 11ce19cd2cdcbc2354daa48ab8f923d46fb45d76 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 3 Jan 2019 16:57:42 -0700 Subject: [PATCH 05/30] WIP --- .../GetCcrRestoreFileChunkAction.java | 22 ++-- .../PutCcrRestoreSessionRequest.java | 5 +- .../xpack/ccr/repository/CcrRepository.java | 111 ++++++------------ .../xpack/ccr/CcrRepositoryIT.java | 79 ++++++------- 4 files changed, 86 insertions(+), 131 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index d5c8f1a0e436b..c53babc309c6d 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -53,26 +53,30 @@ public static class TransportGetCcrRestoreFileChunkAction extends HandledTransportAction { private final CcrRestoreSourceService restoreSourceService; + private final ThreadPool threadPool; @Inject public TransportGetCcrRestoreFileChunkAction(ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, CcrRestoreSourceService restoreSourceService) { super(NAME, transportService, actionFilters, GetCcrRestoreFileChunkRequest::new); TransportActionProxy.registerProxyAction(transportService, NAME, GetCcrRestoreFileChunkResponse::new); + this.threadPool = threadPool; this.restoreSourceService = restoreSourceService; } @Override protected void doExecute(Task task, GetCcrRestoreFileChunkRequest request, ActionListener listener) { - Engine.IndexCommitRef snapshot = restoreSourceService.getSession(request.getSessionUUID()); - try (IndexInput in = snapshot.getIndexCommit().getDirectory().openInput(request.getFileName(), IOContext.READONCE)) { - byte[] chunk = new byte[request.getSize()]; - in.seek(request.getOffset()); - in.readBytes(chunk, 0, request.getSize()); - listener.onResponse(new GetCcrRestoreFileChunkResponse(new BytesArray(chunk))); - } catch (IOException e) { - throw new ElasticsearchException(e); - } + threadPool.generic().execute(() -> { + Engine.IndexCommitRef snapshot = restoreSourceService.getSession(request.getSessionUUID()); + try (IndexInput in = snapshot.getIndexCommit().getDirectory().openInput(request.getFileName(), IOContext.READONCE)) { + byte[] chunk = new byte[request.getSize()]; + in.seek(request.getOffset()); + in.readBytes(chunk, 0, request.getSize()); + listener.onResponse(new GetCcrRestoreFileChunkResponse(new BytesArray(chunk))); + } catch (IOException e) { + throw new ElasticsearchException(e); + } + }); } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java index 6c72840d5d9f5..da0c43116ee76 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java @@ -11,7 +11,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.store.Store; import java.io.IOException; @@ -48,11 +47,11 @@ public void writeTo(StreamOutput out) throws IOException { shardId.writeTo(out); } - public String getSessionUUID() { + String getSessionUUID() { return sessionUUID; } - public ShardId getShardId() { + ShardId getShardId() { return shardId; } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 5d31e2bbc1541..a09d486fc615a 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -67,7 +67,6 @@ import java.io.Closeable; import java.io.IOException; -import java.io.InputStream; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -266,6 +265,8 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v ShardId leaderShardId = new ShardId(leaderIndex, shardId.getId()); Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); + // TODO: There should be some local timeout. And if the remote cluster returns an unknown session + // response, we should be able to retry by creating a new session. try (RestoreSession restoreSession = RestoreSession.openSession(remoteClient, leaderShardId, indexShard, shardId, recoveryState)) { restoreSession.restoreFiles(); restoreSession.closeRemoteSession(); @@ -396,7 +397,6 @@ void restoreFiles() { } recoveryState.getIndex().updateVersion(segmentCommitInfos.getVersion()); - // TODO: Do we need to update mappings prior to doing this? try { store.cleanupAndVerify("restore complete from remote", sourceMetaData); } catch (IOException e) { @@ -422,30 +422,41 @@ private void deleteFile(String name) { private void restoreFile(StoreFileMetaData fileToRecover, Store store) throws IOException { boolean success = false; + long pos = 0; + + try (IndexOutput indexOutput = store.createVerifyingOutput(fileToRecover.name(), fileToRecover, IOContext.DEFAULT)) { + + GetCcrRestoreFileChunkRequest request = new GetCcrRestoreFileChunkRequest(node, sessionUUID, + fileToRecover.name(), pos, (int) Math.min(fileToRecover.length() - pos, BUFFER_SIZE)); + BytesReference fileChunk = remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request).actionGet().getChunk(); + + BytesRefIterator iterator = fileChunk.iterator(); + BytesRef ref; + int bytesWritten = 0; + while ((ref = iterator.next()) != null) { + byte[] refBytes = ref.bytes; + indexOutput.writeBytes(refBytes, 0, ref.length); + recoveryState.getIndex().addRecoveredBytesToFile(fileToRecover.name(), ref.length); + bytesWritten += ref.length; + } + + assert bytesWritten == fileChunk.length() : "Did not write [" + bytesWritten + "] the expected [" + fileChunk.length() + + "] number of bytes"; - try (InputStream stream = new RestoreFileInputStream(remoteClient, sessionUUID, node, fileToRecover)) { - try (IndexOutput indexOutput = store.createVerifyingOutput(fileToRecover.name(), fileToRecover, IOContext.DEFAULT)) { - final byte[] buffer = new byte[BUFFER_SIZE]; - int length; - while ((length = stream.read(buffer)) > 0) { - indexOutput.writeBytes(buffer, 0, length); - recoveryState.getIndex().addRecoveredBytesToFile(fileToRecover.name(), length); - } - Store.verify(indexOutput); - indexOutput.close(); - store.directory().sync(Collections.singleton(fileToRecover.name())); - success = true; - } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { - try { - store.markStoreCorrupted(ex); - } catch (IOException e) { - logger.warn("store cannot be marked as corrupted", e); - } - throw ex; - } finally { - if (success == false) { - store.deleteQuiet(fileToRecover.name()); - } + Store.verify(indexOutput); + indexOutput.close(); + store.directory().sync(Collections.singleton(fileToRecover.name())); + success = true; + } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { + try { + store.markStoreCorrupted(ex); + } catch (IOException e) { + logger.warn("store cannot be marked as corrupted", e); + } + throw ex; + } finally { + if (success == false) { + store.deleteQuiet(fileToRecover.name()); } } } @@ -455,54 +466,4 @@ public void close() { this.store.decRef(); } } - - private static class RestoreFileInputStream extends InputStream { - - private final DiscoveryNode node; - private final Client remoteClient; - private final String sessionUUID; - private final StoreFileMetaData fileInfo; - - private long pos = 0; - - private RestoreFileInputStream(Client remoteClient, String sessionUUID, DiscoveryNode node, StoreFileMetaData fileInfo) { - this.remoteClient = remoteClient; - this.sessionUUID = sessionUUID; - this.node = node; - this.fileInfo = fileInfo; - } - - - @Override - public int read() { - throw new UnsupportedOperationException(); - } - - @Override - public int read(byte[] bytes, int off, int len) throws IOException { - if (pos >= fileInfo.length()) { - return 0; - } - - GetCcrRestoreFileChunkRequest request = new GetCcrRestoreFileChunkRequest(node, sessionUUID, - fileInfo.name(), pos, (int) Math.min(fileInfo.length() - pos, len)); - BytesReference fileChunk = remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request).actionGet().getChunk(); - - - BytesRefIterator iterator = fileChunk.iterator(); - BytesRef ref; - int bytesCopied = 0; - while((ref = iterator.next()) != null) { - byte[] refBytes = ref.bytes; - System.arraycopy(refBytes, 0, bytes, off + bytesCopied, ref.length); - bytesCopied += ref.length; - } - - int chunkLength = fileChunk.length(); - assert bytesCopied == chunkLength : "Did not copy [" + bytesCopied + "] the expected [" + chunkLength + "] number of bytes"; - - pos += chunkLength; - return chunkLength; - } - } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java index aa7cc9536fe29..36e1027dc5f87 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java @@ -22,7 +22,6 @@ import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.IndexSettings; @@ -41,8 +40,8 @@ import java.io.IOException; import java.util.Locale; import java.util.Map; -import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import static java.util.Collections.singletonMap; import static org.elasticsearch.snapshots.RestoreService.restoreInProgress; @@ -160,47 +159,6 @@ public void testThatRepositoryRecoversEmptyIndexBasedOnLeaderSettings() throws I assertNotEquals(leaderMetadata.getIndexUUID(), followerMetadata.getIndexUUID()); } - public void testThatSessionIsRegisteredWithPrimaryShard() throws IOException { - String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster"; - String leaderIndex = "index1"; - String followerIndex = "index2"; - - final int numberOfPrimaryShards = randomIntBetween(1, 3); - final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, between(0, 1), - singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); - assertAcked(leaderClient().admin().indices().prepareCreate(leaderIndex).setSource(leaderIndexSettings, XContentType.JSON)); - ensureLeaderGreen(leaderIndex); - - final RestoreService restoreService = getFollowerCluster().getCurrentMasterNodeInstance(RestoreService.class); - final ClusterService clusterService = getFollowerCluster().getCurrentMasterNodeInstance(ClusterService.class); - - Settings.Builder settingsBuilder = Settings.builder() - .put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, followerIndex) - .put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true); - RestoreService.RestoreRequest restoreRequest = new RestoreService.RestoreRequest(leaderClusterRepoName, - CcrRepository.LATEST, new String[]{leaderIndex}, indicesOptions, - "^(.*)$", followerIndex, Settings.EMPTY, new TimeValue(1, TimeUnit.HOURS), false, - false, true, settingsBuilder.build(), new String[0], - "restore_snapshot[" + leaderClusterRepoName + ":" + leaderIndex + "]"); - - Set sessionsOpened = ConcurrentCollections.newConcurrentSet(); - Set sessionsClosed = ConcurrentCollections.newConcurrentSet(); - for (CcrRestoreSourceService restoreSourceService : getLeaderCluster().getDataNodeInstances(CcrRestoreSourceService.class)) { - restoreSourceService.addOpenSessionListener(sessionsOpened::add); - restoreSourceService.addCloseSessionListener(sessionsClosed::add); - } - - PlainActionFuture future = PlainActionFuture.newFuture(); - restoreService.restoreSnapshot(restoreRequest, waitForRestore(clusterService, future)); - RestoreInfo restoreInfo = future.actionGet(); - - assertEquals(numberOfPrimaryShards, sessionsOpened.size()); - assertEquals(numberOfPrimaryShards, sessionsClosed.size()); - - assertEquals(restoreInfo.totalShards(), restoreInfo.successfulShards()); - assertEquals(0, restoreInfo.failedShards()); - } - public void testDocsAreRecovered() throws Exception { String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster"; String leaderIndex = "index1"; @@ -215,7 +173,7 @@ public void testDocsAreRecovered() throws Exception { final RestoreService restoreService = getFollowerCluster().getCurrentMasterNodeInstance(RestoreService.class); final ClusterService clusterService = getFollowerCluster().getCurrentMasterNodeInstance(ClusterService.class); - final int firstBatchNumDocs = randomIntBetween(64, 128); + final int firstBatchNumDocs = randomIntBetween(1, 64); logger.info("Indexing [{}] docs as first batch", firstBatchNumDocs); for (int i = 0; i < firstBatchNumDocs; i++) { final String source = String.format(Locale.ROOT, "{\"f\":%d}", i); @@ -224,6 +182,36 @@ public void testDocsAreRecovered() throws Exception { leaderClient().admin().indices().prepareFlush(leaderIndex).setForce(true).setWaitIfOngoing(true).get(); + AtomicBoolean isRunning = new AtomicBoolean(true); + + // Concurrently index new docs with mapping changes + Thread thread = new Thread(() -> { + char[] chars = "abcdeghijklmnopqrstuvwxyz".toCharArray(); + for (char c : chars) { + if (isRunning.get() == false) { + break; + } + final String source; + long l = randomLongBetween(0, 50000); + if (randomBoolean()) { + source = String.format(Locale.ROOT, "{\"%c\":%d}", c, l); + } else { + source = String.format(Locale.ROOT, "{\"%c\":\"%d\"}", c, l); + } + for (int i = 64; i < 150; i++) { + if (isRunning.get() == false) { + break; + } + leaderClient().prepareIndex("index1", "doc", Long.toString(i)).setSource(source, XContentType.JSON).get(); + if (rarely()) { + leaderClient().admin().indices().prepareFlush(leaderIndex).setForce(true).get(); + } + } + leaderClient().admin().indices().prepareFlush(leaderIndex).setForce(true).setWaitIfOngoing(true).get(); + } + }); + thread.start(); + Settings.Builder settingsBuilder = Settings.builder() .put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, followerIndex) .put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true); @@ -242,6 +230,9 @@ public void testDocsAreRecovered() throws Exception { for (int i = 0; i < firstBatchNumDocs; ++i) { assertExpectedDocument(followerIndex, i); } + + isRunning.set(false); + thread.join(); } public void testFollowerMappingIsUpdated() throws IOException { From 6af7ec07e0b653b7dd8c5901c6f2f98de1511f51 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 3 Jan 2019 17:30:43 -0700 Subject: [PATCH 06/30] Fix checkstyle --- .../ccr/action/repositories/GetCcrRestoreFileChunkAction.java | 3 ++- .../org/elasticsearch/xpack/ccr/repository/CcrRepository.java | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index c53babc309c6d..184d1ac4ab0a4 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -65,7 +65,8 @@ public TransportGetCcrRestoreFileChunkAction(ThreadPool threadPool, TransportSer } @Override - protected void doExecute(Task task, GetCcrRestoreFileChunkRequest request, ActionListener listener) { + protected void doExecute(Task task, GetCcrRestoreFileChunkRequest request, + ActionListener listener) { threadPool.generic().execute(() -> { Engine.IndexCommitRef snapshot = restoreSourceService.getSession(request.getSessionUUID()); try (IndexInput in = snapshot.getIndexCommit().getDirectory().openInput(request.getFileName(), IOContext.READONCE)) { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index a09d486fc615a..f93cf7666dfe5 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -336,6 +336,7 @@ void closeRemoteSession() { remoteClient.execute(ClearCcrRestoreSessionAction.INSTANCE, clearRequest).actionGet(); } + @SuppressWarnings("unchecked") void restoreFiles() { Store.MetadataSnapshot recoveryMetadata; try { From f774c288acb77233cedd81d4a228f355de3a08eb Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 3 Jan 2019 17:40:58 -0700 Subject: [PATCH 07/30] Change --- .../xpack/ccr/repository/CcrRepository.java | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index f93cf7666dfe5..9fe8afde641bf 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -269,7 +269,6 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v // response, we should be able to retry by creating a new session. try (RestoreSession restoreSession = RestoreSession.openSession(remoteClient, leaderShardId, indexShard, shardId, recoveryState)) { restoreSession.restoreFiles(); - restoreSession.closeRemoteSession(); } maybeUpdateMappings(client, remoteClient, leaderIndex, indexShard.indexSettings()); @@ -330,12 +329,6 @@ static RestoreSession openSession(Client remoteClient, ShardId leaderShardId, In sourceFileMetaData); } - void closeRemoteSession() { - ClearCcrRestoreSessionRequest clearRequest = new ClearCcrRestoreSessionRequest(sessionUUID, node); - ClearCcrRestoreSessionAction.ClearCcrRestoreSessionResponse response = - remoteClient.execute(ClearCcrRestoreSessionAction.INSTANCE, clearRequest).actionGet(); - } - @SuppressWarnings("unchecked") void restoreFiles() { Store.MetadataSnapshot recoveryMetadata; @@ -465,6 +458,9 @@ private void restoreFile(StoreFileMetaData fileToRecover, Store store) throws IO @Override public void close() { this.store.decRef(); + ClearCcrRestoreSessionRequest clearRequest = new ClearCcrRestoreSessionRequest(sessionUUID, node); + ClearCcrRestoreSessionAction.ClearCcrRestoreSessionResponse response = + remoteClient.execute(ClearCcrRestoreSessionAction.INSTANCE, clearRequest).actionGet(); } } } From 359ad5b548e3ac70e2e97848f44904529cd44399 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 3 Jan 2019 18:07:27 -0700 Subject: [PATCH 08/30] Fix logger check --- .../org/elasticsearch/xpack/ccr/repository/CcrRepository.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 9fe8afde641bf..69bf9f2664fea 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -357,7 +357,7 @@ void restoreFiles() { } if (filesToRecover.isEmpty()) { - logger.trace("shard [{}] no files to recover, all exist within the local store"); + logger.trace("shard [{}] no files to recover, all exist within the local store", shardId); } final List deleteIfExistFiles = Arrays.asList(getDirectoryFiles()); From e18df0a79856e5e2902521026ab279a756ab1b60 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 4 Jan 2019 10:04:36 -0700 Subject: [PATCH 09/30] Changes --- .../GetCcrRestoreFileChunkAction.java | 28 +++++++++++-------- .../GetCcrRestoreFileChunkRequest.java | 12 ++++---- .../xpack/ccr/repository/CcrRepository.java | 8 ++++-- 3 files changed, 29 insertions(+), 19 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index 184d1ac4ab0a4..e589da6978aa2 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -8,7 +8,6 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionResponse; @@ -20,6 +19,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; @@ -67,15 +67,21 @@ public TransportGetCcrRestoreFileChunkAction(ThreadPool threadPool, TransportSer @Override protected void doExecute(Task task, GetCcrRestoreFileChunkRequest request, ActionListener listener) { - threadPool.generic().execute(() -> { - Engine.IndexCommitRef snapshot = restoreSourceService.getSession(request.getSessionUUID()); - try (IndexInput in = snapshot.getIndexCommit().getDirectory().openInput(request.getFileName(), IOContext.READONCE)) { - byte[] chunk = new byte[request.getSize()]; - in.seek(request.getOffset()); - in.readBytes(chunk, 0, request.getSize()); - listener.onResponse(new GetCcrRestoreFileChunkResponse(new BytesArray(chunk))); - } catch (IOException e) { - throw new ElasticsearchException(e); + threadPool.generic().execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + + @Override + protected void doRun() throws Exception { + Engine.IndexCommitRef snapshot = restoreSourceService.getSession(request.getSessionUUID()); + try (IndexInput in = snapshot.getIndexCommit().getDirectory().openInput(request.getFileName(), IOContext.READONCE)) { + byte[] chunk = new byte[request.getSize()]; + in.seek(request.getOffset()); + in.readBytes(chunk, 0, request.getSize()); + listener.onResponse(new GetCcrRestoreFileChunkResponse(new BytesArray(chunk))); + } } }); } @@ -83,7 +89,7 @@ protected void doExecute(Task task, GetCcrRestoreFileChunkRequest request, public static class GetCcrRestoreFileChunkResponse extends ActionResponse { - private BytesReference chunk; + private final BytesReference chunk; GetCcrRestoreFileChunkResponse(StreamInput streamInput) throws IOException { super(streamInput); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java index 043efc4c8076e..11a4f1e6491e2 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java @@ -18,10 +18,10 @@ public class GetCcrRestoreFileChunkRequest extends ActionRequest implements RemoteClusterAwareRequest { private DiscoveryNode node; - private String sessionUUID; - private String fileName; - private long offset; - private int size; + private final String sessionUUID; + private final String fileName; + private final long offset; + private final int size; @Override public ActionRequestValidationException validate() { @@ -30,14 +30,14 @@ public ActionRequestValidationException validate() { public GetCcrRestoreFileChunkRequest(DiscoveryNode node, String sessionUUID, String fileName, long offset, int size) { this.sessionUUID = sessionUUID; + this.node = node; this.fileName = fileName; this.offset = offset; this.size = size; - this.node = node; } GetCcrRestoreFileChunkRequest(StreamInput in) throws IOException { - super.readFrom(in); + super(in); sessionUUID = in.readString(); fileName = in.readString(); offset = in.readVLong(); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 69bf9f2664fea..9e3f5afaff2a5 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -329,7 +329,6 @@ static RestoreSession openSession(Client remoteClient, ShardId leaderShardId, In sourceFileMetaData); } - @SuppressWarnings("unchecked") void restoreFiles() { Store.MetadataSnapshot recoveryMetadata; try { @@ -350,7 +349,7 @@ void restoreFiles() { } List filesToRecover = new ArrayList<>(); - for (StoreFileMetaData fileMetaData : Iterables.concat(diff.different, diff.missing)) { + for (StoreFileMetaData fileMetaData : concat(diff)) { filesToRecover.add(fileMetaData); recoveryState.getIndex().addFileDetail(fileMetaData.name(), fileMetaData.length(), false); logger.trace("shard [{}] recovering file [{}]", shardId, fileMetaData.name()); @@ -398,6 +397,11 @@ void restoreFiles() { } } + @SuppressWarnings("unchecked") + private static Iterable concat(Store.RecoveryDiff diff) { + return Iterables.concat(diff.different, diff.missing); + } + private String[] getDirectoryFiles() { try { return store.directory().listAll(); From da5b2e78bacd32cb9bfeffb2a4330eb78f713683 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 4 Jan 2019 13:37:46 -0700 Subject: [PATCH 10/30] Add basic ref counting --- .../GetCcrRestoreFileChunkAction.java | 7 +++- .../repository/CcrRestoreSourceService.java | 37 +++++++++++++++---- 2 files changed, 34 insertions(+), 10 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index e589da6978aa2..4841142fd2234 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -19,6 +19,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.tasks.Task; @@ -75,8 +76,10 @@ public void onFailure(Exception e) { @Override protected void doRun() throws Exception { - Engine.IndexCommitRef snapshot = restoreSourceService.getSession(request.getSessionUUID()); - try (IndexInput in = snapshot.getIndexCommit().getDirectory().openInput(request.getFileName(), IOContext.READONCE)) { + CcrRestoreSourceService.Session session = restoreSourceService.getSession(request.getSessionUUID()); + Engine.IndexCommitRef refCommit = session.refCommit(); + try (IndexInput in = refCommit.getIndexCommit().getDirectory().openInput(request.getFileName(), IOContext.READONCE); + Releasable releasable = session::decRef) { byte[] chunk = new byte[request.getSize()]; in.seek(request.getOffset()); in.readBytes(chunk, 0, request.getSize()); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index bf50ef7cbca0f..92d19c084cb9d 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.Engine; @@ -133,33 +134,33 @@ public synchronized void closeSession(String sessionUUID) { IOUtils.closeWhileHandlingException(restore); } - // TODO: The Engine.IndexCommitRef might be closed by a different thread while it is in use. We need to - // look into the implications of this. - public Engine.IndexCommitRef getSession(String sessionUUID) { + public synchronized Session getSession(String sessionUUID) { RestoreContext restore = onGoingRestores.get(sessionUUID); if (restore == null) { logger.info("could not get session [{}] because session not found", sessionUUID); throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); } - return restore.commitRef; + Session session = restore.session; + session.incRef(); + return session; } private class RestoreContext implements Closeable { private final String sessionUUID; private final IndexShard indexShard; - private final Engine.IndexCommitRef commitRef; + private final Session session; private RestoreContext(String sessionUUID, IndexShard indexShard, Engine.IndexCommitRef commitRef) { this.sessionUUID = sessionUUID; this.indexShard = indexShard; - this.commitRef = commitRef; + this.session = new Session(commitRef); } Store.MetadataSnapshot getMetaData() throws IOException { indexShard.store().incRef(); try { - return indexShard.store().getMetadata(commitRef.getIndexCommit()); + return indexShard.store().getMetadata(session.refCommit().getIndexCommit()); } finally { indexShard.store().decRef(); } @@ -169,7 +170,7 @@ Store.MetadataSnapshot getMetaData() throws IOException { public void close() { assert Thread.holdsLock(CcrRestoreSourceService.this); removeSessionForShard(sessionUUID, indexShard); - IOUtils.closeWhileHandlingException(commitRef); + session.decRef(); } private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { @@ -183,4 +184,24 @@ private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { } } } + + public static class Session extends AbstractRefCounted { + + private static final String NAME = "ref-counted-session"; + private final Engine.IndexCommitRef commitRef; + + private Session(Engine.IndexCommitRef commitRef) { + super(NAME); + this.commitRef = commitRef; + } + + public Engine.IndexCommitRef refCommit() { + return commitRef; + } + + @Override + protected void closeInternal() { + IOUtils.closeWhileHandlingException(commitRef); + } + } } From 2209eacc6d4f64f89520371b26009bd728845e02 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 4 Jan 2019 17:26:24 -0700 Subject: [PATCH 11/30] Extract context --- .../blobstore/BlobRestoreContext.java | 291 ++++++++++++++++++ .../blobstore/BlobStoreRepository.java | 227 +------------- .../xpack/ccr/repository/CcrRepository.java | 20 +- 3 files changed, 319 insertions(+), 219 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/repositories/blobstore/BlobRestoreContext.java diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobRestoreContext.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobRestoreContext.java new file mode 100644 index 0000000000000..b0f75ef8becfe --- /dev/null +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobRestoreContext.java @@ -0,0 +1,291 @@ +/* + * 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.blobstore; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.IndexFormatTooNewException; +import org.apache.lucene.index.IndexFormatTooOldException; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.util.iterable.Iterables; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.IndexShardRestoreFailedException; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; +import org.elasticsearch.index.snapshots.blobstore.SnapshotFiles; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.store.StoreFileMetaData; +import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.snapshots.SnapshotId; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.unmodifiableMap; + +public abstract class BlobRestoreContext { + + private static final Logger logger = LogManager.getLogger(BlobRestoreContext.class); + + private final String repositoryName; + private final IndexShard indexShard; + private final RecoveryState recoveryState; + private final SnapshotId snapshotId; + private final ShardId shardId; + private final int bufferSize; + + /** + * Constructs new restore context + * + * @param indexShard shard to restore into + * @param snapshotId snapshot id + * @param recoveryState recovery state to report progress + * @param bufferSize buffer size for restore + */ + BlobRestoreContext(String repositoryName, IndexShard indexShard, SnapshotId snapshotId, RecoveryState recoveryState, int bufferSize) { + this.repositoryName = repositoryName; + this.recoveryState = recoveryState; + this.indexShard = indexShard; + this.snapshotId = snapshotId; + this.shardId = indexShard.shardId(); + this.bufferSize = bufferSize; + } + + /** + * Performs restore operation + */ + public void restore(SnapshotFiles snapshotFiles) throws IOException { + final Store store = indexShard.store(); + store.incRef(); + try { + logger.debug("[{}] [{}] restoring to [{}] ...", snapshotId, repositoryName, shardId); + + if (snapshotFiles.indexFiles().size() == 1 + && snapshotFiles.indexFiles().get(0).physicalName().startsWith("segments_") + && snapshotFiles.indexFiles().get(0).hasUnknownChecksum()) { + // If the shard has no documents, it will only contain a single segments_N file for the + // shard's snapshot. If we are restoring a snapshot created by a previous supported version, + // it is still possible that in that version, an empty shard has a segments_N file with an unsupported + // version (and no checksum), because we don't know the Lucene version to assign segments_N until we + // have written some data. Since the segments_N for an empty shard could have an incompatible Lucene + // version number and no checksum, even though the index itself is perfectly fine to restore, this + // empty shard would cause exceptions to be thrown. Since there is no data to restore from an empty + // shard anyway, we just create the empty shard here and then exit. + store.createEmpty(indexShard.indexSettings().getIndexMetaData().getCreationVersion().luceneVersion); + return; + } + + Store.MetadataSnapshot recoveryTargetMetadata; + try { + // this will throw an IOException if the store has no segments infos file. The + // store can still have existing files but they will be deleted just before being + // restored. + recoveryTargetMetadata = indexShard.snapshotStoreMetadata(); + } catch (IndexNotFoundException e) { + // happens when restore to an empty shard, not a big deal + logger.trace("[{}] [{}] restoring from to an empty shard", shardId, snapshotId); + recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; + } catch (IOException e) { + logger.warn(() -> new ParameterizedMessage("{} Can't read metadata from store, will not reuse any local file while restoring", shardId), e); + recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; + } + + final List filesToRecover = new ArrayList<>(); + final Map snapshotMetaData = new HashMap<>(); + final Map fileInfos = new HashMap<>(); + for (final BlobStoreIndexShardSnapshot.FileInfo fileInfo : snapshotFiles.indexFiles()) { + try { + // in 1.3.3 we added additional hashes for .si / segments_N files + // to ensure we don't double the space in the repo since old snapshots + // don't have this hash we try to read that hash from the blob store + // in a bwc compatible way. + maybeRecalculateMetadataHash(fileInfo, recoveryTargetMetadata); + } catch (Exception e) { + // if the index is broken we might not be able to read it + logger.warn(() -> new ParameterizedMessage("{} Can't calculate hash from blog for file [{}] [{}]", shardId, fileInfo.physicalName(), fileInfo.metadata()), e); + } + snapshotMetaData.put(fileInfo.metadata().name(), fileInfo.metadata()); + fileInfos.put(fileInfo.metadata().name(), fileInfo); + } + + final Store.MetadataSnapshot sourceMetaData = new Store.MetadataSnapshot(unmodifiableMap(snapshotMetaData), emptyMap(), 0); + + final StoreFileMetaData restoredSegmentsFile = sourceMetaData.getSegmentsFile(); + if (restoredSegmentsFile == null) { + throw new IndexShardRestoreFailedException(shardId, "Snapshot has no segments file"); + } + + final Store.RecoveryDiff diff = sourceMetaData.recoveryDiff(recoveryTargetMetadata); + for (StoreFileMetaData md : diff.identical) { + BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfos.get(md.name()); + recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), true); + if (logger.isTraceEnabled()) { + logger.trace("[{}] [{}] not_recovering [{}] from [{}], exists in local store and is same", shardId, snapshotId, fileInfo.physicalName(), fileInfo.name()); + } + } + + for (StoreFileMetaData md : Iterables.concat(diff.different, diff.missing)) { + BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfos.get(md.name()); + filesToRecover.add(fileInfo); + recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), false); + if (logger.isTraceEnabled()) { + logger.trace("[{}] [{}] recovering [{}] from [{}], exists in local store but is different", shardId, snapshotId, + fileInfo.physicalName(), fileInfo.name()); + } + } + + if (filesToRecover.isEmpty()) { + logger.trace("no files to recover, all exists within the local store"); + } + + try { + // list of all existing store files + final List deleteIfExistFiles = Arrays.asList(store.directory().listAll()); + + // restore the files from the snapshot to the Lucene store + for (final BlobStoreIndexShardSnapshot.FileInfo fileToRecover : filesToRecover) { + // if a file with a same physical name already exist in the store we need to delete it + // before restoring it from the snapshot. We could be lenient and try to reuse the existing + // store files (and compare their names/length/checksum again with the snapshot files) but to + // avoid extra complexity we simply delete them and restore them again like StoreRecovery + // does with dangling indices. Any existing store file that is not restored from the snapshot + // will be clean up by RecoveryTarget.cleanFiles(). + final String physicalName = fileToRecover.physicalName(); + if (deleteIfExistFiles.contains(physicalName)) { + logger.trace("[{}] [{}] deleting pre-existing file [{}]", shardId, snapshotId, physicalName); + store.directory().deleteFile(physicalName); + } + + logger.trace("[{}] [{}] restoring file [{}]", shardId, snapshotId, fileToRecover.name()); + restoreFile(fileToRecover, store); + } + } catch (IOException ex) { + throw new IndexShardRestoreFailedException(shardId, "Failed to recover index", ex); + } + + // read the snapshot data persisted + final SegmentInfos segmentCommitInfos; + try { + segmentCommitInfos = Lucene.pruneUnreferencedFiles(restoredSegmentsFile.name(), store.directory()); + } catch (IOException e) { + throw new IndexShardRestoreFailedException(shardId, "Failed to fetch index version after copying it over", e); + } + recoveryState.getIndex().updateVersion(segmentCommitInfos.getVersion()); + + /// now, go over and clean files that are in the store, but were not in the snapshot + try { + for (String storeFile : store.directory().listAll()) { + if (Store.isAutogenerated(storeFile) || snapshotFiles.containPhysicalIndexFile(storeFile)) { + continue; //skip write.lock, checksum files and files that exist in the snapshot + } + try { + store.deleteQuiet("restore", storeFile); + store.directory().deleteFile(storeFile); + } catch (IOException e) { + logger.warn("[{}] failed to delete file [{}] during snapshot cleanup", snapshotId, storeFile); + } + } + } catch (IOException e) { + logger.warn("[{}] failed to list directory - some of files might not be deleted", snapshotId); + } + } finally { + store.decRef(); + } + } + + /** + * Restores a file + * This is asynchronous method. Upon completion of the operation latch is getting counted down and any failures are + * added to the {@code failures} list + * + * @param fileInfo file to be restored + */ + private void restoreFile(final BlobStoreIndexShardSnapshot.FileInfo fileInfo, final Store store) throws IOException { + boolean success = false; + + try (InputStream stream = fileInputStream(fileInfo)) { + try (IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), fileInfo.metadata(), IOContext.DEFAULT)) { + final byte[] buffer = new byte[bufferSize]; + int length; + while ((length = stream.read(buffer)) > 0) { + indexOutput.writeBytes(buffer, 0, length); + recoveryState.getIndex().addRecoveredBytesToFile(fileInfo.name(), length); + } + Store.verify(indexOutput); + indexOutput.close(); + store.directory().sync(Collections.singleton(fileInfo.physicalName())); + success = true; + } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { + try { + store.markStoreCorrupted(ex); + } catch (IOException e) { + logger.warn("store cannot be marked as corrupted", e); + } + throw ex; + } finally { + if (success == false) { + store.deleteQuiet(fileInfo.physicalName()); + } + } + } + } + + abstract InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo); + + /** + * This is a BWC layer to ensure we update the snapshots metadata with the corresponding hashes before we compare them. + * The new logic for StoreFileMetaData reads the entire {@code .si} and {@code segments.n} files to strengthen the + * comparison of the files on a per-segment / per-commit level. + */ + private void maybeRecalculateMetadataHash(final BlobStoreIndexShardSnapshot.FileInfo fileInfo, Store.MetadataSnapshot snapshot) + throws IOException { + final StoreFileMetaData metadata; + if (fileInfo != null && (metadata = snapshot.get(fileInfo.physicalName())) != null) { + if (metadata.hash().length > 0 && fileInfo.metadata().hash().length == 0) { + // we have a hash - check if our repo has a hash too otherwise we have + // to calculate it. + // we might have multiple parts even though the file is small... make sure we read all of it. + try (InputStream stream = fileInputStream(fileInfo)) { + BytesRefBuilder builder = new BytesRefBuilder(); + Store.MetadataSnapshot.hashFile(builder, stream, fileInfo.length()); + BytesRef hash = fileInfo.metadata().hash(); // reset the file infos metadata hash + assert hash.length == 0; + hash.bytes = builder.bytes(); + hash.offset = 0; + hash.length = builder.length(); + } + } + } + } +} 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 09eb557fe9cad..866486ce8c521 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -22,15 +22,9 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.IndexFormatTooNewException; -import org.apache.lucene.index.IndexFormatTooOldException; -import org.apache.lucene.index.IndexNotFoundException; -import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.RateLimiter; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; @@ -65,7 +59,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -111,17 +104,12 @@ import java.nio.file.FileAlreadyExistsException; import java.nio.file.NoSuchFileException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import static java.util.Collections.emptyMap; -import static java.util.Collections.unmodifiableMap; import static org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName; /** @@ -862,9 +850,12 @@ public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, @Override public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { - final RestoreContext snapshotContext = new RestoreContext(shard, snapshotId, indexId, snapshotShardId, recoveryState); + final Context context = new Context(snapshotId, indexId, shard.shardId(), snapshotShardId); + final RestoreContext snapshotContext = new RestoreContext(shard, snapshotId, recoveryState); try { - snapshotContext.restore(); + BlobStoreIndexShardSnapshot snapshot = context.loadSnapshot(); + SnapshotFiles snapshotFiles = new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles()); + snapshotContext.restore(snapshotFiles); } catch (Exception e) { throw new IndexShardRestoreFailedException(shard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); } @@ -1451,212 +1442,26 @@ protected InputStream openSlice(long slice) throws IOException { /** * Context for restore operations */ - private class RestoreContext extends Context { - - private final IndexShard targetShard; - - private final RecoveryState recoveryState; + private class RestoreContext extends BlobRestoreContext { /** * Constructs new restore context * - * @param shard shard to restore into + * @param indexShard shard to restore into * @param snapshotId snapshot id - * @param indexId id of the index being restored - * @param snapshotShardId shard in the snapshot that data should be restored from * @param recoveryState recovery state to report progress */ - RestoreContext(IndexShard shard, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { - super(snapshotId, indexId, shard.shardId(), snapshotShardId); - this.recoveryState = recoveryState; - this.targetShard = shard; - } - - /** - * Performs restore operation - */ - public void restore() throws IOException { - final Store store = targetShard.store(); - store.incRef(); - try { - logger.debug("[{}] [{}] restoring to [{}] ...", snapshotId, metadata.name(), shardId); - BlobStoreIndexShardSnapshot snapshot = loadSnapshot(); - - if (snapshot.indexFiles().size() == 1 - && snapshot.indexFiles().get(0).physicalName().startsWith("segments_") - && snapshot.indexFiles().get(0).hasUnknownChecksum()) { - // If the shard has no documents, it will only contain a single segments_N file for the - // shard's snapshot. If we are restoring a snapshot created by a previous supported version, - // it is still possible that in that version, an empty shard has a segments_N file with an unsupported - // version (and no checksum), because we don't know the Lucene version to assign segments_N until we - // have written some data. Since the segments_N for an empty shard could have an incompatible Lucene - // version number and no checksum, even though the index itself is perfectly fine to restore, this - // empty shard would cause exceptions to be thrown. Since there is no data to restore from an empty - // shard anyway, we just create the empty shard here and then exit. - store.createEmpty(targetShard.indexSettings().getIndexMetaData().getCreationVersion().luceneVersion); - return; - } - - SnapshotFiles snapshotFiles = new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles()); - Store.MetadataSnapshot recoveryTargetMetadata; - try { - // this will throw an IOException if the store has no segments infos file. The - // store can still have existing files but they will be deleted just before being - // restored. - recoveryTargetMetadata = targetShard.snapshotStoreMetadata(); - } catch (IndexNotFoundException e) { - // happens when restore to an empty shard, not a big deal - logger.trace("[{}] [{}] restoring from to an empty shard", shardId, snapshotId); - recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; - } catch (IOException e) { - logger.warn(() -> new ParameterizedMessage("{} Can't read metadata from store, will not reuse any local file while restoring", shardId), e); - recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; - } - - final List filesToRecover = new ArrayList<>(); - final Map snapshotMetaData = new HashMap<>(); - final Map fileInfos = new HashMap<>(); - for (final BlobStoreIndexShardSnapshot.FileInfo fileInfo : snapshot.indexFiles()) { - try { - // in 1.3.3 we added additional hashes for .si / segments_N files - // to ensure we don't double the space in the repo since old snapshots - // don't have this hash we try to read that hash from the blob store - // in a bwc compatible way. - maybeRecalculateMetadataHash(blobContainer, fileInfo, recoveryTargetMetadata); - } catch (Exception e) { - // if the index is broken we might not be able to read it - logger.warn(() -> new ParameterizedMessage("{} Can't calculate hash from blog for file [{}] [{}]", shardId, fileInfo.physicalName(), fileInfo.metadata()), e); - } - snapshotMetaData.put(fileInfo.metadata().name(), fileInfo.metadata()); - fileInfos.put(fileInfo.metadata().name(), fileInfo); - } - - final Store.MetadataSnapshot sourceMetaData = new Store.MetadataSnapshot(unmodifiableMap(snapshotMetaData), emptyMap(), 0); - - final StoreFileMetaData restoredSegmentsFile = sourceMetaData.getSegmentsFile(); - if (restoredSegmentsFile == null) { - throw new IndexShardRestoreFailedException(shardId, "Snapshot has no segments file"); - } - - final Store.RecoveryDiff diff = sourceMetaData.recoveryDiff(recoveryTargetMetadata); - for (StoreFileMetaData md : diff.identical) { - BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfos.get(md.name()); - recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), true); - if (logger.isTraceEnabled()) { - logger.trace("[{}] [{}] not_recovering [{}] from [{}], exists in local store and is same", shardId, snapshotId, fileInfo.physicalName(), fileInfo.name()); - } - } - - for (StoreFileMetaData md : Iterables.concat(diff.different, diff.missing)) { - BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfos.get(md.name()); - filesToRecover.add(fileInfo); - recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), false); - if (logger.isTraceEnabled()) { - logger.trace("[{}] [{}] recovering [{}] from [{}], exists in local store but is different", shardId, snapshotId, - fileInfo.physicalName(), fileInfo.name()); - } - } - - if (filesToRecover.isEmpty()) { - logger.trace("no files to recover, all exists within the local store"); - } - - try { - // list of all existing store files - final List deleteIfExistFiles = Arrays.asList(store.directory().listAll()); - - // restore the files from the snapshot to the Lucene store - for (final BlobStoreIndexShardSnapshot.FileInfo fileToRecover : filesToRecover) { - // if a file with a same physical name already exist in the store we need to delete it - // before restoring it from the snapshot. We could be lenient and try to reuse the existing - // store files (and compare their names/length/checksum again with the snapshot files) but to - // avoid extra complexity we simply delete them and restore them again like StoreRecovery - // does with dangling indices. Any existing store file that is not restored from the snapshot - // will be clean up by RecoveryTarget.cleanFiles(). - final String physicalName = fileToRecover.physicalName(); - if (deleteIfExistFiles.contains(physicalName)) { - logger.trace("[{}] [{}] deleting pre-existing file [{}]", shardId, snapshotId, physicalName); - store.directory().deleteFile(physicalName); - } - - logger.trace("[{}] [{}] restoring file [{}]", shardId, snapshotId, fileToRecover.name()); - restoreFile(fileToRecover, store); - } - } catch (IOException ex) { - throw new IndexShardRestoreFailedException(shardId, "Failed to recover index", ex); - } - - // read the snapshot data persisted - final SegmentInfos segmentCommitInfos; - try { - segmentCommitInfos = Lucene.pruneUnreferencedFiles(restoredSegmentsFile.name(), store.directory()); - } catch (IOException e) { - throw new IndexShardRestoreFailedException(shardId, "Failed to fetch index version after copying it over", e); - } - recoveryState.getIndex().updateVersion(segmentCommitInfos.getVersion()); - - /// now, go over and clean files that are in the store, but were not in the snapshot - try { - for (String storeFile : store.directory().listAll()) { - if (Store.isAutogenerated(storeFile) || snapshotFiles.containPhysicalIndexFile(storeFile)) { - continue; //skip write.lock, checksum files and files that exist in the snapshot - } - try { - store.deleteQuiet("restore", storeFile); - store.directory().deleteFile(storeFile); - } catch (IOException e) { - logger.warn("[{}] failed to delete file [{}] during snapshot cleanup", snapshotId, storeFile); - } - } - } catch (IOException e) { - logger.warn("[{}] failed to list directory - some of files might not be deleted", snapshotId); - } - } finally { - store.decRef(); - } + RestoreContext(IndexShard indexShard, SnapshotId snapshotId, RecoveryState recoveryState) { + super(metadata.name(), indexShard, snapshotId, recoveryState, BUFFER_SIZE); } - /** - * Restores a file - * This is asynchronous method. Upon completion of the operation latch is getting counted down and any failures are - * added to the {@code failures} list - * - * @param fileInfo file to be restored - */ - private void restoreFile(final BlobStoreIndexShardSnapshot.FileInfo fileInfo, final Store store) throws IOException { - boolean success = false; - - try (InputStream partSliceStream = new PartSliceStream(blobContainer, fileInfo)) { - final InputStream stream; - if (restoreRateLimiter == null) { - stream = partSliceStream; - } else { - stream = new RateLimitingInputStream(partSliceStream, restoreRateLimiter, restoreRateLimitingTimeInNanos::inc); - } - - try (IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), fileInfo.metadata(), IOContext.DEFAULT)) { - final byte[] buffer = new byte[BUFFER_SIZE]; - int length; - while ((length = stream.read(buffer)) > 0) { - indexOutput.writeBytes(buffer, 0, length); - recoveryState.getIndex().addRecoveredBytesToFile(fileInfo.name(), length); - } - Store.verify(indexOutput); - indexOutput.close(); - store.directory().sync(Collections.singleton(fileInfo.physicalName())); - success = true; - } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { - try { - store.markStoreCorrupted(ex); - } catch (IOException e) { - logger.warn("store cannot be marked as corrupted", e); - } - throw ex; - } finally { - if (success == false) { - store.deleteQuiet(fileInfo.physicalName()); - } - } + @Override + InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo) { + if (restoreRateLimiter == null) { + return new PartSliceStream(blobContainer.get(), fileInfo); + } else { + RateLimitingInputStream.Listener listener = restoreRateLimitingTimeInNanos::inc; + return new RateLimitingInputStream(new PartSliceStream(blobContainer.get(), fileInfo), restoreRateLimiter, listener); } } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 9e3f5afaff2a5..de28749d42fb2 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -45,6 +45,8 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.snapshots.IndexShardRestoreFailedException; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; +import org.elasticsearch.index.snapshots.blobstore.SnapshotFiles; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.indices.recovery.RecoveryState; @@ -267,7 +269,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); // TODO: There should be some local timeout. And if the remote cluster returns an unknown session // response, we should be able to retry by creating a new session. - try (RestoreSession restoreSession = RestoreSession.openSession(remoteClient, leaderShardId, indexShard, shardId, recoveryState)) { + try (RestoreSession restoreSession = RestoreSession.openSession(remoteClient, leaderShardId, indexShard, recoveryState)) { restoreSession.restoreFiles(); } @@ -293,6 +295,10 @@ private void maybeUpdateMappings(Client localClient, Client remoteClient, Index } } + private static SnapshotFiles convert(Store.MetadataSnapshot sourceMetaData) { + return new SnapshotFiles(LATEST, null); + } + private static class RestoreSession implements Closeable { private final int BUFFER_SIZE = 1 << 16; @@ -306,27 +312,25 @@ private static class RestoreSession implements Closeable { private final RecoveryState recoveryState; private final Store.MetadataSnapshot sourceMetaData; - RestoreSession(Client remoteClient, String sessionUUID, DiscoveryNode node, IndexShard indexShard, ShardId shardId, - RecoveryState recoveryState, Store.MetadataSnapshot sourceMetaData) { + RestoreSession(Client remoteClient, String sessionUUID, DiscoveryNode node, IndexShard indexShard, RecoveryState recoveryState, + Store.MetadataSnapshot sourceMetaData) { this.remoteClient = remoteClient; this.sessionUUID = sessionUUID; this.node = node; this.indexShard = indexShard; - this.shardId = shardId; + this.shardId = indexShard.shardId(); this.store = indexShard.store(); this.store.incRef(); this.recoveryState = recoveryState; this.sourceMetaData = sourceMetaData; } - static RestoreSession openSession(Client remoteClient, ShardId leaderShardId, IndexShard indexShard, ShardId shardId, - RecoveryState recoveryState) { + static RestoreSession openSession(Client remoteClient, ShardId leaderShardId, IndexShard indexShard, RecoveryState recoveryState) { String sessionUUID = UUIDs.randomBase64UUID(); PutCcrRestoreSessionAction.PutCcrRestoreSessionResponse response = remoteClient.execute(PutCcrRestoreSessionAction.INSTANCE, new PutCcrRestoreSessionRequest(sessionUUID, leaderShardId)).actionGet(); Store.MetadataSnapshot sourceFileMetaData = response.getStoreFileMetaData(); - return new RestoreSession(remoteClient, sessionUUID, response.getNode(), indexShard, shardId, recoveryState, - sourceFileMetaData); + return new RestoreSession(remoteClient, sessionUUID, response.getNode(), indexShard, recoveryState, sourceFileMetaData); } void restoreFiles() { From 79a53e0fecb7c01f5f20ae90b834539f7cb65e9b Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 4 Jan 2019 19:21:02 -0700 Subject: [PATCH 12/30] Use in ccr --- .../blobstore/BlobRestoreContext.java | 16 +- .../blobstore/BlobStoreRepository.java | 2 +- .../xpack/ccr/repository/CcrRepository.java | 270 ++++++++++-------- 3 files changed, 158 insertions(+), 130 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobRestoreContext.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobRestoreContext.java index b0f75ef8becfe..dc1a837e21923 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobRestoreContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobRestoreContext.java @@ -73,7 +73,8 @@ public abstract class BlobRestoreContext { * @param recoveryState recovery state to report progress * @param bufferSize buffer size for restore */ - BlobRestoreContext(String repositoryName, IndexShard indexShard, SnapshotId snapshotId, RecoveryState recoveryState, int bufferSize) { + protected BlobRestoreContext(String repositoryName, IndexShard indexShard, SnapshotId snapshotId, RecoveryState recoveryState, + int bufferSize) { this.repositoryName = repositoryName; this.recoveryState = recoveryState; this.indexShard = indexShard; @@ -155,7 +156,7 @@ public void restore(SnapshotFiles snapshotFiles) throws IOException { } } - for (StoreFileMetaData md : Iterables.concat(diff.different, diff.missing)) { + for (StoreFileMetaData md : concat(diff)) { BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfos.get(md.name()); filesToRecover.add(fileInfo); recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), false); @@ -224,10 +225,15 @@ public void restore(SnapshotFiles snapshotFiles) throws IOException { } } + protected abstract InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo); + + @SuppressWarnings("unchecked") + private Iterable concat(Store.RecoveryDiff diff) { + return Iterables.concat(diff.different, diff.missing); + } + /** * Restores a file - * This is asynchronous method. Upon completion of the operation latch is getting counted down and any failures are - * added to the {@code failures} list * * @param fileInfo file to be restored */ @@ -261,8 +267,6 @@ private void restoreFile(final BlobStoreIndexShardSnapshot.FileInfo fileInfo, fi } } - abstract InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo); - /** * This is a BWC layer to ensure we update the snapshots metadata with the corresponding hashes before we compare them. * The new logic for StoreFileMetaData reads the entire {@code .si} and {@code segments.n} files to strengthen the 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 866486ce8c521..20f72cfef7110 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1456,7 +1456,7 @@ private class RestoreContext extends BlobRestoreContext { } @Override - InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo) { + protected InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo) { if (restoreRateLimiter == null) { return new PartSliceStream(blobContainer.get(), fileInfo); } else { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index de28749d42fb2..0e06820721383 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -8,12 +8,10 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; -import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.BytesRef; @@ -34,8 +32,8 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; @@ -53,6 +51,7 @@ import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.repositories.blobstore.BlobRestoreContext; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotShardFailure; @@ -69,8 +68,8 @@ import java.io.Closeable; import java.io.IOException; +import java.io.InputStream; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -269,8 +268,11 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); // TODO: There should be some local timeout. And if the remote cluster returns an unknown session // response, we should be able to retry by creating a new session. - try (RestoreSession restoreSession = RestoreSession.openSession(remoteClient, leaderShardId, indexShard, recoveryState)) { + String name = metadata.name(); + try (RestoreSession restoreSession = RestoreSession.openSession(name, remoteClient, leaderShardId, indexShard, recoveryState)) { restoreSession.restoreFiles(); + } catch (IOException e) { + throw new IndexShardRestoreFailedException(indexShard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); } maybeUpdateMappings(client, remoteClient, leaderIndex, indexShard.indexSettings()); @@ -295,110 +297,119 @@ private void maybeUpdateMappings(Client localClient, Client remoteClient, Index } } - private static SnapshotFiles convert(Store.MetadataSnapshot sourceMetaData) { - return new SnapshotFiles(LATEST, null); - } - - private static class RestoreSession implements Closeable { + private static class RestoreSession extends BlobRestoreContext implements Closeable { - private final int BUFFER_SIZE = 1 << 16; + private static final int BUFFER_SIZE = 1 << 16; private final Client remoteClient; private final String sessionUUID; private final DiscoveryNode node; - private final IndexShard indexShard; private final ShardId shardId; private final Store store; - private final RecoveryState recoveryState; private final Store.MetadataSnapshot sourceMetaData; - RestoreSession(Client remoteClient, String sessionUUID, DiscoveryNode node, IndexShard indexShard, RecoveryState recoveryState, - Store.MetadataSnapshot sourceMetaData) { + RestoreSession(String repositoryName, Client remoteClient, String sessionUUID, DiscoveryNode node, IndexShard indexShard, + RecoveryState recoveryState, Store.MetadataSnapshot sourceMetaData) { + super(repositoryName, indexShard, SNAPSHOT_ID, recoveryState, BUFFER_SIZE); this.remoteClient = remoteClient; this.sessionUUID = sessionUUID; this.node = node; - this.indexShard = indexShard; this.shardId = indexShard.shardId(); this.store = indexShard.store(); this.store.incRef(); - this.recoveryState = recoveryState; this.sourceMetaData = sourceMetaData; } - static RestoreSession openSession(Client remoteClient, ShardId leaderShardId, IndexShard indexShard, RecoveryState recoveryState) { + static RestoreSession openSession(String repositoryName, Client remoteClient, ShardId leaderShardId, IndexShard indexShard, + RecoveryState recoveryState) { String sessionUUID = UUIDs.randomBase64UUID(); PutCcrRestoreSessionAction.PutCcrRestoreSessionResponse response = remoteClient.execute(PutCcrRestoreSessionAction.INSTANCE, new PutCcrRestoreSessionRequest(sessionUUID, leaderShardId)).actionGet(); Store.MetadataSnapshot sourceFileMetaData = response.getStoreFileMetaData(); - return new RestoreSession(remoteClient, sessionUUID, response.getNode(), indexShard, recoveryState, sourceFileMetaData); + return new RestoreSession(repositoryName, remoteClient, sessionUUID, response.getNode(), indexShard, recoveryState, sourceFileMetaData); } - void restoreFiles() { - Store.MetadataSnapshot recoveryMetadata; - try { - recoveryMetadata = indexShard.snapshotStoreMetadata(); - } catch (IOException e) { - throw new IndexShardRecoveryException(shardId, "failed access store metadata", e); - } - - final StoreFileMetaData restoredSegmentsFile = sourceMetaData.getSegmentsFile(); - if (restoredSegmentsFile == null) { - throw new IndexShardRestoreFailedException(shardId, "Ccr leader index has no segments file"); - } - - final Store.RecoveryDiff diff = sourceMetaData.recoveryDiff(recoveryMetadata); - for (StoreFileMetaData fileMetaData : diff.identical) { - recoveryState.getIndex().addFileDetail(fileMetaData.name(), fileMetaData.length(), true); - logger.trace("shard [{}] not recovering file [{}], exists in local store and is identical", shardId, fileMetaData.name()); - } - - List filesToRecover = new ArrayList<>(); - for (StoreFileMetaData fileMetaData : concat(diff)) { - filesToRecover.add(fileMetaData); - recoveryState.getIndex().addFileDetail(fileMetaData.name(), fileMetaData.length(), false); - logger.trace("shard [{}] recovering file [{}]", shardId, fileMetaData.name()); - } - - if (filesToRecover.isEmpty()) { - logger.trace("shard [{}] no files to recover, all exist within the local store", shardId); - } - - final List deleteIfExistFiles = Arrays.asList(getDirectoryFiles()); - for (final StoreFileMetaData fileToRecover : filesToRecover) { - // if a file with a same physical name already exist in the store we need to delete it - // before restoring it from the snapshot. We could be lenient and try to reuse the existing - // store files (and compare their names/length/checksum again with the snapshot files) but to - // avoid extra complexity we simply delete them and restore them again like StoreRecovery - // does with dangling indices. Any existing store file that is not restored from the snapshot - // will be clean up by RecoveryTarget.cleanFiles(). - final String name = fileToRecover.name(); - if (deleteIfExistFiles.contains(name)) { - logger.trace("shard [{}] deleting pre-existing file [{}]", shardId, name); - deleteFile(name); - } - - logger.trace("[{}] restoring file [{}]", shardId, name); - try { - restoreFile(fileToRecover, store); - } catch (Exception e) { - logger.info(() -> new ParameterizedMessage("shard [{}] failed to restore file [{}]", shardId, name), e); - throw new IndexShardRecoveryException(shardId, "failed to restore file", e); - } + void restoreFiles() throws IOException { + ArrayList fileInfos = new ArrayList<>(); + for (StoreFileMetaData fileMetaData : sourceMetaData) { + ByteSizeValue fileSize = new ByteSizeValue(fileMetaData.length()); + fileInfos.add(new BlobStoreIndexShardSnapshot.FileInfo(fileMetaData.name(), fileMetaData, fileSize)); } + SnapshotFiles snapshotFiles = new SnapshotFiles(LATEST, fileInfos); + restore(snapshotFiles); + } - final SegmentInfos segmentCommitInfos; - try { - segmentCommitInfos = Lucene.pruneUnreferencedFiles(restoredSegmentsFile.name(), store.directory()); - } catch (IOException e) { - throw new IndexShardRestoreFailedException(shardId, "Failed to fetch index version after copying it over", e); - } - recoveryState.getIndex().updateVersion(segmentCommitInfos.getVersion()); +// void restoreFiles() { +// Store.MetadataSnapshot recoveryMetadata; +// try { +// recoveryMetadata = indexShard.snapshotStoreMetadata(); +// } catch (IOException e) { +// throw new IndexShardRecoveryException(shardId, "failed access store metadata", e); +// } +// +// final StoreFileMetaData restoredSegmentsFile = sourceMetaData.getSegmentsFile(); +// if (restoredSegmentsFile == null) { +// throw new IndexShardRestoreFailedException(shardId, "Ccr leader index has no segments file"); +// } +// +// final Store.RecoveryDiff diff = sourceMetaData.recoveryDiff(recoveryMetadata); +// for (StoreFileMetaData fileMetaData : diff.identical) { +// recoveryState.getIndex().addFileDetail(fileMetaData.name(), fileMetaData.length(), true); +// logger.trace("shard [{}] not recovering file [{}], exists in local store and is identical", shardId, fileMetaData.name()); +// } +// +// List filesToRecover = new ArrayList<>(); +// for (StoreFileMetaData fileMetaData : concat(diff)) { +// filesToRecover.add(fileMetaData); +// recoveryState.getIndex().addFileDetail(fileMetaData.name(), fileMetaData.length(), false); +// logger.trace("shard [{}] recovering file [{}]", shardId, fileMetaData.name()); +// } +// +// if (filesToRecover.isEmpty()) { +// logger.trace("shard [{}] no files to recover, all exist within the local store", shardId); +// } +// +// final List deleteIfExistFiles = Arrays.asList(getDirectoryFiles()); +// for (final StoreFileMetaData fileToRecover : filesToRecover) { +// // if a file with a same physical name already exist in the store we need to delete it +// // before restoring it from the snapshot. We could be lenient and try to reuse the existing +// // store files (and compare their names/length/checksum again with the snapshot files) but to +// // avoid extra complexity we simply delete them and restore them again like StoreRecovery +// // does with dangling indices. Any existing store file that is not restored from the snapshot +// // will be clean up by RecoveryTarget.cleanFiles(). +// final String name = fileToRecover.name(); +// if (deleteIfExistFiles.contains(name)) { +// logger.trace("shard [{}] deleting pre-existing file [{}]", shardId, name); +// deleteFile(name); +// } +// +// logger.trace("[{}] restoring file [{}]", shardId, name); +// try { +// restoreFile(fileToRecover, store); +// } catch (Exception e) { +// logger.info(() -> new ParameterizedMessage("shard [{}] failed to restore file [{}]", shardId, name), e); +// throw new IndexShardRecoveryException(shardId, "failed to restore file", e); +// } +// } +// +// final SegmentInfos segmentCommitInfos; +// try { +// segmentCommitInfos = Lucene.pruneUnreferencedFiles(restoredSegmentsFile.name(), store.directory()); +// } catch (IOException e) { +// throw new IndexShardRestoreFailedException(shardId, "Failed to fetch index version after copying it over", e); +// } +// recoveryState.getIndex().updateVersion(segmentCommitInfos.getVersion()); +// +// try { +// store.cleanupAndVerify("restore complete from remote", sourceMetaData); +// } catch (IOException e) { +// throw new IndexShardRecoveryException(shardId, "failed to cleanup and verify the store", e); +// } +// } - try { - store.cleanupAndVerify("restore complete from remote", sourceMetaData); - } catch (IOException e) { - throw new IndexShardRecoveryException(shardId, "failed to cleanup and verify the store", e); - } + @Override + protected InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo) { + return new RestoreFileInputStream(remoteClient, sessionUUID, node, fileInfo.metadata()); } @SuppressWarnings("unchecked") @@ -422,47 +433,6 @@ private void deleteFile(String name) { } } - private void restoreFile(StoreFileMetaData fileToRecover, Store store) throws IOException { - boolean success = false; - long pos = 0; - - try (IndexOutput indexOutput = store.createVerifyingOutput(fileToRecover.name(), fileToRecover, IOContext.DEFAULT)) { - - GetCcrRestoreFileChunkRequest request = new GetCcrRestoreFileChunkRequest(node, sessionUUID, - fileToRecover.name(), pos, (int) Math.min(fileToRecover.length() - pos, BUFFER_SIZE)); - BytesReference fileChunk = remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request).actionGet().getChunk(); - - BytesRefIterator iterator = fileChunk.iterator(); - BytesRef ref; - int bytesWritten = 0; - while ((ref = iterator.next()) != null) { - byte[] refBytes = ref.bytes; - indexOutput.writeBytes(refBytes, 0, ref.length); - recoveryState.getIndex().addRecoveredBytesToFile(fileToRecover.name(), ref.length); - bytesWritten += ref.length; - } - - assert bytesWritten == fileChunk.length() : "Did not write [" + bytesWritten + "] the expected [" + fileChunk.length() - + "] number of bytes"; - - Store.verify(indexOutput); - indexOutput.close(); - store.directory().sync(Collections.singleton(fileToRecover.name())); - success = true; - } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { - try { - store.markStoreCorrupted(ex); - } catch (IOException e) { - logger.warn("store cannot be marked as corrupted", e); - } - throw ex; - } finally { - if (success == false) { - store.deleteQuiet(fileToRecover.name()); - } - } - } - @Override public void close() { this.store.decRef(); @@ -471,4 +441,58 @@ public void close() { remoteClient.execute(ClearCcrRestoreSessionAction.INSTANCE, clearRequest).actionGet(); } } + + private static class RestoreFileInputStream extends InputStream { + + private final Client remoteClient; + private final String sessionUUID; + private final DiscoveryNode node; + private final StoreFileMetaData fileToRecover; + + private long pos = 0; + + private RestoreFileInputStream(Client remoteClient, String sessionUUID, DiscoveryNode node, StoreFileMetaData fileToRecover) { + this.remoteClient = remoteClient; + this.sessionUUID = sessionUUID; + this.node = node; + this.fileToRecover = fileToRecover; + } + + + @Override + public int read() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int read(byte[] bytes, int off, int len) throws IOException { + long remainingBytes = fileToRecover.length() - pos; + if (remainingBytes <= 0) { + return 0; + } + + int bytesRequested = (int) Math.min(remainingBytes, len); + String fileName = fileToRecover.name(); + GetCcrRestoreFileChunkRequest request = new GetCcrRestoreFileChunkRequest(node, sessionUUID, fileName, pos, bytesRequested); + BytesReference fileChunk = remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request).actionGet().getChunk(); + + int bytesReceived = fileChunk.length(); + if (bytesReceived > bytesRequested) { + throw new IOException("More bytes [" + bytesReceived + "] received than requested [" + bytesRequested + "]"); + } + + BytesRefIterator iterator = fileChunk.iterator(); + BytesRef ref; + int bytesWritten = 0; + while ((ref = iterator.next()) != null) { + byte[] refBytes = ref.bytes; + System.arraycopy(refBytes, 0, bytes, off + bytesWritten, refBytes.length); + bytesWritten += ref.length; + } + + pos += bytesReceived; + + return bytesReceived; + } + } } From 142800d233c30c8d4422fb90c9f692fcfe522e20 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Sat, 5 Jan 2019 19:52:34 -0700 Subject: [PATCH 13/30] Add file reader --- .../GetCcrRestoreFileChunkAction.java | 12 +---- .../repository/CcrRestoreSourceService.java | 45 +++++++++++++------ 2 files changed, 34 insertions(+), 23 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index 4841142fd2234..7a61762c7efe1 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -6,8 +6,6 @@ package org.elasticsearch.xpack.ccr.action.repositories; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexInput; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionResponse; @@ -19,9 +17,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.index.engine.Engine; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportActionProxy; @@ -76,13 +72,9 @@ public void onFailure(Exception e) { @Override protected void doRun() throws Exception { - CcrRestoreSourceService.Session session = restoreSourceService.getSession(request.getSessionUUID()); - Engine.IndexCommitRef refCommit = session.refCommit(); - try (IndexInput in = refCommit.getIndexCommit().getDirectory().openInput(request.getFileName(), IOContext.READONCE); - Releasable releasable = session::decRef) { + try (CcrRestoreSourceService.FileReader fileReader = restoreSourceService.getSession(request.getSessionUUID())) { byte[] chunk = new byte[request.getSize()]; - in.seek(request.getOffset()); - in.readBytes(chunk, 0, request.getSize()); + fileReader.readFileBytes(request.getFileName(), chunk, request.getOffset(), request.getSize()); listener.onResponse(new GetCcrRestoreFileChunkResponse(new BytesArray(chunk))); } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index 92d19c084cb9d..628af67fd2788 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -8,6 +8,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Settings; @@ -134,33 +136,31 @@ public synchronized void closeSession(String sessionUUID) { IOUtils.closeWhileHandlingException(restore); } - public synchronized Session getSession(String sessionUUID) { + public synchronized FileReader getSession(String sessionUUID) { RestoreContext restore = onGoingRestores.get(sessionUUID); if (restore == null) { logger.info("could not get session [{}] because session not found", sessionUUID); throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); } - Session session = restore.session; - session.incRef(); - return session; + return new FileReader(restore.session); } private class RestoreContext implements Closeable { private final String sessionUUID; private final IndexShard indexShard; - private final Session session; + private final RefCountedCommit session; private RestoreContext(String sessionUUID, IndexShard indexShard, Engine.IndexCommitRef commitRef) { this.sessionUUID = sessionUUID; this.indexShard = indexShard; - this.session = new Session(commitRef); + this.session = new RefCountedCommit(commitRef); } Store.MetadataSnapshot getMetaData() throws IOException { indexShard.store().incRef(); try { - return indexShard.store().getMetadata(session.refCommit().getIndexCommit()); + return indexShard.store().getMetadata(session.commitRef.getIndexCommit()); } finally { indexShard.store().decRef(); } @@ -185,23 +185,42 @@ private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { } } - public static class Session extends AbstractRefCounted { + private static class RefCountedCommit extends AbstractRefCounted { private static final String NAME = "ref-counted-session"; private final Engine.IndexCommitRef commitRef; - private Session(Engine.IndexCommitRef commitRef) { + private RefCountedCommit(Engine.IndexCommitRef commitRef) { super(NAME); this.commitRef = commitRef; } - public Engine.IndexCommitRef refCommit() { - return commitRef; - } - @Override protected void closeInternal() { IOUtils.closeWhileHandlingException(commitRef); } } + + public static class FileReader implements Closeable { + + private final RefCountedCommit session; + + private FileReader(RefCountedCommit session) { + this.session = session; + session.incRef(); + } + + public void readFileBytes(String fileName, byte[] chunk, long offset, int length) throws IOException { + Engine.IndexCommitRef commitRef = session.commitRef; + try (IndexInput in = commitRef.getIndexCommit().getDirectory().openInput(fileName, IOContext.READONCE)) { + in.seek(offset); + in.readBytes(chunk, 0, length); + } + } + + @Override + public void close() { + session.decRef(); + } + } } From 478e25c82f9a28177380bfe45cac804296631181 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 7 Jan 2019 14:54:37 -0700 Subject: [PATCH 14/30] WIP --- .../blobstore/BlobStoreRepository.java | 2 +- ...reContext.java => FileRestoreContext.java} | 23 +++-- .../xpack/ccr/repository/CcrRepository.java | 99 +------------------ 3 files changed, 16 insertions(+), 108 deletions(-) rename server/src/main/java/org/elasticsearch/repositories/blobstore/{BlobRestoreContext.java => FileRestoreContext.java} (92%) 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 20f72cfef7110..39b8b2447f869 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1442,7 +1442,7 @@ protected InputStream openSlice(long slice) throws IOException { /** * Context for restore operations */ - private class RestoreContext extends BlobRestoreContext { + private class RestoreContext extends FileRestoreContext { /** * Constructs new restore context diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobRestoreContext.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java similarity index 92% rename from server/src/main/java/org/elasticsearch/repositories/blobstore/BlobRestoreContext.java rename to server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java index dc1a837e21923..89fed13bfd7f5 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobRestoreContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java @@ -54,9 +54,9 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; -public abstract class BlobRestoreContext { +public abstract class FileRestoreContext { - private static final Logger logger = LogManager.getLogger(BlobRestoreContext.class); + private static final Logger logger = LogManager.getLogger(FileRestoreContext.class); private final String repositoryName; private final IndexShard indexShard; @@ -73,7 +73,7 @@ public abstract class BlobRestoreContext { * @param recoveryState recovery state to report progress * @param bufferSize buffer size for restore */ - protected BlobRestoreContext(String repositoryName, IndexShard indexShard, SnapshotId snapshotId, RecoveryState recoveryState, + protected FileRestoreContext(String repositoryName, IndexShard indexShard, SnapshotId snapshotId, RecoveryState recoveryState, int bufferSize) { this.repositoryName = repositoryName; this.recoveryState = recoveryState; @@ -118,7 +118,8 @@ public void restore(SnapshotFiles snapshotFiles) throws IOException { logger.trace("[{}] [{}] restoring from to an empty shard", shardId, snapshotId); recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; } catch (IOException e) { - logger.warn(() -> new ParameterizedMessage("{} Can't read metadata from store, will not reuse any local file while restoring", shardId), e); + logger.warn(new ParameterizedMessage("[{}] [{}] Can't read metadata from store, will not reuse local files during restore", + shardId, snapshotId), e); recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; } @@ -134,7 +135,8 @@ public void restore(SnapshotFiles snapshotFiles) throws IOException { maybeRecalculateMetadataHash(fileInfo, recoveryTargetMetadata); } catch (Exception e) { // if the index is broken we might not be able to read it - logger.warn(() -> new ParameterizedMessage("{} Can't calculate hash from blog for file [{}] [{}]", shardId, fileInfo.physicalName(), fileInfo.metadata()), e); + logger.warn(new ParameterizedMessage("[{}] Can't calculate hash from blog for file [{}] [{}]", shardId, + fileInfo.physicalName(), fileInfo.metadata()), e); } snapshotMetaData.put(fileInfo.metadata().name(), fileInfo.metadata()); fileInfos.put(fileInfo.metadata().name(), fileInfo); @@ -152,7 +154,8 @@ public void restore(SnapshotFiles snapshotFiles) throws IOException { BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfos.get(md.name()); recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), true); if (logger.isTraceEnabled()) { - logger.trace("[{}] [{}] not_recovering [{}] from [{}], exists in local store and is same", shardId, snapshotId, fileInfo.physicalName(), fileInfo.name()); + logger.trace("[{}] [{}] not_recovering file [{}] from [{}], exists in local store and is same", shardId, snapshotId, + fileInfo.physicalName(), fileInfo.name()); } } @@ -161,13 +164,13 @@ public void restore(SnapshotFiles snapshotFiles) throws IOException { filesToRecover.add(fileInfo); recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), false); if (logger.isTraceEnabled()) { - logger.trace("[{}] [{}] recovering [{}] from [{}], exists in local store but is different", shardId, snapshotId, + logger.trace("[{}] [{}] recovering [{}] from [{}]", shardId, snapshotId, fileInfo.physicalName(), fileInfo.name()); } } if (filesToRecover.isEmpty()) { - logger.trace("no files to recover, all exists within the local store"); + logger.trace("[{}] [{}] no files to recover, all exist within the local store", shardId, snapshotId); } try { @@ -214,11 +217,11 @@ public void restore(SnapshotFiles snapshotFiles) throws IOException { store.deleteQuiet("restore", storeFile); store.directory().deleteFile(storeFile); } catch (IOException e) { - logger.warn("[{}] failed to delete file [{}] during snapshot cleanup", snapshotId, storeFile); + logger.warn("[{}] [{}] failed to delete file [{}] during snapshot cleanup", shardId, snapshotId, storeFile); } } } catch (IOException e) { - logger.warn("[{}] failed to list directory - some of files might not be deleted", snapshotId); + logger.warn("[{}] [{}] failed to list directory - some of files might not be deleted", shardId, snapshotId); } } finally { store.decRef(); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 0e06820721383..9b780464ed0c1 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -8,12 +8,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.IndexFormatTooNewException; -import org.apache.lucene.index.IndexFormatTooOldException; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.Version; @@ -34,7 +29,6 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.EngineException; @@ -51,7 +45,7 @@ import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; -import org.elasticsearch.repositories.blobstore.BlobRestoreContext; +import org.elasticsearch.repositories.blobstore.FileRestoreContext; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotShardFailure; @@ -297,7 +291,7 @@ private void maybeUpdateMappings(Client localClient, Client remoteClient, Index } } - private static class RestoreSession extends BlobRestoreContext implements Closeable { + private static class RestoreSession extends FileRestoreContext implements Closeable { private static final int BUFFER_SIZE = 1 << 16; @@ -339,100 +333,11 @@ void restoreFiles() throws IOException { restore(snapshotFiles); } -// void restoreFiles() { -// Store.MetadataSnapshot recoveryMetadata; -// try { -// recoveryMetadata = indexShard.snapshotStoreMetadata(); -// } catch (IOException e) { -// throw new IndexShardRecoveryException(shardId, "failed access store metadata", e); -// } -// -// final StoreFileMetaData restoredSegmentsFile = sourceMetaData.getSegmentsFile(); -// if (restoredSegmentsFile == null) { -// throw new IndexShardRestoreFailedException(shardId, "Ccr leader index has no segments file"); -// } -// -// final Store.RecoveryDiff diff = sourceMetaData.recoveryDiff(recoveryMetadata); -// for (StoreFileMetaData fileMetaData : diff.identical) { -// recoveryState.getIndex().addFileDetail(fileMetaData.name(), fileMetaData.length(), true); -// logger.trace("shard [{}] not recovering file [{}], exists in local store and is identical", shardId, fileMetaData.name()); -// } -// -// List filesToRecover = new ArrayList<>(); -// for (StoreFileMetaData fileMetaData : concat(diff)) { -// filesToRecover.add(fileMetaData); -// recoveryState.getIndex().addFileDetail(fileMetaData.name(), fileMetaData.length(), false); -// logger.trace("shard [{}] recovering file [{}]", shardId, fileMetaData.name()); -// } -// -// if (filesToRecover.isEmpty()) { -// logger.trace("shard [{}] no files to recover, all exist within the local store", shardId); -// } -// -// final List deleteIfExistFiles = Arrays.asList(getDirectoryFiles()); -// for (final StoreFileMetaData fileToRecover : filesToRecover) { -// // if a file with a same physical name already exist in the store we need to delete it -// // before restoring it from the snapshot. We could be lenient and try to reuse the existing -// // store files (and compare their names/length/checksum again with the snapshot files) but to -// // avoid extra complexity we simply delete them and restore them again like StoreRecovery -// // does with dangling indices. Any existing store file that is not restored from the snapshot -// // will be clean up by RecoveryTarget.cleanFiles(). -// final String name = fileToRecover.name(); -// if (deleteIfExistFiles.contains(name)) { -// logger.trace("shard [{}] deleting pre-existing file [{}]", shardId, name); -// deleteFile(name); -// } -// -// logger.trace("[{}] restoring file [{}]", shardId, name); -// try { -// restoreFile(fileToRecover, store); -// } catch (Exception e) { -// logger.info(() -> new ParameterizedMessage("shard [{}] failed to restore file [{}]", shardId, name), e); -// throw new IndexShardRecoveryException(shardId, "failed to restore file", e); -// } -// } -// -// final SegmentInfos segmentCommitInfos; -// try { -// segmentCommitInfos = Lucene.pruneUnreferencedFiles(restoredSegmentsFile.name(), store.directory()); -// } catch (IOException e) { -// throw new IndexShardRestoreFailedException(shardId, "Failed to fetch index version after copying it over", e); -// } -// recoveryState.getIndex().updateVersion(segmentCommitInfos.getVersion()); -// -// try { -// store.cleanupAndVerify("restore complete from remote", sourceMetaData); -// } catch (IOException e) { -// throw new IndexShardRecoveryException(shardId, "failed to cleanup and verify the store", e); -// } -// } - @Override protected InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo) { return new RestoreFileInputStream(remoteClient, sessionUUID, node, fileInfo.metadata()); } - @SuppressWarnings("unchecked") - private static Iterable concat(Store.RecoveryDiff diff) { - return Iterables.concat(diff.different, diff.missing); - } - - private String[] getDirectoryFiles() { - try { - return store.directory().listAll(); - } catch (IOException e) { - throw new IndexShardRecoveryException(shardId, "failed to list directory files", e); - } - } - - private void deleteFile(String name) { - try { - store.directory().deleteFile(name); - } catch (IOException e) { - throw new IndexShardRecoveryException(shardId, "failed to delete file", e); - } - } - @Override public void close() { this.store.decRef(); From 0eba9d799be4dd9ef9d17426522cb3e44565fd11 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 7 Jan 2019 15:39:46 -0700 Subject: [PATCH 15/30] Cache file --- .../GetCcrRestoreFileChunkAction.java | 6 +- .../repository/CcrRestoreSourceService.java | 65 ++++++++++++++----- .../CcrRestoreSourceServiceTests.java | 52 +++++++++++++++ 3 files changed, 105 insertions(+), 18 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index 7a61762c7efe1..a6615babe4d31 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -72,9 +72,11 @@ public void onFailure(Exception e) { @Override protected void doRun() throws Exception { - try (CcrRestoreSourceService.FileReader fileReader = restoreSourceService.getSession(request.getSessionUUID())) { + String fileName = request.getFileName(); + String sessionUUID = request.getSessionUUID(); + try (CcrRestoreSourceService.FileReader fileReader = restoreSourceService.getSessionReader(sessionUUID, fileName)) { byte[] chunk = new byte[request.getSize()]; - fileReader.readFileBytes(request.getFileName(), chunk, request.getOffset(), request.getSize()); + fileReader.readFileBytes(chunk, request.getOffset(), request.getSize()); listener.onResponse(new GetCcrRestoreFileChunkResponse(new BytesArray(chunk))); } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index 628af67fd2788..d3c4c2df130f2 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -136,43 +136,69 @@ public synchronized void closeSession(String sessionUUID) { IOUtils.closeWhileHandlingException(restore); } - public synchronized FileReader getSession(String sessionUUID) { + public synchronized FileReader getSessionReader(String sessionUUID, String fileName) throws IOException { RestoreContext restore = onGoingRestores.get(sessionUUID); if (restore == null) { logger.info("could not get session [{}] because session not found", sessionUUID); throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); } - return new FileReader(restore.session); + return restore.getFileReader(fileName); } private class RestoreContext implements Closeable { private final String sessionUUID; private final IndexShard indexShard; - private final RefCountedCommit session; + private final RefCountedCloseable session; + private RefCountedCloseable cachedInput; private RestoreContext(String sessionUUID, IndexShard indexShard, Engine.IndexCommitRef commitRef) { this.sessionUUID = sessionUUID; this.indexShard = indexShard; - this.session = new RefCountedCommit(commitRef); + this.session = new RefCountedCloseable<>("commit-ref", commitRef); } Store.MetadataSnapshot getMetaData() throws IOException { indexShard.store().incRef(); try { - return indexShard.store().getMetadata(session.commitRef.getIndexCommit()); + return indexShard.store().getMetadata(session.object.getIndexCommit()); } finally { indexShard.store().decRef(); } } + FileReader getFileReader(String fileName) throws IOException { + if (cachedInput != null) { + if (fileName.equals(cachedInput.name)) { + return new FileReader(session, cachedInput); + } else { + cachedInput.decRef(); + openNewIndexInput(fileName); + return new FileReader(session, cachedInput); + } + } else { + openNewIndexInput(fileName); + return new FileReader(session, cachedInput); + + } + } + @Override public void close() { assert Thread.holdsLock(CcrRestoreSourceService.this); removeSessionForShard(sessionUUID, indexShard); + if (cachedInput != null) { + cachedInput.decRef(); + } session.decRef(); } + private void openNewIndexInput(String fileName) throws IOException { + Engine.IndexCommitRef commitRef = session.object; + IndexInput indexInput = commitRef.getIndexCommit().getDirectory().openInput(fileName, IOContext.READONCE); + cachedInput = new RefCountedCloseable<>(fileName, indexInput); + } + private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { logger.debug("closing session [{}] for shard [{}]", sessionUUID, indexShard.shardId()); HashSet sessions = sessionsForShard.get(indexShard); @@ -185,34 +211,40 @@ private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { } } - private static class RefCountedCommit extends AbstractRefCounted { + private static class RefCountedCloseable extends AbstractRefCounted { + + private static final String NAME = "ref-counted-session-object"; - private static final String NAME = "ref-counted-session"; - private final Engine.IndexCommitRef commitRef; + private final String name; + private final T object; - private RefCountedCommit(Engine.IndexCommitRef commitRef) { + private RefCountedCloseable(String name, T object) { super(NAME); - this.commitRef = commitRef; + this.name = name; + this.object = object; } @Override protected void closeInternal() { - IOUtils.closeWhileHandlingException(commitRef); + IOUtils.closeWhileHandlingException(object); } } public static class FileReader implements Closeable { - private final RefCountedCommit session; + private final RefCountedCloseable session; + private final RefCountedCloseable input; - private FileReader(RefCountedCommit session) { + private FileReader(RefCountedCloseable session, RefCountedCloseable input) { this.session = session; + this.input = input; + input.incRef(); session.incRef(); } - public void readFileBytes(String fileName, byte[] chunk, long offset, int length) throws IOException { - Engine.IndexCommitRef commitRef = session.commitRef; - try (IndexInput in = commitRef.getIndexCommit().getDirectory().openInput(fileName, IOContext.READONCE)) { + public void readFileBytes(byte[] chunk, long offset, int length) throws IOException { + synchronized (input.object) { + IndexInput in = input.object; in.seek(offset); in.readBytes(chunk, 0, length); } @@ -220,6 +252,7 @@ public void readFileBytes(String fileName, byte[] chunk, long offset, int length @Override public void close() { + input.decRef(); session.decRef(); } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java index dfa7e5ef12660..d750a03d8772a 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java @@ -6,14 +6,19 @@ package org.elasticsearch.xpack.ccr.repository; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; +import org.elasticsearch.index.store.StoreFileMetaData; import org.junit.Before; import java.io.IOException; +import java.util.ArrayList; import java.util.HashSet; public class CcrRestoreSourceServiceTests extends IndexShardTestCase { @@ -122,4 +127,51 @@ public void testCloseShardListenerFunctionality() throws IOException { restoreSourceService.closeSession(sessionUUID3); closeShards(indexShard1, indexShard2); } + + public void testGetSessionReader() throws IOException { + IndexShard indexShard1 = newStartedShard(true); + final String sessionUUID1 = UUIDs.randomBase64UUID(); + + restoreSourceService.openSession(sessionUUID1, indexShard1); + + ArrayList files = new ArrayList<>(); + indexShard1.snapshotStoreMetadata().forEach(files::add); + + StoreFileMetaData fileMetaData = files.get(0); + String fileName = fileMetaData.name(); + + byte[] expectedBytes = new byte[(int) fileMetaData.length()]; + byte[] actualBytes = new byte[(int) fileMetaData.length()]; + Engine.IndexCommitRef indexCommitRef = indexShard1.acquireSafeIndexCommit(); + try (IndexInput indexInput = indexCommitRef.getIndexCommit().getDirectory().openInput(fileName, IOContext.READONCE)) { + indexInput.seek(0); + indexInput.readBytes(expectedBytes, 0, (int) fileMetaData.length()); + } + + try (CcrRestoreSourceService.FileReader reader = restoreSourceService.getSessionReader(sessionUUID1, fileName)) { + reader.readFileBytes(actualBytes, 0, (int) fileMetaData.length()); + } + + assertArrayEquals(expectedBytes, actualBytes); + restoreSourceService.closeSession(sessionUUID1); + closeShards(indexShard1); + } + + public void testGetSessionDoesNotLeakFileIfClosed() throws IOException { + IndexShard indexShard1 = newStartedShard(true); + final String sessionUUID1 = UUIDs.randomBase64UUID(); + + restoreSourceService.openSession(sessionUUID1, indexShard1); + + ArrayList files = new ArrayList<>(); + indexShard1.snapshotStoreMetadata().forEach(files::add); + try (CcrRestoreSourceService.FileReader reader = restoreSourceService.getSessionReader(sessionUUID1, files.get(0).name())) { + // Using try with close to ensure that reader is closed. + assertNotNull(reader); + } + + restoreSourceService.closeSession(sessionUUID1); + closeShards(indexShard1); + // Exception will be thrown if file is not closed. + } } From 0e5699566b493f6b85f9437bae99f4a392efc328 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 7 Jan 2019 16:04:07 -0700 Subject: [PATCH 16/30] Work in progress --- .../common/bytes/PagedBytesReference.java | 13 +++---- .../bytes/ReleasablePagedBytesReference.java | 6 ++-- .../common/io/stream/BytesStreamOutput.java | 2 +- .../stream/ReleasableBytesStreamOutput.java | 2 +- .../bytes/PagedBytesReferenceTests.java | 4 +-- .../bytes/AbstractBytesReferenceTestCase.java | 2 +- .../GetCcrRestoreFileChunkAction.java | 34 +++++++++++++++---- 7 files changed, 39 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/bytes/PagedBytesReference.java b/server/src/main/java/org/elasticsearch/common/bytes/PagedBytesReference.java index c754d6b8eae9e..f15b3b9cf3295 100644 --- a/server/src/main/java/org/elasticsearch/common/bytes/PagedBytesReference.java +++ b/server/src/main/java/org/elasticsearch/common/bytes/PagedBytesReference.java @@ -21,7 +21,6 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.ByteArray; import org.elasticsearch.common.util.PageCacheRecycler; @@ -35,17 +34,15 @@ public class PagedBytesReference extends BytesReference { private static final int PAGE_SIZE = PageCacheRecycler.BYTE_PAGE_SIZE; - private final BigArrays bigarrays; - protected final ByteArray byteArray; + private final ByteArray byteArray; private final int offset; private final int length; - public PagedBytesReference(BigArrays bigarrays, ByteArray byteArray, int length) { - this(bigarrays, byteArray, 0, length); + public PagedBytesReference(ByteArray byteArray, int length) { + this(byteArray, 0, length); } - public PagedBytesReference(BigArrays bigarrays, ByteArray byteArray, int from, int length) { - this.bigarrays = bigarrays; + private PagedBytesReference(ByteArray byteArray, int from, int length) { this.byteArray = byteArray; this.offset = from; this.length = length; @@ -67,7 +64,7 @@ public BytesReference slice(int from, int length) { throw new IllegalArgumentException("can't slice a buffer with length [" + length() + "], with slice parameters from [" + from + "], length [" + length + "]"); } - return new PagedBytesReference(bigarrays, byteArray, offset + from, length); + return new PagedBytesReference(byteArray, offset + from, length); } @Override diff --git a/server/src/main/java/org/elasticsearch/common/bytes/ReleasablePagedBytesReference.java b/server/src/main/java/org/elasticsearch/common/bytes/ReleasablePagedBytesReference.java index ac90e546f7eb5..209a6edc5696a 100644 --- a/server/src/main/java/org/elasticsearch/common/bytes/ReleasablePagedBytesReference.java +++ b/server/src/main/java/org/elasticsearch/common/bytes/ReleasablePagedBytesReference.java @@ -21,7 +21,6 @@ import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.ByteArray; /** @@ -32,9 +31,8 @@ public final class ReleasablePagedBytesReference extends PagedBytesReference imp private final Releasable releasable; - public ReleasablePagedBytesReference(BigArrays bigarrays, ByteArray byteArray, int length, - Releasable releasable) { - super(bigarrays, byteArray, length); + public ReleasablePagedBytesReference(ByteArray byteArray, int length, Releasable releasable) { + super(byteArray, length); this.releasable = releasable; } diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java b/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java index 768dbd1d17775..ad9cde3abbc48 100644 --- a/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java +++ b/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java @@ -140,7 +140,7 @@ public int size() { @Override public BytesReference bytes() { - return new PagedBytesReference(bigArrays, bytes, count); + return new PagedBytesReference(bytes, count); } /** diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java b/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java index 5616da2db9636..725ecd1c3cc4f 100644 --- a/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java +++ b/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java @@ -56,7 +56,7 @@ public ReleasableBytesStreamOutput(int expectedSize, BigArrays bigArrays) { */ @Override public ReleasablePagedBytesReference bytes() { - return new ReleasablePagedBytesReference(bigArrays, bytes, count, releasable); + return new ReleasablePagedBytesReference(bytes, count, releasable); } @Override diff --git a/server/src/test/java/org/elasticsearch/common/bytes/PagedBytesReferenceTests.java b/server/src/test/java/org/elasticsearch/common/bytes/PagedBytesReferenceTests.java index 0938b70906f52..40e02c560a9c0 100644 --- a/server/src/test/java/org/elasticsearch/common/bytes/PagedBytesReferenceTests.java +++ b/server/src/test/java/org/elasticsearch/common/bytes/PagedBytesReferenceTests.java @@ -120,8 +120,8 @@ public void testEquals() { } // get refs & compare - BytesReference pbr = new PagedBytesReference(bigarrays, ba1, length); - BytesReference pbr2 = new PagedBytesReference(bigarrays, ba2, length); + BytesReference pbr = new PagedBytesReference(ba1, length); + BytesReference pbr2 = new PagedBytesReference(ba2, length); assertEquals(pbr, pbr2); int offsetToFlip = randomIntBetween(0, length - 1); int value = ~Byte.toUnsignedInt(ba1.get(offsetToFlip)); diff --git a/test/framework/src/main/java/org/elasticsearch/common/bytes/AbstractBytesReferenceTestCase.java b/test/framework/src/main/java/org/elasticsearch/common/bytes/AbstractBytesReferenceTestCase.java index 1a00698325974..1a907feabe24a 100644 --- a/test/framework/src/main/java/org/elasticsearch/common/bytes/AbstractBytesReferenceTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/common/bytes/AbstractBytesReferenceTestCase.java @@ -528,7 +528,7 @@ public void testEquals() throws IOException { public void testSliceEquals() { int length = randomIntBetween(100, PAGE_SIZE * randomIntBetween(2, 5)); ByteArray ba1 = bigarrays.newByteArray(length, false); - BytesReference pbr = new PagedBytesReference(bigarrays, ba1, length); + BytesReference pbr = new PagedBytesReference(ba1, length); // test equality of slices int sliceFrom = randomIntBetween(0, pbr.length()); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index a6615babe4d31..2ed9c2f61db61 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -6,17 +6,21 @@ package org.elasticsearch.xpack.ccr.action.repositories; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.ReleasablePagedBytesReference; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.ByteArray; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; @@ -51,14 +55,16 @@ public static class TransportGetCcrRestoreFileChunkAction private final CcrRestoreSourceService restoreSourceService; private final ThreadPool threadPool; + private final BigArrays bigArrays; @Inject - public TransportGetCcrRestoreFileChunkAction(ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, + public TransportGetCcrRestoreFileChunkAction(BigArrays bigArrays, TransportService transportService, ActionFilters actionFilters, CcrRestoreSourceService restoreSourceService) { super(NAME, transportService, actionFilters, GetCcrRestoreFileChunkRequest::new); TransportActionProxy.registerProxyAction(transportService, NAME, GetCcrRestoreFileChunkResponse::new); - this.threadPool = threadPool; + this.threadPool = transportService.getThreadPool(); this.restoreSourceService = restoreSourceService; + this.bigArrays = bigArrays; } @Override @@ -72,12 +78,26 @@ public void onFailure(Exception e) { @Override protected void doRun() throws Exception { + int bytesRequested = request.getSize(); + long fileOffset = request.getOffset(); + ByteArray byteArray = bigArrays.newByteArray(bytesRequested, false); String fileName = request.getFileName(); String sessionUUID = request.getSessionUUID(); - try (CcrRestoreSourceService.FileReader fileReader = restoreSourceService.getSessionReader(sessionUUID, fileName)) { - byte[] chunk = new byte[request.getSize()]; - fileReader.readFileBytes(chunk, request.getOffset(), request.getSize()); - listener.onResponse(new GetCcrRestoreFileChunkResponse(new BytesArray(chunk))); + // This is currently safe to do because calling `onResponse` will serialize the bytes to the network layer data + // structure on the same thread. So the bytes will be copied before the reference is released. + try (ReleasablePagedBytesReference reference = new ReleasablePagedBytesReference(byteArray, bytesRequested, byteArray)) { + try (CcrRestoreSourceService.FileReader fileReader = restoreSourceService.getSessionReader(sessionUUID, fileName)) { + BytesRefIterator refIterator = reference.iterator(); + BytesRef ref; + int bytesWritten = 0; + while ((ref = refIterator.next()) != null) { + byte[] refBytes = ref.bytes; + fileReader.readFileBytes(refBytes, fileOffset + bytesWritten, ref.length); + bytesWritten += ref.length; + } + + listener.onResponse(new GetCcrRestoreFileChunkResponse(reference)); + } } } }); From 8083076e4b7d248f2a97dabd97d76243e6978d4e Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 7 Jan 2019 16:18:03 -0700 Subject: [PATCH 17/30] Checkstyle --- .../repositories/GetCcrRestoreFileChunkAction.java | 8 ++++---- .../xpack/ccr/repository/CcrRepository.java | 6 ++---- .../ccr/repository/CcrRestoreSourceService.java | 14 +++++++------- .../repository/CcrRestoreSourceServiceTests.java | 4 ++-- 4 files changed, 15 insertions(+), 17 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index 2ed9c2f61db61..83eee27a39478 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -80,19 +80,19 @@ public void onFailure(Exception e) { protected void doRun() throws Exception { int bytesRequested = request.getSize(); long fileOffset = request.getOffset(); - ByteArray byteArray = bigArrays.newByteArray(bytesRequested, false); + ByteArray array = bigArrays.newByteArray(bytesRequested, false); String fileName = request.getFileName(); String sessionUUID = request.getSessionUUID(); // This is currently safe to do because calling `onResponse` will serialize the bytes to the network layer data // structure on the same thread. So the bytes will be copied before the reference is released. - try (ReleasablePagedBytesReference reference = new ReleasablePagedBytesReference(byteArray, bytesRequested, byteArray)) { - try (CcrRestoreSourceService.FileReader fileReader = restoreSourceService.getSessionReader(sessionUUID, fileName)) { + try (ReleasablePagedBytesReference reference = new ReleasablePagedBytesReference(array, bytesRequested, array)) { + try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID, fileName)) { BytesRefIterator refIterator = reference.iterator(); BytesRef ref; int bytesWritten = 0; while ((ref = refIterator.next()) != null) { byte[] refBytes = ref.bytes; - fileReader.readFileBytes(refBytes, fileOffset + bytesWritten, ref.length); + reader.readFileBytes(refBytes, fileOffset + bytesWritten, ref.length); bytesWritten += ref.length; } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 9b780464ed0c1..9e7425b3d1659 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -298,7 +298,6 @@ private static class RestoreSession extends FileRestoreContext implements Closea private final Client remoteClient; private final String sessionUUID; private final DiscoveryNode node; - private final ShardId shardId; private final Store store; private final Store.MetadataSnapshot sourceMetaData; @@ -308,7 +307,6 @@ private static class RestoreSession extends FileRestoreContext implements Closea this.remoteClient = remoteClient; this.sessionUUID = sessionUUID; this.node = node; - this.shardId = indexShard.shardId(); this.store = indexShard.store(); this.store.incRef(); this.sourceMetaData = sourceMetaData; @@ -319,8 +317,8 @@ static RestoreSession openSession(String repositoryName, Client remoteClient, Sh String sessionUUID = UUIDs.randomBase64UUID(); PutCcrRestoreSessionAction.PutCcrRestoreSessionResponse response = remoteClient.execute(PutCcrRestoreSessionAction.INSTANCE, new PutCcrRestoreSessionRequest(sessionUUID, leaderShardId)).actionGet(); - Store.MetadataSnapshot sourceFileMetaData = response.getStoreFileMetaData(); - return new RestoreSession(repositoryName, remoteClient, sessionUUID, response.getNode(), indexShard, recoveryState, sourceFileMetaData); + return new RestoreSession(repositoryName, remoteClient, sessionUUID, response.getNode(), indexShard, recoveryState, + response.getStoreFileMetaData()); } void restoreFiles() throws IOException { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index d3c4c2df130f2..f4bd8c83a5fea 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -136,7 +136,7 @@ public synchronized void closeSession(String sessionUUID) { IOUtils.closeWhileHandlingException(restore); } - public synchronized FileReader getSessionReader(String sessionUUID, String fileName) throws IOException { + public synchronized Reader getSessionReader(String sessionUUID, String fileName) throws IOException { RestoreContext restore = onGoingRestores.get(sessionUUID); if (restore == null) { logger.info("could not get session [{}] because session not found", sessionUUID); @@ -167,18 +167,18 @@ Store.MetadataSnapshot getMetaData() throws IOException { } } - FileReader getFileReader(String fileName) throws IOException { + Reader getFileReader(String fileName) throws IOException { if (cachedInput != null) { if (fileName.equals(cachedInput.name)) { - return new FileReader(session, cachedInput); + return new Reader(session, cachedInput); } else { cachedInput.decRef(); openNewIndexInput(fileName); - return new FileReader(session, cachedInput); + return new Reader(session, cachedInput); } } else { openNewIndexInput(fileName); - return new FileReader(session, cachedInput); + return new Reader(session, cachedInput); } } @@ -230,12 +230,12 @@ protected void closeInternal() { } } - public static class FileReader implements Closeable { + public static class Reader implements Closeable { private final RefCountedCloseable session; private final RefCountedCloseable input; - private FileReader(RefCountedCloseable session, RefCountedCloseable input) { + private Reader(RefCountedCloseable session, RefCountedCloseable input) { this.session = session; this.input = input; input.incRef(); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java index d750a03d8772a..1bce9df3136a3 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java @@ -148,7 +148,7 @@ public void testGetSessionReader() throws IOException { indexInput.readBytes(expectedBytes, 0, (int) fileMetaData.length()); } - try (CcrRestoreSourceService.FileReader reader = restoreSourceService.getSessionReader(sessionUUID1, fileName)) { + try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID1, fileName)) { reader.readFileBytes(actualBytes, 0, (int) fileMetaData.length()); } @@ -165,7 +165,7 @@ public void testGetSessionDoesNotLeakFileIfClosed() throws IOException { ArrayList files = new ArrayList<>(); indexShard1.snapshotStoreMetadata().forEach(files::add); - try (CcrRestoreSourceService.FileReader reader = restoreSourceService.getSessionReader(sessionUUID1, files.get(0).name())) { + try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID1, files.get(0).name())) { // Using try with close to ensure that reader is closed. assertNotNull(reader); } From 6c011bca14fcc30aa6b30d35cabfb355f036c2b3 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 7 Jan 2019 16:42:54 -0700 Subject: [PATCH 18/30] Fix blob store repository --- .../blobstore/BlobStoreRepository.java | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) 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 39b8b2447f869..97837b111f65a 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -851,7 +851,9 @@ public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, @Override public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { final Context context = new Context(snapshotId, indexId, shard.shardId(), snapshotShardId); - final RestoreContext snapshotContext = new RestoreContext(shard, snapshotId, recoveryState); + BlobPath path = basePath().add("indices").add(indexId.getId()).add(Integer.toString(snapshotShardId.getId())); + BlobContainer blobContainer = blobStore().blobContainer(path); + final RestoreContext snapshotContext = new RestoreContext(shard, snapshotId, recoveryState, blobContainer); try { BlobStoreIndexShardSnapshot snapshot = context.loadSnapshot(); SnapshotFiles snapshotFiles = new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles()); @@ -1444,24 +1446,27 @@ protected InputStream openSlice(long slice) throws IOException { */ private class RestoreContext extends FileRestoreContext { + private final BlobContainer blobContainer; + /** * Constructs new restore context - * - * @param indexShard shard to restore into + * @param indexShard shard to restore into * @param snapshotId snapshot id * @param recoveryState recovery state to report progress + * @param blobContainer */ - RestoreContext(IndexShard indexShard, SnapshotId snapshotId, RecoveryState recoveryState) { + RestoreContext(IndexShard indexShard, SnapshotId snapshotId, RecoveryState recoveryState, BlobContainer blobContainer) { super(metadata.name(), indexShard, snapshotId, recoveryState, BUFFER_SIZE); + this.blobContainer = blobContainer; } @Override protected InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo) { if (restoreRateLimiter == null) { - return new PartSliceStream(blobContainer.get(), fileInfo); + return new PartSliceStream(blobContainer, fileInfo); } else { RateLimitingInputStream.Listener listener = restoreRateLimitingTimeInNanos::inc; - return new RateLimitingInputStream(new PartSliceStream(blobContainer.get(), fileInfo), restoreRateLimiter, listener); + return new RateLimitingInputStream(new PartSliceStream(blobContainer, fileInfo), restoreRateLimiter, listener); } } } From 47d7576c53608de78e7328be90236cb5855bd781 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 7 Jan 2019 16:56:05 -0700 Subject: [PATCH 19/30] Add descriptor --- .../repositories/blobstore/BlobStoreRepository.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 97837b111f65a..175ec132a34f3 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1450,10 +1450,10 @@ private class RestoreContext extends FileRestoreContext { /** * Constructs new restore context - * @param indexShard shard to restore into - * @param snapshotId snapshot id - * @param recoveryState recovery state to report progress - * @param blobContainer + * @param indexShard shard to restore into + * @param snapshotId snapshot id + * @param recoveryState recovery state to report progress + * @param blobContainer the blob container to read the files from */ RestoreContext(IndexShard indexShard, SnapshotId snapshotId, RecoveryState recoveryState, BlobContainer blobContainer) { super(metadata.name(), indexShard, snapshotId, recoveryState, BUFFER_SIZE); From bcef903740234a0b9fa9da0a4b5abbe9cb6e31dc Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 8 Jan 2019 09:24:07 -0700 Subject: [PATCH 20/30] Changes --- .../repositories/blobstore/BlobStoreRepository.java | 2 +- .../xpack/ccr/repository/CcrRepository.java | 8 ++------ .../xpack/ccr/repository/CcrRestoreSourceService.java | 9 +++++---- 3 files changed, 8 insertions(+), 11 deletions(-) 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 175ec132a34f3..f9311a52b98b2 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1450,7 +1450,7 @@ private class RestoreContext extends FileRestoreContext { /** * Constructs new restore context - * @param indexShard shard to restore into + * @param indexShard shard to restore into * @param snapshotId snapshot id * @param recoveryState recovery state to report progress * @param blobContainer the blob container to read the files from diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 9e7425b3d1659..10dd419935d27 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -265,7 +265,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v String name = metadata.name(); try (RestoreSession restoreSession = RestoreSession.openSession(name, remoteClient, leaderShardId, indexShard, recoveryState)) { restoreSession.restoreFiles(); - } catch (IOException e) { + } catch (Exception e) { throw new IndexShardRestoreFailedException(indexShard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); } @@ -298,7 +298,6 @@ private static class RestoreSession extends FileRestoreContext implements Closea private final Client remoteClient; private final String sessionUUID; private final DiscoveryNode node; - private final Store store; private final Store.MetadataSnapshot sourceMetaData; RestoreSession(String repositoryName, Client remoteClient, String sessionUUID, DiscoveryNode node, IndexShard indexShard, @@ -307,8 +306,6 @@ private static class RestoreSession extends FileRestoreContext implements Closea this.remoteClient = remoteClient; this.sessionUUID = sessionUUID; this.node = node; - this.store = indexShard.store(); - this.store.incRef(); this.sourceMetaData = sourceMetaData; } @@ -338,7 +335,6 @@ protected InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileI @Override public void close() { - this.store.decRef(); ClearCcrRestoreSessionRequest clearRequest = new ClearCcrRestoreSessionRequest(sessionUUID, node); ClearCcrRestoreSessionAction.ClearCcrRestoreSessionResponse response = remoteClient.execute(ClearCcrRestoreSessionAction.INSTANCE, clearRequest).actionGet(); @@ -383,7 +379,7 @@ public int read(byte[] bytes, int off, int len) throws IOException { if (bytesReceived > bytesRequested) { throw new IOException("More bytes [" + bytesReceived + "] received than requested [" + bytesRequested + "]"); } - + BytesRefIterator iterator = fileChunk.iterator(); BytesRef ref; int bytesWritten = 0; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index f4bd8c83a5fea..d754444cac70c 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -238,15 +238,16 @@ public static class Reader implements Closeable { private Reader(RefCountedCloseable session, RefCountedCloseable input) { this.session = session; this.input = input; - input.incRef(); session.incRef(); + input.incRef(); } public void readFileBytes(byte[] chunk, long offset, int length) throws IOException { synchronized (input.object) { - IndexInput in = input.object; - in.seek(offset); - in.readBytes(chunk, 0, length); + try (IndexInput in = input.object.clone()) { + in.seek(offset); + in.readBytes(chunk, 0, length); + } } } From 483f5c73653e75b427ec90a02beb0d43b1aef59a Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 8 Jan 2019 10:59:24 -0700 Subject: [PATCH 21/30] Chnages --- .../index/shard/IndexShardTestCase.java | 10 ++++++++++ .../xpack/ccr/repository/CcrRepository.java | 15 +++++---------- .../repository/CcrRestoreSourceService.java | 4 +--- .../CcrRestoreSourceServiceTests.java | 19 +++++++++++++++---- 4 files changed, 31 insertions(+), 17 deletions(-) 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 84e88b6accc5b..d19d4eaf2af38 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 @@ -438,6 +438,16 @@ protected IndexShard newStartedShard(final boolean primary) throws IOException { return newStartedShard(primary, Settings.EMPTY, new InternalEngineFactory()); } + /** + * Creates a new empty shard and starts it. + * + * @param primary controls whether the shard will be a primary or a replica. + * @param settings the settings to use for this shard + */ + protected IndexShard newStartedShard(final boolean primary, Settings settings) throws IOException { + return newStartedShard(primary, settings, new InternalEngineFactory()); + } + /** * Creates a new empty shard with the specified settings and engine factory and starts it. * diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 10dd419935d27..6e9b6308bc663 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -9,8 +9,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; @@ -27,6 +25,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.index.Index; @@ -379,14 +378,10 @@ public int read(byte[] bytes, int off, int len) throws IOException { if (bytesReceived > bytesRequested) { throw new IOException("More bytes [" + bytesReceived + "] received than requested [" + bytesRequested + "]"); } - - BytesRefIterator iterator = fileChunk.iterator(); - BytesRef ref; - int bytesWritten = 0; - while ((ref = iterator.next()) != null) { - byte[] refBytes = ref.bytes; - System.arraycopy(refBytes, 0, bytes, off + bytesWritten, refBytes.length); - bytesWritten += ref.length; + + try (StreamInput streamInput = fileChunk.streamInput()) { + int bytesRead = streamInput.read(bytes, 0, bytesReceived); + assert bytesRead == bytesReceived : "Did not read the correct number of bytes"; } pos += bytesReceived; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index d754444cac70c..0dfbf2433e9a5 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -213,13 +213,11 @@ private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { private static class RefCountedCloseable extends AbstractRefCounted { - private static final String NAME = "ref-counted-session-object"; - private final String name; private final T object; private RefCountedCloseable(String name, T object) { - super(NAME); + super(name); this.name = name; this.object = object; } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java index 1bce9df3136a3..f9b67e2119662 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java @@ -158,20 +158,31 @@ public void testGetSessionReader() throws IOException { } public void testGetSessionDoesNotLeakFileIfClosed() throws IOException { - IndexShard indexShard1 = newStartedShard(true); + Settings settings = Settings.builder().put("index.merge.enabled", false).build(); + IndexShard indexShard = newStartedShard(true, settings); + for (int i = 0; i < 5; i++) { + indexDoc(indexShard, "_doc", Integer.toString(i)); + flushShard(indexShard, true); + } final String sessionUUID1 = UUIDs.randomBase64UUID(); - restoreSourceService.openSession(sessionUUID1, indexShard1); + restoreSourceService.openSession(sessionUUID1, indexShard); ArrayList files = new ArrayList<>(); - indexShard1.snapshotStoreMetadata().forEach(files::add); + indexShard.snapshotStoreMetadata().forEach(files::add); try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID1, files.get(0).name())) { // Using try with close to ensure that reader is closed. assertNotNull(reader); } + // Request a second file to ensure that original file is not leaked + try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID1, files.get(1).name())) { + // Using try with close to ensure that reader is closed. + assertNotNull(reader); + } + restoreSourceService.closeSession(sessionUUID1); - closeShards(indexShard1); + closeShards(indexShard); // Exception will be thrown if file is not closed. } } From 83a1c65e1f52db82ddad29a7fe52fd6654ed5132 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 9 Jan 2019 10:19:36 -0700 Subject: [PATCH 22/30] Checkstyle --- .../repositories/blobstore/BlobStoreRepository.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 c7668ebec3533..9f2297b48775b 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -850,7 +850,8 @@ public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, } @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) { final Context context = new Context(snapshotId, indexId, shard.shardId(), snapshotShardId); BlobPath path = basePath().add("indices").add(indexId.getId()).add(Integer.toString(snapshotShardId.getId())); BlobContainer blobContainer = blobStore().blobContainer(path); From 7835006b66d940d4f366cfebb8b47f2b7787b271 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 9 Jan 2019 13:45:26 -0700 Subject: [PATCH 23/30] Changes --- .../blobstore/FileRestoreContext.java | 6 ++ .../GetCcrRestoreFileChunkAction.java | 18 ++---- .../GetCcrRestoreFileChunkRequest.java | 17 ++---- .../xpack/ccr/repository/CcrRepository.java | 2 +- .../repository/CcrRestoreSourceService.java | 55 ++++++++++++++----- .../CcrRestoreSourceServiceTests.java | 25 ++++++++- 6 files changed, 83 insertions(+), 40 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java index 89fed13bfd7f5..2f837812ae2e2 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java @@ -54,6 +54,12 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; +/** + * This context will execute a file restore of the lucene files. It is primarily designed to be used to + * restore from some form of a snapshot. It will setup a new store, identify files that need to be copied + * for the source, and perform the copies. Implementers must implement the functionality of opening the + * underlying file streams for snapshotted lucene file. + */ public abstract class FileRestoreContext { private static final Logger logger = LogManager.getLogger(FileRestoreContext.class); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index 83eee27a39478..765e00b60910d 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -6,8 +6,6 @@ package org.elasticsearch.xpack.ccr.action.repositories; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionResponse; @@ -79,7 +77,6 @@ public void onFailure(Exception e) { @Override protected void doRun() throws Exception { int bytesRequested = request.getSize(); - long fileOffset = request.getOffset(); ByteArray array = bigArrays.newByteArray(bytesRequested, false); String fileName = request.getFileName(); String sessionUUID = request.getSessionUUID(); @@ -87,16 +84,13 @@ protected void doRun() throws Exception { // structure on the same thread. So the bytes will be copied before the reference is released. try (ReleasablePagedBytesReference reference = new ReleasablePagedBytesReference(array, bytesRequested, array)) { try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID, fileName)) { - BytesRefIterator refIterator = reference.iterator(); - BytesRef ref; - int bytesWritten = 0; - while ((ref = refIterator.next()) != null) { - byte[] refBytes = ref.bytes; - reader.readFileBytes(refBytes, fileOffset + bytesWritten, ref.length); - bytesWritten += ref.length; + int bytesRead = reader.readFileBytes(reference); + if (bytesRead < bytesRequested) { + listener.onFailure(new IOException("[" + bytesRead + "] bytes requested. Only [" + bytesRead + + "] could be read from file.")); + } else { + listener.onResponse(new GetCcrRestoreFileChunkResponse(reference)); } - - listener.onResponse(new GetCcrRestoreFileChunkResponse(reference)); } } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java index 11a4f1e6491e2..d181056a88b72 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java @@ -20,7 +20,6 @@ public class GetCcrRestoreFileChunkRequest extends ActionRequest implements Remo private DiscoveryNode node; private final String sessionUUID; private final String fileName; - private final long offset; private final int size; @Override @@ -28,19 +27,18 @@ public ActionRequestValidationException validate() { return null; } - public GetCcrRestoreFileChunkRequest(DiscoveryNode node, String sessionUUID, String fileName, long offset, int size) { + public GetCcrRestoreFileChunkRequest(DiscoveryNode node, String sessionUUID, String fileName, int size) { this.sessionUUID = sessionUUID; this.node = node; this.fileName = fileName; - this.offset = offset; this.size = size; + assert size > -1 : "The file chunk request size must be positive. Found: [" + size + "]."; } GetCcrRestoreFileChunkRequest(StreamInput in) throws IOException { super(in); sessionUUID = in.readString(); fileName = in.readString(); - offset = in.readVLong(); size = in.readVInt(); } @@ -49,7 +47,6 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(sessionUUID); out.writeString(fileName); - out.writeVLong(offset); out.writeVInt(size); } @@ -58,19 +55,15 @@ public void readFrom(StreamInput in) throws IOException { throw new UnsupportedOperationException(); } - public String getSessionUUID() { + String getSessionUUID() { return sessionUUID; } - public String getFileName() { + String getFileName() { return fileName; } - public long getOffset() { - return offset; - } - - public int getSize() { + int getSize() { return size; } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 6e9b6308bc663..3e49224fe87fe 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -371,7 +371,7 @@ public int read(byte[] bytes, int off, int len) throws IOException { int bytesRequested = (int) Math.min(remainingBytes, len); String fileName = fileToRecover.name(); - GetCcrRestoreFileChunkRequest request = new GetCcrRestoreFileChunkRequest(node, sessionUUID, fileName, pos, bytesRequested); + GetCcrRestoreFileChunkRequest request = new GetCcrRestoreFileChunkRequest(node, sessionUUID, fileName, bytesRequested); BytesReference fileChunk = remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request).actionGet().getChunk(); int bytesReceived = fileChunk.length(); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index 0dfbf2433e9a5..ff7f68426344f 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -10,8 +10,12 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; @@ -151,6 +155,8 @@ private class RestoreContext implements Closeable { private final IndexShard indexShard; private final RefCountedCloseable session; private RefCountedCloseable cachedInput; + private volatile boolean sessionLocked = false; + private Releasable lockRelease = () -> sessionLocked = false; private RestoreContext(String sessionUUID, IndexShard indexShard, Engine.IndexCommitRef commitRef) { this.sessionUUID = sessionUUID; @@ -168,17 +174,21 @@ Store.MetadataSnapshot getMetaData() throws IOException { } Reader getFileReader(String fileName) throws IOException { + if (sessionLocked) { + throw new IllegalStateException("Session is currently locked by another get file chunk request"); + } + sessionLocked = true; if (cachedInput != null) { if (fileName.equals(cachedInput.name)) { - return new Reader(session, cachedInput); + return new Reader(session, cachedInput, lockRelease); } else { cachedInput.decRef(); openNewIndexInput(fileName); - return new Reader(session, cachedInput); + return new Reader(session, cachedInput, lockRelease); } } else { openNewIndexInput(fileName); - return new Reader(session, cachedInput); + return new Reader(session, cachedInput, lockRelease); } } @@ -232,27 +242,44 @@ public static class Reader implements Closeable { private final RefCountedCloseable session; private final RefCountedCloseable input; + private final Releasable lockRelease; - private Reader(RefCountedCloseable session, RefCountedCloseable input) { + private Reader(RefCountedCloseable session, RefCountedCloseable input, Releasable lockRelease) { this.session = session; this.input = input; - session.incRef(); - input.incRef(); + this.lockRelease = lockRelease; + boolean sessionRefIncremented = false; + boolean inputRefIncremented = false; + try { + session.incRef(); + sessionRefIncremented = true; + input.incRef(); + inputRefIncremented = true; + } finally { + if (sessionRefIncremented == false) { + IOUtils.closeWhileHandlingException(lockRelease); + } else if (inputRefIncremented == false) { + IOUtils.closeWhileHandlingException(session::decRef, lockRelease); + } + } } - public void readFileBytes(byte[] chunk, long offset, int length) throws IOException { - synchronized (input.object) { - try (IndexInput in = input.object.clone()) { - in.seek(offset); - in.readBytes(chunk, 0, length); - } + public int readFileBytes(BytesReference reference) throws IOException { + IndexInput in = input.object; + BytesRefIterator refIterator = reference.iterator(); + BytesRef ref; + int bytesWritten = 0; + while ((ref = refIterator.next()) != null) { + byte[] refBytes = ref.bytes; + in.readBytes(refBytes, 0, refBytes.length); + bytesWritten += ref.length; } + return bytesWritten; } @Override public void close() { - input.decRef(); - session.decRef(); + IOUtils.closeWhileHandlingException(input::decRef, session::decRef, lockRelease); } } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java index f9b67e2119662..d0b0dafdc278a 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java @@ -9,6 +9,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IllegalIndexShardStateException; @@ -148,8 +149,9 @@ public void testGetSessionReader() throws IOException { indexInput.readBytes(expectedBytes, 0, (int) fileMetaData.length()); } + BytesArray byteArray = new BytesArray(actualBytes); try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID1, fileName)) { - reader.readFileBytes(actualBytes, 0, (int) fileMetaData.length()); + reader.readFileBytes(byteArray); } assertArrayEquals(expectedBytes, actualBytes); @@ -185,4 +187,25 @@ public void testGetSessionDoesNotLeakFileIfClosed() throws IOException { closeShards(indexShard); // Exception will be thrown if file is not closed. } + + public void testCannotAccessTwoSessionReadersAtOnce() throws IOException { + IndexShard indexShard = newStartedShard(true); + final String sessionUUID1 = UUIDs.randomBase64UUID(); + + restoreSourceService.openSession(sessionUUID1, indexShard); + + ArrayList files = new ArrayList<>(); + indexShard.snapshotStoreMetadata().forEach(files::add); + + StoreFileMetaData fileMetaData = files.get(0); + String fileName = fileMetaData.name(); + + indexShard.snapshotStoreMetadata().forEach(files::add); + try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID1, fileName)) { + expectThrows(IllegalStateException.class, () -> restoreSourceService.getSessionReader(sessionUUID1, fileName)); + } + + restoreSourceService.closeSession(sessionUUID1); + closeShards(indexShard); + } } From a201e9329c99a20fbefe7f34a308fa1954e43067 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 10 Jan 2019 17:30:41 -0700 Subject: [PATCH 24/30] Small changes --- .../repositories/GetCcrRestoreFileChunkRequest.java | 5 +++-- .../ccr/repository/CcrRestoreSourceService.java | 13 +++++-------- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java index d181056a88b72..25b35964c6c90 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java @@ -17,7 +17,7 @@ public class GetCcrRestoreFileChunkRequest extends ActionRequest implements RemoteClusterAwareRequest { - private DiscoveryNode node; + private final DiscoveryNode node; private final String sessionUUID; private final String fileName; private final int size; @@ -28,8 +28,8 @@ public ActionRequestValidationException validate() { } public GetCcrRestoreFileChunkRequest(DiscoveryNode node, String sessionUUID, String fileName, int size) { - this.sessionUUID = sessionUUID; this.node = node; + this.sessionUUID = sessionUUID; this.fileName = fileName; this.size = size; assert size > -1 : "The file chunk request size must be positive. Found: [" + size + "]."; @@ -37,6 +37,7 @@ public GetCcrRestoreFileChunkRequest(DiscoveryNode node, String sessionUUID, Str GetCcrRestoreFileChunkRequest(StreamInput in) throws IOException { super(in); + node = null; sessionUUID = in.readString(); fileName = in.readString(); size = in.readVInt(); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index ff7f68426344f..829c89df2b21d 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -143,7 +143,7 @@ public synchronized void closeSession(String sessionUUID) { public synchronized Reader getSessionReader(String sessionUUID, String fileName) throws IOException { RestoreContext restore = onGoingRestores.get(sessionUUID); if (restore == null) { - logger.info("could not get session [{}] because session not found", sessionUUID); + logger.debug("could not get session [{}] because session not found", sessionUUID); throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); } return restore.getFileReader(fileName); @@ -179,7 +179,7 @@ Reader getFileReader(String fileName) throws IOException { } sessionLocked = true; if (cachedInput != null) { - if (fileName.equals(cachedInput.name)) { + if (fileName.equals(cachedInput.getName())) { return new Reader(session, cachedInput, lockRelease); } else { cachedInput.decRef(); @@ -189,7 +189,6 @@ Reader getFileReader(String fileName) throws IOException { } else { openNewIndexInput(fileName); return new Reader(session, cachedInput, lockRelease); - } } @@ -223,12 +222,10 @@ private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { private static class RefCountedCloseable extends AbstractRefCounted { - private final String name; private final T object; private RefCountedCloseable(String name, T object) { super(name); - this.name = name; this.object = object; } @@ -268,13 +265,13 @@ public int readFileBytes(BytesReference reference) throws IOException { IndexInput in = input.object; BytesRefIterator refIterator = reference.iterator(); BytesRef ref; - int bytesWritten = 0; + int bytesRead = 0; while ((ref = refIterator.next()) != null) { byte[] refBytes = ref.bytes; in.readBytes(refBytes, 0, refBytes.length); - bytesWritten += ref.length; + bytesRead += ref.length; } - return bytesWritten; + return bytesRead; } @Override From 083b9ae0ef317242b9516b306e2a0a3ed1f5069f Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 10 Jan 2019 18:54:53 -0700 Subject: [PATCH 25/30] Changes --- .../GetCcrRestoreFileChunkAction.java | 14 +- .../repository/CcrRestoreSourceService.java | 163 ++++++------------ .../CcrRestoreSourceServiceTests.java | 44 ++--- 3 files changed, 75 insertions(+), 146 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index 765e00b60910d..76559af9fd0ef 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -83,14 +83,12 @@ protected void doRun() throws Exception { // This is currently safe to do because calling `onResponse` will serialize the bytes to the network layer data // structure on the same thread. So the bytes will be copied before the reference is released. try (ReleasablePagedBytesReference reference = new ReleasablePagedBytesReference(array, bytesRequested, array)) { - try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID, fileName)) { - int bytesRead = reader.readFileBytes(reference); - if (bytesRead < bytesRequested) { - listener.onFailure(new IOException("[" + bytesRead + "] bytes requested. Only [" + bytesRead + - "] could be read from file.")); - } else { - listener.onResponse(new GetCcrRestoreFileChunkResponse(reference)); - } + CcrRestoreSourceService.RestoreSession session = restoreSourceService.getRestoreSession(sessionUUID); + try { + session.readFileBytes(fileName, reference); + listener.onResponse(new GetCcrRestoreFileChunkResponse(reference)); + } finally { + session.decRef(); } } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index 829c89df2b21d..c3cbac8c02315 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -14,8 +14,8 @@ import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; @@ -28,7 +28,6 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; -import java.io.Closeable; import java.io.IOException; import java.util.HashMap; import java.util.HashSet; @@ -40,7 +39,7 @@ public class CcrRestoreSourceService extends AbstractLifecycleComponent implemen private static final Logger logger = LogManager.getLogger(CcrRestoreSourceService.class); - private final Map onGoingRestores = ConcurrentCollections.newConcurrentMap(); + private final Map onGoingRestores = ConcurrentCollections.newConcurrentMap(); private final Map> sessionsForShard = new HashMap<>(); private final CopyOnWriteArrayList> openSessionListeners = new CopyOnWriteArrayList<>(); private final CopyOnWriteArrayList> closeSessionListeners = new CopyOnWriteArrayList<>(); @@ -55,8 +54,8 @@ public synchronized void afterIndexShardClosed(ShardId shardId, @Nullable IndexS HashSet sessions = sessionsForShard.remove(indexShard); if (sessions != null) { for (String sessionUUID : sessions) { - RestoreContext restore = onGoingRestores.remove(sessionUUID); - IOUtils.closeWhileHandlingException(restore); + RestoreSession restore = onGoingRestores.remove(sessionUUID); + restore.decRef(); } } } @@ -75,7 +74,7 @@ protected void doStop() { @Override protected synchronized void doClose() throws IOException { sessionsForShard.clear(); - IOUtils.closeWhileHandlingException(onGoingRestores.values()); + onGoingRestores.values().forEach(AbstractRefCounted::decRef); onGoingRestores.clear(); } @@ -95,7 +94,7 @@ synchronized HashSet getSessionsForShard(IndexShard indexShard) { } // default visibility for testing - synchronized RestoreContext getOngoingRestore(String sessionUUID) { + synchronized RestoreSession getOngoingRestore(String sessionUUID) { return onGoingRestores.get(sessionUUID); } @@ -103,7 +102,7 @@ synchronized RestoreContext getOngoingRestore(String sessionUUID) { // complete. Or it could be for session to have been touched. public synchronized Store.MetadataSnapshot openSession(String sessionUUID, IndexShard indexShard) throws IOException { boolean success = false; - RestoreContext restore = null; + RestoreSession restore = null; try { if (onGoingRestores.containsKey(sessionUUID)) { logger.debug("not opening new session [{}] as it already exists", sessionUUID); @@ -113,170 +112,116 @@ public synchronized Store.MetadataSnapshot openSession(String sessionUUID, Index if (indexShard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(indexShard.shardId(), "cannot open ccr restore session if shard closed"); } - restore = new RestoreContext(sessionUUID, indexShard, indexShard.acquireSafeIndexCommit()); + restore = new RestoreSession(sessionUUID, indexShard, indexShard.acquireSafeIndexCommit()); onGoingRestores.put(sessionUUID, restore); openSessionListeners.forEach(c -> c.accept(sessionUUID)); - HashSet sessions = sessionsForShard.computeIfAbsent(indexShard, (s) -> new HashSet<>()); + HashSet sessions = sessionsForShard.computeIfAbsent(indexShard, (s) -> new HashSet<>()); sessions.add(sessionUUID); } Store.MetadataSnapshot metaData = restore.getMetaData(); success = true; return metaData; } finally { - if (success == false) { + if (success == false) { onGoingRestores.remove(sessionUUID); - IOUtils.closeWhileHandlingException(restore); + if (restore != null) { + restore.decRef(); + } } } } public synchronized void closeSession(String sessionUUID) { closeSessionListeners.forEach(c -> c.accept(sessionUUID)); - RestoreContext restore = onGoingRestores.remove(sessionUUID); + RestoreSession restore = onGoingRestores.remove(sessionUUID); if (restore == null) { logger.info("could not close session [{}] because session not found", sessionUUID); throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); } - IOUtils.closeWhileHandlingException(restore); + restore.decRef(); } - public synchronized Reader getSessionReader(String sessionUUID, String fileName) throws IOException { - RestoreContext restore = onGoingRestores.get(sessionUUID); + public synchronized RestoreSession getRestoreSession(String sessionUUID) { + RestoreSession restore = onGoingRestores.get(sessionUUID); if (restore == null) { logger.debug("could not get session [{}] because session not found", sessionUUID); throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); } - return restore.getFileReader(fileName); + restore.incRef(); + return restore; } - private class RestoreContext implements Closeable { + public class RestoreSession extends AbstractRefCounted { private final String sessionUUID; private final IndexShard indexShard; - private final RefCountedCloseable session; - private RefCountedCloseable cachedInput; - private volatile boolean sessionLocked = false; - private Releasable lockRelease = () -> sessionLocked = false; + private final Engine.IndexCommitRef commitRef; + private volatile Tuple cachedInput; - private RestoreContext(String sessionUUID, IndexShard indexShard, Engine.IndexCommitRef commitRef) { + private RestoreSession(String sessionUUID, IndexShard indexShard, Engine.IndexCommitRef commitRef) { + super("restore-session"); this.sessionUUID = sessionUUID; this.indexShard = indexShard; - this.session = new RefCountedCloseable<>("commit-ref", commitRef); + this.commitRef = commitRef; } - Store.MetadataSnapshot getMetaData() throws IOException { + private Store.MetadataSnapshot getMetaData() throws IOException { indexShard.store().incRef(); try { - return indexShard.store().getMetadata(session.object.getIndexCommit()); + return indexShard.store().getMetadata(commitRef.getIndexCommit()); } finally { indexShard.store().decRef(); } } - Reader getFileReader(String fileName) throws IOException { - if (sessionLocked) { - throw new IllegalStateException("Session is currently locked by another get file chunk request"); - } - sessionLocked = true; + public synchronized void readFileBytes(String fileName, BytesReference reference) throws IOException { + // Should not access this method while holding global lock as that might block the cluster state + // update thread on IO if it calls afterIndexShardClosed + assert Thread.holdsLock(CcrRestoreSourceService.this) == false : "Should not hold CcrRestoreSourceService lock"; if (cachedInput != null) { - if (fileName.equals(cachedInput.getName())) { - return new Reader(session, cachedInput, lockRelease); - } else { - cachedInput.decRef(); + if (fileName.equals(cachedInput.v2()) == false) { + cachedInput.v2().close(); openNewIndexInput(fileName); - return new Reader(session, cachedInput, lockRelease); } } else { openNewIndexInput(fileName); - return new Reader(session, cachedInput, lockRelease); } - } - - @Override - public void close() { - assert Thread.holdsLock(CcrRestoreSourceService.this); - removeSessionForShard(sessionUUID, indexShard); - if (cachedInput != null) { - cachedInput.decRef(); + BytesRefIterator refIterator = reference.iterator(); + BytesRef ref; + IndexInput in = cachedInput.v2(); + while ((ref = refIterator.next()) != null) { + byte[] refBytes = ref.bytes; + in.readBytes(refBytes, 0, refBytes.length); } - session.decRef(); } private void openNewIndexInput(String fileName) throws IOException { - Engine.IndexCommitRef commitRef = session.object; - IndexInput indexInput = commitRef.getIndexCommit().getDirectory().openInput(fileName, IOContext.READONCE); - cachedInput = new RefCountedCloseable<>(fileName, indexInput); - } - - private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { - logger.debug("closing session [{}] for shard [{}]", sessionUUID, indexShard.shardId()); - HashSet sessions = sessionsForShard.get(indexShard); - if (sessions != null) { - sessions.remove(sessionUUID); - if (sessions.isEmpty()) { - sessionsForShard.remove(indexShard); - } - } - } - } - - private static class RefCountedCloseable extends AbstractRefCounted { - - private final T object; - - private RefCountedCloseable(String name, T object) { - super(name); - this.object = object; + IndexInput in = commitRef.getIndexCommit().getDirectory().openInput(fileName, IOContext.READONCE); + cachedInput = new Tuple<>(fileName, in); } @Override protected void closeInternal() { - IOUtils.closeWhileHandlingException(object); - } - } - - public static class Reader implements Closeable { - - private final RefCountedCloseable session; - private final RefCountedCloseable input; - private final Releasable lockRelease; - - private Reader(RefCountedCloseable session, RefCountedCloseable input, Releasable lockRelease) { - this.session = session; - this.input = input; - this.lockRelease = lockRelease; - boolean sessionRefIncremented = false; - boolean inputRefIncremented = false; try { - session.incRef(); - sessionRefIncremented = true; - input.incRef(); - inputRefIncremented = true; + removeSessionForShard(sessionUUID, indexShard); } finally { - if (sessionRefIncremented == false) { - IOUtils.closeWhileHandlingException(lockRelease); - } else if (inputRefIncremented == false) { - IOUtils.closeWhileHandlingException(session::decRef, lockRelease); + if (cachedInput != null) { + IOUtils.closeWhileHandlingException(cachedInput.v2()); } } } - public int readFileBytes(BytesReference reference) throws IOException { - IndexInput in = input.object; - BytesRefIterator refIterator = reference.iterator(); - BytesRef ref; - int bytesRead = 0; - while ((ref = refIterator.next()) != null) { - byte[] refBytes = ref.bytes; - in.readBytes(refBytes, 0, refBytes.length); - bytesRead += ref.length; + private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { + synchronized (CcrRestoreSourceService.this) { + logger.debug("closing session [{}] for shard [{}]", sessionUUID, indexShard.shardId()); + HashSet sessions = sessionsForShard.get(indexShard); + if (sessions != null) { + sessions.remove(sessionUUID); + if (sessions.isEmpty()) { + sessionsForShard.remove(indexShard); + } + } } - return bytesRead; - } - - @Override - public void close() { - IOUtils.closeWhileHandlingException(input::decRef, session::decRef, lockRelease); } } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java index d0b0dafdc278a..68e56667b65fa 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java @@ -150,8 +150,11 @@ public void testGetSessionReader() throws IOException { } BytesArray byteArray = new BytesArray(actualBytes); - try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID1, fileName)) { - reader.readFileBytes(byteArray); + CcrRestoreSourceService.RestoreSession session = restoreSourceService.getRestoreSession(sessionUUID1); + try { + session.readFileBytes(fileName, byteArray); + } finally { + session.decRef(); } assertArrayEquals(expectedBytes, actualBytes); @@ -172,40 +175,23 @@ public void testGetSessionDoesNotLeakFileIfClosed() throws IOException { ArrayList files = new ArrayList<>(); indexShard.snapshotStoreMetadata().forEach(files::add); - try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID1, files.get(0).name())) { - // Using try with close to ensure that reader is closed. - assertNotNull(reader); + CcrRestoreSourceService.RestoreSession session = restoreSourceService.getRestoreSession(sessionUUID1); + try { + session.readFileBytes(files.get(0).name(), new BytesArray(new byte[10])); + } finally { + session.decRef(); } // Request a second file to ensure that original file is not leaked - try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID1, files.get(1).name())) { - // Using try with close to ensure that reader is closed. - assertNotNull(reader); + session = restoreSourceService.getRestoreSession(sessionUUID1); + try { + session.readFileBytes(files.get(0).name(), new BytesArray(new byte[10])); + } finally { + session.decRef(); } restoreSourceService.closeSession(sessionUUID1); closeShards(indexShard); // Exception will be thrown if file is not closed. } - - public void testCannotAccessTwoSessionReadersAtOnce() throws IOException { - IndexShard indexShard = newStartedShard(true); - final String sessionUUID1 = UUIDs.randomBase64UUID(); - - restoreSourceService.openSession(sessionUUID1, indexShard); - - ArrayList files = new ArrayList<>(); - indexShard.snapshotStoreMetadata().forEach(files::add); - - StoreFileMetaData fileMetaData = files.get(0); - String fileName = fileMetaData.name(); - - indexShard.snapshotStoreMetadata().forEach(files::add); - try (CcrRestoreSourceService.Reader reader = restoreSourceService.getSessionReader(sessionUUID1, fileName)) { - expectThrows(IllegalStateException.class, () -> restoreSourceService.getSessionReader(sessionUUID1, fileName)); - } - - restoreSourceService.closeSession(sessionUUID1); - closeShards(indexShard); - } } From cf35fcd8ca61d74aea4357d29fdc4ae53afdf2a5 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 10 Jan 2019 19:03:05 -0700 Subject: [PATCH 26/30] Changes --- .../GetCcrRestoreFileChunkAction.java | 7 ++--- .../repository/CcrRestoreSourceService.java | 29 +++++++++++++++---- .../CcrRestoreSourceServiceTests.java | 22 +++++--------- 3 files changed, 33 insertions(+), 25 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index 76559af9fd0ef..b75ad63a4c9cd 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -83,12 +83,9 @@ protected void doRun() throws Exception { // This is currently safe to do because calling `onResponse` will serialize the bytes to the network layer data // structure on the same thread. So the bytes will be copied before the reference is released. try (ReleasablePagedBytesReference reference = new ReleasablePagedBytesReference(array, bytesRequested, array)) { - CcrRestoreSourceService.RestoreSession session = restoreSourceService.getRestoreSession(sessionUUID); - try { - session.readFileBytes(fileName, reference); + try (CcrRestoreSourceService.SessionReader sessionReader = restoreSourceService.getSessionReader(sessionUUID)) { + sessionReader.readFileBytes(fileName, reference); listener.onResponse(new GetCcrRestoreFileChunkResponse(reference)); - } finally { - session.decRef(); } } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index c3cbac8c02315..ad2fb2a4e2c7b 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -28,6 +28,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; +import java.io.Closeable; import java.io.IOException; import java.util.HashMap; import java.util.HashSet; @@ -141,17 +142,16 @@ public synchronized void closeSession(String sessionUUID) { restore.decRef(); } - public synchronized RestoreSession getRestoreSession(String sessionUUID) { + public synchronized SessionReader getSessionReader(String sessionUUID) { RestoreSession restore = onGoingRestores.get(sessionUUID); if (restore == null) { logger.debug("could not get session [{}] because session not found", sessionUUID); throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); } - restore.incRef(); - return restore; + return new SessionReader(restore); } - public class RestoreSession extends AbstractRefCounted { + private class RestoreSession extends AbstractRefCounted { private final String sessionUUID; private final IndexShard indexShard; @@ -174,7 +174,7 @@ private Store.MetadataSnapshot getMetaData() throws IOException { } } - public synchronized void readFileBytes(String fileName, BytesReference reference) throws IOException { + private synchronized void readFileBytes(String fileName, BytesReference reference) throws IOException { // Should not access this method while holding global lock as that might block the cluster state // update thread on IO if it calls afterIndexShardClosed assert Thread.holdsLock(CcrRestoreSourceService.this) == false : "Should not hold CcrRestoreSourceService lock"; @@ -224,4 +224,23 @@ private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { } } } + + public static class SessionReader implements Closeable { + + private final RestoreSession restoreSession; + + private SessionReader(RestoreSession restoreSession) { + this.restoreSession = restoreSession; + restoreSession.incRef(); + } + + public synchronized void readFileBytes(String fileName, BytesReference reference) throws IOException { + restoreSession.readFileBytes(fileName, reference); + } + + @Override + public void close() { + restoreSession.decRef(); + } + } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java index 68e56667b65fa..1b228592adb49 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java @@ -150,11 +150,8 @@ public void testGetSessionReader() throws IOException { } BytesArray byteArray = new BytesArray(actualBytes); - CcrRestoreSourceService.RestoreSession session = restoreSourceService.getRestoreSession(sessionUUID1); - try { - session.readFileBytes(fileName, byteArray); - } finally { - session.decRef(); + try (CcrRestoreSourceService.SessionReader sessionReader = restoreSourceService.getSessionReader(sessionUUID1)) { + sessionReader.readFileBytes(fileName, byteArray); } assertArrayEquals(expectedBytes, actualBytes); @@ -175,19 +172,14 @@ public void testGetSessionDoesNotLeakFileIfClosed() throws IOException { ArrayList files = new ArrayList<>(); indexShard.snapshotStoreMetadata().forEach(files::add); - CcrRestoreSourceService.RestoreSession session = restoreSourceService.getRestoreSession(sessionUUID1); - try { - session.readFileBytes(files.get(0).name(), new BytesArray(new byte[10])); - } finally { - session.decRef(); + + try (CcrRestoreSourceService.SessionReader sessionReader = restoreSourceService.getSessionReader(sessionUUID1)) { + sessionReader.readFileBytes(files.get(0).name(), new BytesArray(new byte[10])); } // Request a second file to ensure that original file is not leaked - session = restoreSourceService.getRestoreSession(sessionUUID1); - try { - session.readFileBytes(files.get(0).name(), new BytesArray(new byte[10])); - } finally { - session.decRef(); + try (CcrRestoreSourceService.SessionReader sessionReader = restoreSourceService.getSessionReader(sessionUUID1)) { + sessionReader.readFileBytes(files.get(1).name(), new BytesArray(new byte[10])); } restoreSourceService.closeSession(sessionUUID1); From fa59f494bba93aa2e99cde683c870e5393a5c197 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 11 Jan 2019 19:07:52 +0100 Subject: [PATCH 27/30] no recursive dependencies --- .../repository/CcrRestoreSourceService.java | 52 +++++++++---------- 1 file changed, 24 insertions(+), 28 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index ad2fb2a4e2c7b..8562ffbf091d1 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -56,6 +56,7 @@ public synchronized void afterIndexShardClosed(ShardId shardId, @Nullable IndexS if (sessions != null) { for (String sessionUUID : sessions) { RestoreSession restore = onGoingRestores.remove(sessionUUID); + assert restore != null; restore.decRef(); } } @@ -132,12 +133,24 @@ public synchronized Store.MetadataSnapshot openSession(String sessionUUID, Index } } - public synchronized void closeSession(String sessionUUID) { - closeSessionListeners.forEach(c -> c.accept(sessionUUID)); - RestoreSession restore = onGoingRestores.remove(sessionUUID); - if (restore == null) { - logger.info("could not close session [{}] because session not found", sessionUUID); - throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); + public void closeSession(String sessionUUID) { + final RestoreSession restore; + synchronized (this) { + closeSessionListeners.forEach(c -> c.accept(sessionUUID)); + restore = onGoingRestores.remove(sessionUUID); + if (restore == null) { + logger.debug("could not close session [{}] because session not found", sessionUUID); + throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); + } + HashSet sessions = sessionsForShard.get(restore.indexShard); + assert sessions != null; + if (sessions != null) { + boolean removed = sessions.remove(sessionUUID); + assert removed; + if (sessions.isEmpty()) { + sessionsForShard.remove(restore.indexShard); + } + } } restore.decRef(); } @@ -151,7 +164,7 @@ public synchronized SessionReader getSessionReader(String sessionUUID) { return new SessionReader(restore); } - private class RestoreSession extends AbstractRefCounted { + private static class RestoreSession extends AbstractRefCounted { private final String sessionUUID; private final IndexShard indexShard; @@ -177,7 +190,6 @@ private Store.MetadataSnapshot getMetaData() throws IOException { private synchronized void readFileBytes(String fileName, BytesReference reference) throws IOException { // Should not access this method while holding global lock as that might block the cluster state // update thread on IO if it calls afterIndexShardClosed - assert Thread.holdsLock(CcrRestoreSourceService.this) == false : "Should not hold CcrRestoreSourceService lock"; if (cachedInput != null) { if (fileName.equals(cachedInput.v2()) == false) { cachedInput.v2().close(); @@ -202,25 +214,9 @@ private void openNewIndexInput(String fileName) throws IOException { @Override protected void closeInternal() { - try { - removeSessionForShard(sessionUUID, indexShard); - } finally { - if (cachedInput != null) { - IOUtils.closeWhileHandlingException(cachedInput.v2()); - } - } - } - - private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { - synchronized (CcrRestoreSourceService.this) { - logger.debug("closing session [{}] for shard [{}]", sessionUUID, indexShard.shardId()); - HashSet sessions = sessionsForShard.get(indexShard); - if (sessions != null) { - sessions.remove(sessionUUID); - if (sessions.isEmpty()) { - sessionsForShard.remove(indexShard); - } - } + logger.debug("closing session [{}] for shard [{}]", sessionUUID, indexShard.shardId()); + if (cachedInput != null) { + IOUtils.closeWhileHandlingException(cachedInput.v2()); } } } @@ -234,7 +230,7 @@ private SessionReader(RestoreSession restoreSession) { restoreSession.incRef(); } - public synchronized void readFileBytes(String fileName, BytesReference reference) throws IOException { + public void readFileBytes(String fileName, BytesReference reference) throws IOException { restoreSession.readFileBytes(fileName, reference); } From d8723349327a5ca4f3cd7f3b0a45a3f6edf4d72e Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 11 Jan 2019 19:07:07 -0700 Subject: [PATCH 28/30] Changes --- .../GetCcrRestoreFileChunkAction.java | 15 +++- .../xpack/ccr/repository/CcrRepository.java | 11 +-- .../repository/CcrRestoreSourceService.java | 80 ++++++++++++------- .../CcrRestoreSourceServiceTests.java | 3 +- 4 files changed, 69 insertions(+), 40 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index b75ad63a4c9cd..6f777d1b55d17 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -84,8 +84,8 @@ protected void doRun() throws Exception { // structure on the same thread. So the bytes will be copied before the reference is released. try (ReleasablePagedBytesReference reference = new ReleasablePagedBytesReference(array, bytesRequested, array)) { try (CcrRestoreSourceService.SessionReader sessionReader = restoreSourceService.getSessionReader(sessionUUID)) { - sessionReader.readFileBytes(fileName, reference); - listener.onResponse(new GetCcrRestoreFileChunkResponse(reference)); + long offsetAfterRead = sessionReader.readFileBytes(fileName, reference); + listener.onResponse(new GetCcrRestoreFileChunkResponse(offsetAfterRead, reference)); } } } @@ -95,17 +95,24 @@ protected void doRun() throws Exception { public static class GetCcrRestoreFileChunkResponse extends ActionResponse { + private final long offset; private final BytesReference chunk; GetCcrRestoreFileChunkResponse(StreamInput streamInput) throws IOException { super(streamInput); + offset = streamInput.readVLong(); chunk = streamInput.readBytesReference(); } - GetCcrRestoreFileChunkResponse(BytesReference chunk) { + GetCcrRestoreFileChunkResponse(long offset, BytesReference chunk) { + this.offset = offset; this.chunk = chunk; } + public long getOffset() { + return offset; + } + public BytesReference getChunk() { return chunk; } @@ -113,7 +120,9 @@ public BytesReference getChunk() { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); + out.writeVLong(offset); out.writeBytesReference(chunk); } + } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 3e49224fe87fe..68d4b89d7e6a7 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -6,8 +6,6 @@ package org.elasticsearch.xpack.ccr.repository; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.apache.lucene.index.IndexCommit; import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; @@ -80,8 +78,6 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit public static final String NAME_PREFIX = "_ccr_"; private static final SnapshotId SNAPSHOT_ID = new SnapshotId(LATEST, LATEST); - private static final Logger logger = LogManager.getLogger(CcrRepository.class); - private final RepositoryMetaData metadata; private final String remoteClusterAlias; private final Client client; @@ -372,7 +368,9 @@ public int read(byte[] bytes, int off, int len) throws IOException { int bytesRequested = (int) Math.min(remainingBytes, len); String fileName = fileToRecover.name(); GetCcrRestoreFileChunkRequest request = new GetCcrRestoreFileChunkRequest(node, sessionUUID, fileName, bytesRequested); - BytesReference fileChunk = remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request).actionGet().getChunk(); + GetCcrRestoreFileChunkAction.GetCcrRestoreFileChunkResponse response = + remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request).actionGet(); + BytesReference fileChunk = response.getChunk(); int bytesReceived = fileChunk.length(); if (bytesReceived > bytesRequested) { @@ -386,6 +384,9 @@ public int read(byte[] bytes, int off, int len) throws IOException { pos += bytesReceived; + long leaderOffset = response.getOffset(); + assert pos == leaderOffset : "Position [" + pos + "] should be equal to the leader file offset [" + leaderOffset + "]."; + return bytesReceived; } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index 8562ffbf091d1..e59650babf8f2 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -14,11 +14,12 @@ import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.KeyedLock; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexEventListener; @@ -30,9 +31,11 @@ import java.io.Closeable; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.function.Consumer; @@ -56,7 +59,7 @@ public synchronized void afterIndexShardClosed(ShardId shardId, @Nullable IndexS if (sessions != null) { for (String sessionUUID : sessions) { RestoreSession restore = onGoingRestores.remove(sessionUUID); - assert restore != null; + assert restore != null : "Restore registered for shard but not found in ongoing restores"; restore.decRef(); } } @@ -143,10 +146,10 @@ public void closeSession(String sessionUUID) { throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); } HashSet sessions = sessionsForShard.get(restore.indexShard); - assert sessions != null; + assert sessions != null : "No session UUIDs for shard even though once is active in ongoing restores"; if (sessions != null) { boolean removed = sessions.remove(sessionUUID); - assert removed; + assert removed : "No session found for UUID"; if (sessions.isEmpty()) { sessionsForShard.remove(restore.indexShard); } @@ -169,7 +172,8 @@ private static class RestoreSession extends AbstractRefCounted { private final String sessionUUID; private final IndexShard indexShard; private final Engine.IndexCommitRef commitRef; - private volatile Tuple cachedInput; + private final KeyedLock keyedLock = new KeyedLock<>(); + private final Map cachedInputs = new ConcurrentHashMap<>(); private RestoreSession(String sessionUUID, IndexShard indexShard, Engine.IndexCommitRef commitRef) { super("restore-session"); @@ -187,37 +191,42 @@ private Store.MetadataSnapshot getMetaData() throws IOException { } } - private synchronized void readFileBytes(String fileName, BytesReference reference) throws IOException { - // Should not access this method while holding global lock as that might block the cluster state - // update thread on IO if it calls afterIndexShardClosed - if (cachedInput != null) { - if (fileName.equals(cachedInput.v2()) == false) { - cachedInput.v2().close(); - openNewIndexInput(fileName); - } - } else { - openNewIndexInput(fileName); - } - BytesRefIterator refIterator = reference.iterator(); - BytesRef ref; - IndexInput in = cachedInput.v2(); - while ((ref = refIterator.next()) != null) { - byte[] refBytes = ref.bytes; - in.readBytes(refBytes, 0, refBytes.length); + private long readFileBytes(String fileName, BytesReference reference) throws IOException { + Releasable lock = keyedLock.tryAcquire(fileName); + if (lock == null) { + throw new IllegalStateException("can't read from the same file on the same session concurrently"); } - } + try (Releasable releasable = lock) { + final IndexInput indexInput = cachedInputs.computeIfAbsent(fileName, f -> { + try { + return commitRef.getIndexCommit().getDirectory().openInput(fileName, IOContext.READONCE); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + + BytesRefIterator refIterator = reference.iterator(); + BytesRef ref; + while ((ref = refIterator.next()) != null) { + byte[] refBytes = ref.bytes; + indexInput.readBytes(refBytes, 0, refBytes.length); + } - private void openNewIndexInput(String fileName) throws IOException { - IndexInput in = commitRef.getIndexCommit().getDirectory().openInput(fileName, IOContext.READONCE); - cachedInput = new Tuple<>(fileName, in); + long offsetAfterRead = indexInput.getFilePointer(); + + if (offsetAfterRead == indexInput.length()) { + cachedInputs.remove(fileName); + IOUtils.closeWhileHandlingException(indexInput); + } + + return offsetAfterRead; + } } @Override protected void closeInternal() { logger.debug("closing session [{}] for shard [{}]", sessionUUID, indexShard.shardId()); - if (cachedInput != null) { - IOUtils.closeWhileHandlingException(cachedInput.v2()); - } + IOUtils.closeWhileHandlingException(cachedInputs.values()); } } @@ -230,8 +239,17 @@ private SessionReader(RestoreSession restoreSession) { restoreSession.incRef(); } - public void readFileBytes(String fileName, BytesReference reference) throws IOException { - restoreSession.readFileBytes(fileName, reference); + /** + * Read bytes into the reference from the file. This method will return the offset in the file where + * the read completed. + * + * @param fileName to read + * @param reference to read bytes into + * @return the offset of the file after the read is complete + * @throws IOException if the read fails + */ + public long readFileBytes(String fileName, BytesReference reference) throws IOException { + return restoreSession.readFileBytes(fileName, reference); } @Override diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java index 1b228592adb49..efcd93e90fd07 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java @@ -151,7 +151,8 @@ public void testGetSessionReader() throws IOException { BytesArray byteArray = new BytesArray(actualBytes); try (CcrRestoreSourceService.SessionReader sessionReader = restoreSourceService.getSessionReader(sessionUUID1)) { - sessionReader.readFileBytes(fileName, byteArray); + long offset = sessionReader.readFileBytes(fileName, byteArray); + assertEquals(offset, fileMetaData.length()); } assertArrayEquals(expectedBytes, actualBytes); From 92435b088b08c773ce55377462e7399674c56bbc Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Sat, 12 Jan 2019 10:32:59 -0700 Subject: [PATCH 29/30] Don't swallow --- .../xpack/ccr/repository/CcrRestoreSourceService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index e59650babf8f2..5bf97465fb1d7 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -216,7 +216,7 @@ private long readFileBytes(String fileName, BytesReference reference) throws IOE if (offsetAfterRead == indexInput.length()) { cachedInputs.remove(fileName); - IOUtils.closeWhileHandlingException(indexInput); + IOUtils.close(indexInput); } return offsetAfterRead; From 2bfa3e1c3e8380d606b803015d994aa34b00e32e Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 14 Jan 2019 09:38:31 -0700 Subject: [PATCH 30/30] Changes --- .../action/repositories/GetCcrRestoreFileChunkAction.java | 3 ++- .../action/repositories/GetCcrRestoreFileChunkRequest.java | 1 + .../elasticsearch/xpack/ccr/repository/CcrRepository.java | 6 +++--- .../xpack/ccr/repository/CcrRestoreSourceService.java | 7 ++++--- 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java index 6f777d1b55d17..3f473f25c2411 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -85,7 +85,8 @@ protected void doRun() throws Exception { try (ReleasablePagedBytesReference reference = new ReleasablePagedBytesReference(array, bytesRequested, array)) { try (CcrRestoreSourceService.SessionReader sessionReader = restoreSourceService.getSessionReader(sessionUUID)) { long offsetAfterRead = sessionReader.readFileBytes(fileName, reference); - listener.onResponse(new GetCcrRestoreFileChunkResponse(offsetAfterRead, reference)); + long offsetBeforeRead = offsetAfterRead - reference.length(); + listener.onResponse(new GetCcrRestoreFileChunkResponse(offsetBeforeRead, reference)); } } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java index 25b35964c6c90..5da0efcb372f5 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java @@ -70,6 +70,7 @@ int getSize() { @Override public DiscoveryNode getPreferredTargetNode() { + assert node != null : "Target node is null"; return node; } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 68d4b89d7e6a7..025892f80d834 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -377,6 +377,9 @@ public int read(byte[] bytes, int off, int len) throws IOException { throw new IOException("More bytes [" + bytesReceived + "] received than requested [" + bytesRequested + "]"); } + long leaderOffset = response.getOffset(); + assert pos == leaderOffset : "Position [" + pos + "] should be equal to the leader file offset [" + leaderOffset + "]."; + try (StreamInput streamInput = fileChunk.streamInput()) { int bytesRead = streamInput.read(bytes, 0, bytesReceived); assert bytesRead == bytesReceived : "Did not read the correct number of bytes"; @@ -384,9 +387,6 @@ public int read(byte[] bytes, int off, int len) throws IOException { pos += bytesReceived; - long leaderOffset = response.getOffset(); - assert pos == leaderOffset : "Position [" + pos + "] should be equal to the leader file offset [" + leaderOffset + "]."; - return bytesReceived; } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index 5bf97465fb1d7..197d5ddbf38ff 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -59,7 +59,7 @@ public synchronized void afterIndexShardClosed(ShardId shardId, @Nullable IndexS if (sessions != null) { for (String sessionUUID : sessions) { RestoreSession restore = onGoingRestores.remove(sessionUUID); - assert restore != null : "Restore registered for shard but not found in ongoing restores"; + assert restore != null : "Session UUID [" + sessionUUID + "] registered for shard but not found in ongoing restores"; restore.decRef(); } } @@ -146,10 +146,10 @@ public void closeSession(String sessionUUID) { throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); } HashSet sessions = sessionsForShard.get(restore.indexShard); - assert sessions != null : "No session UUIDs for shard even though once is active in ongoing restores"; + assert sessions != null : "No session UUIDs for shard even though one [" + sessionUUID + "] is active in ongoing restores"; if (sessions != null) { boolean removed = sessions.remove(sessionUUID); - assert removed : "No session found for UUID"; + assert removed : "No session found for UUID [" + sessionUUID +"]"; if (sessions.isEmpty()) { sessionsForShard.remove(restore.indexShard); } @@ -226,6 +226,7 @@ private long readFileBytes(String fileName, BytesReference reference) throws IOE @Override protected void closeInternal() { logger.debug("closing session [{}] for shard [{}]", sessionUUID, indexShard.shardId()); + assert keyedLock.hasLockedKeys() == false : "Should not hold any file locks when closing"; IOUtils.closeWhileHandlingException(cachedInputs.values()); } }