public RecoveryStatus recoveryStatus() { if (recoveryStatus == null) { return recoveryStatus; } if (recoveryStatus.startTime() > 0 && recoveryStatus.stage() != RecoveryStatus.Stage.DONE) { recoveryStatus.time(System.currentTimeMillis() - recoveryStatus.startTime()); } return recoveryStatus; }
@Override public void messageReceived(RecoveryCleanFilesRequest request, TransportChannel channel) throws Exception { try (RecoveriesCollection.StatusRef statusRef = onGoingRecoveries.getStatusSafe(request.recoveryId(), request.shardId())) { final RecoveryStatus recoveryStatus = statusRef.status(); recoveryStatus.state().getTranslog().totalOperations(request.totalTranslogOps()); // first, we go and move files that were created with the recovery id suffix to // the actual names, its ok if we have a corrupted index here, since we have replicas // to recover from in case of a full cluster shutdown just when this code executes... recoveryStatus .indexShard() .deleteShardState(); // we have to delete it first since even if we fail to rename the // shard might be invalid recoveryStatus.renameAllTempFiles(); final Store store = recoveryStatus.store(); // now write checksums recoveryStatus.legacyChecksums().write(store); Store.MetadataSnapshot sourceMetaData = request.sourceMetaSnapshot(); try { store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData); } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { // this is a fatal exception at this stage. // this means we transferred files from the remote that have not be checksummed and they // are // broken. We have to clean up this shard entirely, remove all files and bubble it up to // the // source shard since this index might be broken there as well? The Source can handle this // and checks // its content on disk if possible. try { try { store.removeCorruptionMarker(); } finally { Lucene.cleanLuceneIndex(store.directory()); // clean up and delete all files } } catch (Throwable e) { logger.debug("Failed to clean lucene index", e); ex.addSuppressed(e); } RecoveryFailedException rfe = new RecoveryFailedException( recoveryStatus.state(), "failed to clean after recovery", ex); recoveryStatus.fail(rfe, true); throw rfe; } catch (Exception ex) { RecoveryFailedException rfe = new RecoveryFailedException( recoveryStatus.state(), "failed to clean after recovery", ex); recoveryStatus.fail(rfe, true); throw rfe; } channel.sendResponse(TransportResponse.Empty.INSTANCE); } }
@Override public void messageReceived(RecoveryFinalizeRecoveryRequest request, TransportChannel channel) throws Exception { try (RecoveriesCollection.StatusRef statusRef = onGoingRecoveries.getStatusSafe(request.recoveryId(), request.shardId())) { final RecoveryStatus recoveryStatus = statusRef.status(); recoveryStatus.indexShard().finalizeRecovery(); } channel.sendResponse(TransportResponse.Empty.INSTANCE); }
@Override public void messageReceived(final RecoveryFileChunkRequest request, TransportChannel channel) throws Exception { try (RecoveriesCollection.StatusRef statusRef = onGoingRecoveries.getStatusSafe(request.recoveryId(), request.shardId())) { final RecoveryStatus recoveryStatus = statusRef.status(); final Store store = recoveryStatus.store(); recoveryStatus.state().getTranslog().totalOperations(request.totalTranslogOps()); final RecoveryState.Index indexState = recoveryStatus.state().getIndex(); if (request.sourceThrottleTimeInNanos() != RecoveryState.Index.UNKNOWN) { indexState.addSourceThrottling(request.sourceThrottleTimeInNanos()); } IndexOutput indexOutput; if (request.position() == 0) { indexOutput = recoveryStatus.openAndPutIndexOutput(request.name(), request.metadata(), store); } else { indexOutput = recoveryStatus.getOpenIndexOutput(request.name()); } BytesReference content = request.content(); if (!content.hasArray()) { content = content.toBytesArray(); } RateLimiter rl = recoverySettings.rateLimiter(); if (rl != null) { long bytes = bytesSinceLastPause.addAndGet(content.length()); if (bytes > rl.getMinPauseCheckBytes()) { // Time to pause bytesSinceLastPause.addAndGet(-bytes); long throttleTimeInNanos = rl.pause(bytes); indexState.addTargetThrottling(throttleTimeInNanos); recoveryStatus.indexShard().recoveryStats().addThrottleTime(throttleTimeInNanos); } } indexOutput.writeBytes(content.array(), content.arrayOffset(), content.length()); indexState.addRecoveredBytesToFile(request.name(), content.length()); if (indexOutput.getFilePointer() >= request.length() || request.lastChunk()) { try { Store.verify(indexOutput); } finally { // we are done indexOutput.close(); } // write the checksum recoveryStatus.legacyChecksums().add(request.metadata()); final String temporaryFileName = recoveryStatus.getTempNameForFile(request.name()); assert Arrays.asList(store.directory().listAll()).contains(temporaryFileName); store.directory().sync(Collections.singleton(temporaryFileName)); IndexOutput remove = recoveryStatus.removeOpenIndexOutputs(request.name()); assert remove == null || remove == indexOutput; // remove maybe null if we got finished } } channel.sendResponse(TransportResponse.Empty.INSTANCE); }
@Override public void messageReceived( RecoveryPrepareForTranslogOperationsRequest request, TransportChannel channel) throws Exception { try (RecoveriesCollection.StatusRef statusRef = onGoingRecoveries.getStatusSafe(request.recoveryId(), request.shardId())) { final RecoveryStatus recoveryStatus = statusRef.status(); recoveryStatus.state().getTranslog().totalOperations(request.totalTranslogOps()); recoveryStatus.indexShard().skipTranslogRecovery(); } channel.sendResponse(TransportResponse.Empty.INSTANCE); }
private void retryRecovery( final RecoveryStatus recoveryStatus, TimeValue retryAfter, final StartRecoveryRequest currentRequest) { try { recoveryStatus.resetRecovery(); } catch (Throwable e) { onGoingRecoveries.failRecovery( recoveryStatus.recoveryId(), new RecoveryFailedException(currentRequest, e), true); } threadPool.schedule( retryAfter, ThreadPool.Names.GENERIC, new RecoveryRunner(recoveryStatus.recoveryId())); }
@Override public void messageReceived(RecoveryFilesInfoRequest request, TransportChannel channel) throws Exception { try (RecoveriesCollection.StatusRef statusRef = onGoingRecoveries.getStatusSafe(request.recoveryId(), request.shardId())) { final RecoveryStatus recoveryStatus = statusRef.status(); final RecoveryState.Index index = recoveryStatus.state().getIndex(); for (int i = 0; i < request.phase1ExistingFileNames.size(); i++) { index.addFileDetail( request.phase1ExistingFileNames.get(i), request.phase1ExistingFileSizes.get(i), true); } for (int i = 0; i < request.phase1FileNames.size(); i++) { index.addFileDetail( request.phase1FileNames.get(i), request.phase1FileSizes.get(i), false); } recoveryStatus.state().getTranslog().totalOperations(request.totalTranslogOps); recoveryStatus.state().getTranslog().totalOperationsOnStart(request.totalTranslogOps); // recoveryBytesCount / recoveryFileCount will be set as we go... channel.sendResponse(TransportResponse.Empty.INSTANCE); } }
protected void retryRecovery( final RecoveryStatus recoveryStatus, final String reason, TimeValue retryAfter, final StartRecoveryRequest currentRequest) { logger.trace( "will retry recovery with id [{}] in [{}] (reason [{}])", recoveryStatus.recoveryId(), retryAfter, reason); retryRecovery(recoveryStatus, retryAfter, currentRequest); }
@Override public void messageReceived( final RecoveryTranslogOperationsRequest request, final TransportChannel channel) throws Exception { try (RecoveriesCollection.StatusRef statusRef = onGoingRecoveries.getStatusSafe(request.recoveryId(), request.shardId())) { final ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger); final RecoveryStatus recoveryStatus = statusRef.status(); final RecoveryState.Translog translog = recoveryStatus.state().getTranslog(); translog.totalOperations(request.totalTranslogOps()); assert recoveryStatus.indexShard().recoveryState() == recoveryStatus.state(); try { recoveryStatus.indexShard().performBatchRecovery(request.operations()); channel.sendResponse(TransportResponse.Empty.INSTANCE); } catch (TranslogRecoveryPerformer.BatchOperationException exception) { MapperException mapperException = (MapperException) ExceptionsHelper.unwrap(exception, MapperException.class); if (mapperException == null) { throw exception; } // in very rare cases a translog replay from primary is processed before a mapping update // on this node // which causes local mapping changes. we want to wait until these mappings are processed. logger.trace( "delaying recovery due to missing mapping changes (rolling back stats for [{}] ops)", exception, exception.completedOperations()); translog.decrementRecoveredOperations(exception.completedOperations()); // we do not need to use a timeout here since the entire recovery mechanism has an // inactivity protection (it will be // canceled) observer.waitForNextChange( new ClusterStateObserver.Listener() { @Override public void onNewClusterState(ClusterState state) { try { messageReceived(request, channel); } catch (Exception e) { onFailure(e); } } protected void onFailure(Exception e) { try { channel.sendResponse(e); } catch (IOException e1) { logger.warn("failed to send error back to recovery source", e1); } } @Override public void onClusterServiceClose() { onFailure( new ElasticsearchException( "cluster service was closed while waiting for mapping updates")); } @Override public void onTimeout(TimeValue timeout) { // note that we do not use a timeout (see comment above) onFailure( new ElasticsearchTimeoutException( "timed out waiting for mapping updates (timeout [" + timeout + "])")); } }); } } }
private void doRecovery(final RecoveryStatus recoveryStatus) { assert recoveryStatus.sourceNode() != null : "can't do a recovery without a source node"; logger.trace("collecting local files for {}", recoveryStatus); Store.MetadataSnapshot metadataSnapshot = null; try { metadataSnapshot = recoveryStatus.store().getMetadataOrEmpty(); } catch (IOException e) { logger.warn("error while listing local files, recover as if there are none", e); metadataSnapshot = Store.MetadataSnapshot.EMPTY; } catch (Exception e) { // this will be logged as warning later on... logger.trace("unexpected error while listing local files, failing recovery", e); onGoingRecoveries.failRecovery( recoveryStatus.recoveryId(), new RecoveryFailedException(recoveryStatus.state(), "failed to list local files", e), true); return; } final StartRecoveryRequest request = new StartRecoveryRequest( recoveryStatus.shardId(), recoveryStatus.sourceNode(), clusterService.localNode(), false, metadataSnapshot, recoveryStatus.state().getType(), recoveryStatus.recoveryId()); final AtomicReference<RecoveryResponse> responseHolder = new AtomicReference<>(); try { logger.trace( "[{}][{}] starting recovery from {}", request.shardId().index().name(), request.shardId().id(), request.sourceNode()); recoveryStatus.indexShard().prepareForIndexRecovery(); recoveryStatus .CancellableThreads() .execute( new CancellableThreads.Interruptable() { @Override public void run() throws InterruptedException { responseHolder.set( transportService .submitRequest( request.sourceNode(), RecoverySource.Actions.START_RECOVERY, request, new FutureTransportResponseHandler<RecoveryResponse>() { @Override public RecoveryResponse newInstance() { return new RecoveryResponse(); } }) .txGet()); } }); final RecoveryResponse recoveryResponse = responseHolder.get(); assert responseHolder != null; final TimeValue recoveryTime = new TimeValue(recoveryStatus.state().getTimer().time()); // do this through ongoing recoveries to remove it from the collection onGoingRecoveries.markRecoveryAsDone(recoveryStatus.recoveryId()); if (logger.isTraceEnabled()) { StringBuilder sb = new StringBuilder(); sb.append('[') .append(request.shardId().index().name()) .append(']') .append('[') .append(request.shardId().id()) .append("] "); sb.append("recovery completed from ") .append(request.sourceNode()) .append(", took[") .append(recoveryTime) .append("]\n"); sb.append(" phase1: recovered_files [") .append(recoveryResponse.phase1FileNames.size()) .append("]") .append(" with total_size of [") .append(new ByteSizeValue(recoveryResponse.phase1TotalSize)) .append("]") .append(", took [") .append(timeValueMillis(recoveryResponse.phase1Time)) .append("], throttling_wait [") .append(timeValueMillis(recoveryResponse.phase1ThrottlingWaitTime)) .append(']') .append("\n"); sb.append(" : reusing_files [") .append(recoveryResponse.phase1ExistingFileNames.size()) .append("] with total_size of [") .append(new ByteSizeValue(recoveryResponse.phase1ExistingTotalSize)) .append("]\n"); sb.append(" phase2: start took [") .append(timeValueMillis(recoveryResponse.startTime)) .append("]\n"); sb.append(" : recovered [") .append(recoveryResponse.phase2Operations) .append("]") .append(" transaction log operations") .append(", took [") .append(timeValueMillis(recoveryResponse.phase2Time)) .append("]") .append("\n"); logger.trace(sb.toString()); } else { logger.debug( "{} recovery done from [{}], took [{}]", request.shardId(), recoveryStatus.sourceNode(), recoveryTime); } } catch (CancellableThreads.ExecutionCancelledException e) { logger.trace("recovery cancelled", e); } catch (Throwable e) { if (logger.isTraceEnabled()) { logger.trace( "[{}][{}] Got exception on recovery", e, request.shardId().index().name(), request.shardId().id()); } Throwable cause = ExceptionsHelper.unwrapCause(e); if (cause instanceof RecoveryEngineException) { // unwrap an exception that was thrown as part of the recovery cause = cause.getCause(); } // do it twice, in case we have double transport exception cause = ExceptionsHelper.unwrapCause(cause); if (cause instanceof RecoveryEngineException) { // unwrap an exception that was thrown as part of the recovery cause = cause.getCause(); } // here, we would add checks against exception that need to be retried (and not removeAndClean // in this case) if (cause instanceof IllegalIndexShardStateException || cause instanceof IndexNotFoundException || cause instanceof ShardNotFoundException) { // if the target is not ready yet, retry retryRecovery( recoveryStatus, "remote shard not ready", recoverySettings.retryDelayStateSync(), request); return; } if (cause instanceof DelayRecoveryException) { retryRecovery(recoveryStatus, cause, recoverySettings.retryDelayStateSync(), request); return; } if (cause instanceof ConnectTransportException) { logger.debug( "delaying recovery of {} for [{}] due to networking error [{}]", recoveryStatus.shardId(), recoverySettings.retryDelayNetwork(), cause.getMessage()); retryRecovery( recoveryStatus, cause.getMessage(), recoverySettings.retryDelayNetwork(), request); return; } if (cause instanceof IndexShardClosedException) { onGoingRecoveries.failRecovery( recoveryStatus.recoveryId(), new RecoveryFailedException(request, "source shard is closed", cause), false); return; } if (cause instanceof AlreadyClosedException) { onGoingRecoveries.failRecovery( recoveryStatus.recoveryId(), new RecoveryFailedException(request, "source shard is closed", cause), false); return; } onGoingRecoveries.failRecovery( recoveryStatus.recoveryId(), new RecoveryFailedException(request, e), true); } }