|
@@ -291,26 +291,29 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) {
|
|
|
- // TODO: Add timeouts to network calls / the restore process.
|
|
|
- createEmptyStore(store);
|
|
|
- ShardId shardId = store.shardId();
|
|
|
-
|
|
|
- final Map<String, String> ccrMetaData = store.indexSettings().getIndexMetaData().getCustomData(Ccr.CCR_CUSTOM_METADATA_KEY);
|
|
|
- final String leaderIndexName = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_NAME_KEY);
|
|
|
- final String leaderUUID = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_UUID_KEY);
|
|
|
- final Index leaderIndex = new Index(leaderIndexName, leaderUUID);
|
|
|
- final ShardId leaderShardId = new ShardId(leaderIndex, shardId.getId());
|
|
|
-
|
|
|
- final Client remoteClient = getRemoteClusterClient();
|
|
|
-
|
|
|
- final String retentionLeaseId =
|
|
|
+ public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState,
|
|
|
+ ActionListener<Void> listener) {
|
|
|
+ // TODO: Instead of blocking in the restore logic and synchronously completing the listener we should just make below logic async
|
|
|
+ ActionListener.completeWith(listener, () -> {
|
|
|
+ // TODO: Add timeouts to network calls / the restore process.
|
|
|
+ createEmptyStore(store);
|
|
|
+ ShardId shardId = store.shardId();
|
|
|
+
|
|
|
+ final Map<String, String> ccrMetaData = store.indexSettings().getIndexMetaData().getCustomData(Ccr.CCR_CUSTOM_METADATA_KEY);
|
|
|
+ final String leaderIndexName = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_NAME_KEY);
|
|
|
+ final String leaderUUID = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_UUID_KEY);
|
|
|
+ final Index leaderIndex = new Index(leaderIndexName, leaderUUID);
|
|
|
+ final ShardId leaderShardId = new ShardId(leaderIndex, shardId.getId());
|
|
|
+
|
|
|
+ final Client remoteClient = getRemoteClusterClient();
|
|
|
+
|
|
|
+ final String retentionLeaseId =
|
|
|
retentionLeaseId(localClusterName, shardId.getIndex(), remoteClusterAlias, leaderIndex);
|
|
|
|
|
|
- acquireRetentionLeaseOnLeader(shardId, retentionLeaseId, leaderShardId, remoteClient);
|
|
|
+ acquireRetentionLeaseOnLeader(shardId, retentionLeaseId, leaderShardId, remoteClient);
|
|
|
|
|
|
- // schedule renewals to run during the restore
|
|
|
- final Scheduler.Cancellable renewable = threadPool.scheduleWithFixedDelay(
|
|
|
+ // schedule renewals to run during the restore
|
|
|
+ final Scheduler.Cancellable renewable = threadPool.scheduleWithFixedDelay(
|
|
|
() -> {
|
|
|
logger.trace("{} background renewal of retention lease [{}] during restore", shardId, retentionLeaseId);
|
|
|
final ThreadContext threadContext = threadPool.getThreadContext();
|
|
@@ -318,38 +321,40 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
|
|
|
// we have to execute under the system context so that if security is enabled the renewal is authorized
|
|
|
threadContext.markAsSystemContext();
|
|
|
CcrRetentionLeases.asyncRenewRetentionLease(
|
|
|
- leaderShardId,
|
|
|
- retentionLeaseId,
|
|
|
- RETAIN_ALL,
|
|
|
- remoteClient,
|
|
|
- ActionListener.wrap(
|
|
|
- r -> {},
|
|
|
- e -> {
|
|
|
- final Throwable cause = ExceptionsHelper.unwrapCause(e);
|
|
|
- assert cause instanceof ElasticsearchSecurityException == false : cause;
|
|
|
- if (cause instanceof RetentionLeaseInvalidRetainingSeqNoException == false) {
|
|
|
- logger.warn(new ParameterizedMessage(
|
|
|
- "{} background renewal of retention lease [{}] failed during restore", shardId,
|
|
|
- retentionLeaseId), cause);
|
|
|
- }
|
|
|
- }));
|
|
|
+ leaderShardId,
|
|
|
+ retentionLeaseId,
|
|
|
+ RETAIN_ALL,
|
|
|
+ remoteClient,
|
|
|
+ ActionListener.wrap(
|
|
|
+ r -> {},
|
|
|
+ e -> {
|
|
|
+ final Throwable cause = ExceptionsHelper.unwrapCause(e);
|
|
|
+ assert cause instanceof ElasticsearchSecurityException == false : cause;
|
|
|
+ if (cause instanceof RetentionLeaseInvalidRetainingSeqNoException == false) {
|
|
|
+ logger.warn(new ParameterizedMessage(
|
|
|
+ "{} background renewal of retention lease [{}] failed during restore", shardId,
|
|
|
+ retentionLeaseId), cause);
|
|
|
+ }
|
|
|
+ }));
|
|
|
}
|
|
|
},
|
|
|
CcrRetentionLeases.RETENTION_LEASE_RENEW_INTERVAL_SETTING.get(store.indexSettings().getNodeSettings()),
|
|
|
Ccr.CCR_THREAD_POOL_NAME);
|
|
|
|
|
|
- // 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 = openSession(metadata.name(), remoteClient, leaderShardId, shardId, recoveryState)) {
|
|
|
- restoreSession.restoreFiles(store);
|
|
|
- updateMappings(remoteClient, leaderIndex, restoreSession.mappingVersion, client, shardId.getIndex());
|
|
|
- } catch (Exception e) {
|
|
|
- throw new IndexShardRestoreFailedException(shardId, "failed to restore snapshot [" + snapshotId + "]", e);
|
|
|
- } finally {
|
|
|
- logger.trace("{} canceling background renewal of retention lease [{}] at the end of restore", shardId,
|
|
|
- retentionLeaseId);
|
|
|
- renewable.cancel();
|
|
|
- }
|
|
|
+ // 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 = openSession(metadata.name(), remoteClient, leaderShardId, shardId, recoveryState)) {
|
|
|
+ restoreSession.restoreFiles(store);
|
|
|
+ updateMappings(remoteClient, leaderIndex, restoreSession.mappingVersion, client, shardId.getIndex());
|
|
|
+ } catch (Exception e) {
|
|
|
+ throw new IndexShardRestoreFailedException(shardId, "failed to restore snapshot [" + snapshotId + "]", e);
|
|
|
+ } finally {
|
|
|
+ logger.trace("{} canceling background renewal of retention lease [{}] at the end of restore", shardId,
|
|
|
+ retentionLeaseId);
|
|
|
+ renewable.cancel();
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ });
|
|
|
}
|
|
|
|
|
|
private void createEmptyStore(Store store) {
|
|
@@ -465,86 +470,92 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
|
|
|
fileInfos.add(new FileInfo(fileMetaData.name(), fileMetaData, fileSize));
|
|
|
}
|
|
|
SnapshotFiles snapshotFiles = new SnapshotFiles(LATEST, fileInfos);
|
|
|
- restore(snapshotFiles, store);
|
|
|
+ final PlainActionFuture<Void> future = PlainActionFuture.newFuture();
|
|
|
+ restore(snapshotFiles, store, future);
|
|
|
+ future.actionGet();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- protected void restoreFiles(List<FileInfo> filesToRecover, Store store) {
|
|
|
- logger.trace("[{}] starting CCR restore of {} files", shardId, filesToRecover);
|
|
|
- final PlainActionFuture<Void> restoreFilesFuture = new PlainActionFuture<>();
|
|
|
- final List<StoreFileMetaData> mds = filesToRecover.stream().map(FileInfo::metadata).collect(Collectors.toList());
|
|
|
- final MultiFileTransfer<FileChunk> multiFileTransfer = new MultiFileTransfer<>(
|
|
|
- logger, threadPool.getThreadContext(), restoreFilesFuture, ccrSettings.getMaxConcurrentFileChunks(), mds) {
|
|
|
-
|
|
|
- final MultiFileWriter multiFileWriter = new MultiFileWriter(store, recoveryState.getIndex(), "", logger, () -> {});
|
|
|
- long offset = 0;
|
|
|
-
|
|
|
- @Override
|
|
|
- protected void onNewFile(StoreFileMetaData md) {
|
|
|
- offset = 0;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- protected FileChunk nextChunkRequest(StoreFileMetaData md) {
|
|
|
- final int bytesRequested = Math.toIntExact(Math.min(ccrSettings.getChunkSize().getBytes(), md.length() - offset));
|
|
|
- offset += bytesRequested;
|
|
|
- return new FileChunk(md, bytesRequested, offset == md.length());
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- protected void executeChunkRequest(FileChunk request, ActionListener<Void> listener) {
|
|
|
- final ActionListener<GetCcrRestoreFileChunkAction.GetCcrRestoreFileChunkResponse> threadedListener
|
|
|
- = new ThreadedActionListener<>(logger, threadPool, ThreadPool.Names.GENERIC, ActionListener.wrap(
|
|
|
+ protected void restoreFiles(List<FileInfo> filesToRecover, Store store, ActionListener<Void> listener) {
|
|
|
+ ActionListener.completeWith(listener, () -> {
|
|
|
+ logger.trace("[{}] starting CCR restore of {} files", shardId, filesToRecover);
|
|
|
+ final PlainActionFuture<Void> restoreFilesFuture = new PlainActionFuture<>();
|
|
|
+ final List<StoreFileMetaData> mds = filesToRecover.stream().map(FileInfo::metadata).collect(Collectors.toList());
|
|
|
+ final MultiFileTransfer<FileChunk> multiFileTransfer = new MultiFileTransfer<>(
|
|
|
+ logger, threadPool.getThreadContext(), restoreFilesFuture, ccrSettings.getMaxConcurrentFileChunks(), mds) {
|
|
|
+
|
|
|
+ final MultiFileWriter multiFileWriter = new MultiFileWriter(store, recoveryState.getIndex(), "", logger, () -> {
|
|
|
+ });
|
|
|
+ long offset = 0;
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void onNewFile(StoreFileMetaData md) {
|
|
|
+ offset = 0;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected FileChunk nextChunkRequest(StoreFileMetaData md) {
|
|
|
+ final int bytesRequested = Math.toIntExact(Math.min(ccrSettings.getChunkSize().getBytes(), md.length() - offset));
|
|
|
+ offset += bytesRequested;
|
|
|
+ return new FileChunk(md, bytesRequested, offset == md.length());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void executeChunkRequest(FileChunk request, ActionListener<Void> listener) {
|
|
|
+ final ActionListener<GetCcrRestoreFileChunkAction.GetCcrRestoreFileChunkResponse> threadedListener
|
|
|
+ = new ThreadedActionListener<>(logger, threadPool, ThreadPool.Names.GENERIC, ActionListener.wrap(
|
|
|
r -> {
|
|
|
writeFileChunk(request.md, r);
|
|
|
listener.onResponse(null);
|
|
|
}, listener::onFailure), false);
|
|
|
|
|
|
- remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE,
|
|
|
- new GetCcrRestoreFileChunkRequest(node, sessionUUID, request.md.name(), request.bytesRequested),
|
|
|
- ListenerTimeouts.wrapWithTimeout(threadPool, threadedListener, ccrSettings.getRecoveryActionTimeout(),
|
|
|
- ThreadPool.Names.GENERIC, GetCcrRestoreFileChunkAction.NAME));
|
|
|
- }
|
|
|
-
|
|
|
- private void writeFileChunk(StoreFileMetaData md,
|
|
|
- GetCcrRestoreFileChunkAction.GetCcrRestoreFileChunkResponse r) throws Exception {
|
|
|
- final int actualChunkSize = r.getChunk().length();
|
|
|
- logger.trace("[{}] [{}] got response for file [{}], offset: {}, length: {}",
|
|
|
- shardId, snapshotId, md.name(), r.getOffset(), actualChunkSize);
|
|
|
- final long nanosPaused = ccrSettings.getRateLimiter().maybePause(actualChunkSize);
|
|
|
- throttleListener.accept(nanosPaused);
|
|
|
- multiFileWriter.incRef();
|
|
|
- try (Releasable ignored = multiFileWriter::decRef) {
|
|
|
- final boolean lastChunk = r.getOffset() + actualChunkSize >= md.length();
|
|
|
- multiFileWriter.writeFileChunk(md, r.getOffset(), r.getChunk(), lastChunk);
|
|
|
- } catch (Exception e) {
|
|
|
- handleError(md, e);
|
|
|
- throw e;
|
|
|
+ remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE,
|
|
|
+ new GetCcrRestoreFileChunkRequest(node, sessionUUID, request.md.name(), request.bytesRequested),
|
|
|
+ ListenerTimeouts.wrapWithTimeout(threadPool, threadedListener, ccrSettings.getRecoveryActionTimeout(),
|
|
|
+ ThreadPool.Names.GENERIC, GetCcrRestoreFileChunkAction.NAME));
|
|
|
+ }
|
|
|
+
|
|
|
+ private void writeFileChunk(StoreFileMetaData md,
|
|
|
+ GetCcrRestoreFileChunkAction.GetCcrRestoreFileChunkResponse r) throws Exception {
|
|
|
+ final int actualChunkSize = r.getChunk().length();
|
|
|
+ logger.trace("[{}] [{}] got response for file [{}], offset: {}, length: {}",
|
|
|
+ shardId, snapshotId, md.name(), r.getOffset(), actualChunkSize);
|
|
|
+ final long nanosPaused = ccrSettings.getRateLimiter().maybePause(actualChunkSize);
|
|
|
+ throttleListener.accept(nanosPaused);
|
|
|
+ multiFileWriter.incRef();
|
|
|
+ try (Releasable ignored = multiFileWriter::decRef) {
|
|
|
+ final boolean lastChunk = r.getOffset() + actualChunkSize >= md.length();
|
|
|
+ multiFileWriter.writeFileChunk(md, r.getOffset(), r.getChunk(), lastChunk);
|
|
|
+ } catch (Exception e) {
|
|
|
+ handleError(md, e);
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- protected void handleError(StoreFileMetaData md, Exception e) throws Exception {
|
|
|
- final IOException corruptIndexException;
|
|
|
- if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(e)) != null) {
|
|
|
- try {
|
|
|
- store.markStoreCorrupted(corruptIndexException);
|
|
|
- } catch (IOException ioe) {
|
|
|
- logger.warn("store cannot be marked as corrupted", e);
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void handleError(StoreFileMetaData md, Exception e) throws Exception {
|
|
|
+ final IOException corruptIndexException;
|
|
|
+ if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(e)) != null) {
|
|
|
+ try {
|
|
|
+ store.markStoreCorrupted(corruptIndexException);
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ logger.warn("store cannot be marked as corrupted", e);
|
|
|
+ }
|
|
|
+ throw corruptIndexException;
|
|
|
}
|
|
|
- throw corruptIndexException;
|
|
|
+ throw e;
|
|
|
}
|
|
|
- throw e;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void close() {
|
|
|
- multiFileWriter.close();
|
|
|
- }
|
|
|
- };
|
|
|
- multiFileTransfer.start();
|
|
|
- restoreFilesFuture.actionGet();
|
|
|
- logger.trace("[{}] completed CCR restore", shardId);
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void close() {
|
|
|
+ multiFileWriter.close();
|
|
|
+ }
|
|
|
+ };
|
|
|
+ multiFileTransfer.start();
|
|
|
+ restoreFilesFuture.actionGet();
|
|
|
+ logger.trace("[{}] completed CCR restore", shardId);
|
|
|
+ return null;
|
|
|
+ });
|
|
|
}
|
|
|
|
|
|
@Override
|