From 25d01564b1ba1eae49a81b8e4247282d02a3a598 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 24 Jun 2019 02:30:41 -0400 Subject: [PATCH 01/27] Peer recovery sends file chunks non-blocking --- .../indices/recovery/MultiFileReader.java | 133 ++++++++++++++ .../recovery/PeerRecoverySourceService.java | 2 +- .../recovery/RecoverySourceHandler.java | 167 ++++++++++++------ .../recovery/RecoverySourceHandlerTests.java | 128 ++++++++------ .../index/shard/IndexShardTestCase.java | 4 +- .../indices/recovery/AsyncRecoveryTarget.java | 6 +- 6 files changed, 327 insertions(+), 113 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java new file mode 100644 index 0000000000000..dba2b3a52b8d6 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java @@ -0,0 +1,133 @@ +/* + * 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.indices.recovery; + +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lucene.store.InputStreamIndexInput; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.store.StoreFileMetaData; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Arrays; +import java.util.Deque; +import java.util.Iterator; +import java.util.Objects; + +/** + * A thread safe implementation that allows reading multiple files sequentially chunk by chunk. + */ +final class MultiFileReader implements Closeable { + private boolean closed = false; + private final Store store; + private final Iterator remainingFiles; + private StoreFileMetaData md; + private InputStreamIndexInput currentInput = null; + private long currentChunkPosition = 0; + private final int chunkSizeInBytes; + private final Deque recycledBuffers; + + MultiFileReader(Store store, StoreFileMetaData[] files, int chunkSizeInBytes) { + this.store = store; + this.remainingFiles = Arrays.asList(files).iterator(); + this.chunkSizeInBytes = chunkSizeInBytes; + this.recycledBuffers = ConcurrentCollections.newDeque(); + } + + synchronized StoreFileMetaData currentFile() { + assert md != null; + return md; + } + + /** + * Reads the next file chunk if available. This method returns {@code null} when all provided files are exhaustedly read. + * When the caller no longer needs the returned file chunk, it should call {@link FileChunk#close()} to release that file + * chunk so the reader can reuse the associated buffer to reduce allocation. + */ + synchronized FileChunk readNextChunk() throws IOException { + if (closed) { + throw new IllegalStateException("MultiFileReader was closed"); + } + if (currentInput == null) { + if (remainingFiles.hasNext() == false) { + return null; + } + currentChunkPosition = 0; + md = remainingFiles.next(); + final IndexInput indexInput = store.directory().openInput(md.name(), IOContext.READONCE); + currentInput = new InputStreamIndexInput(indexInput, md.length()) { + @Override + public void close() throws IOException { + indexInput.close(); //InputStreamIndexInput's close is noop + } + }; + } + final byte[] buffer = Objects.requireNonNullElseGet(recycledBuffers.pollFirst(), () -> new byte[chunkSizeInBytes]); + final int bytesRead = currentInput.read(buffer); + if (bytesRead == -1) { + throw new CorruptIndexException("file truncated, expected length= " + md.length() + " position=" + currentChunkPosition, md.name()); + } + final long chunkPosition = currentChunkPosition; + currentChunkPosition += bytesRead; + final boolean lastChunk = currentChunkPosition == md.length(); + final FileChunk chunk = new FileChunk( + md, new BytesArray(buffer, 0, bytesRead), chunkPosition, lastChunk, () -> recycledBuffers.addFirst(buffer)); + if (lastChunk) { + IOUtils.close(currentInput, () -> currentInput = null); + } + return chunk; + } + + @Override + public synchronized void close() throws IOException { + if (closed == false) { + closed = true; + IOUtils.close(currentInput, () -> currentInput = null); + } + } + + static final class FileChunk implements Releasable { + final StoreFileMetaData md; + final BytesReference content; + final long position; + final boolean lastChunk; + private final Releasable onClose; + + private FileChunk(StoreFileMetaData md, BytesReference content, long position, boolean lastChunk, Releasable onClose) { + this.md = md; + this.content = content; + this.position = position; + this.lastChunk = lastChunk; + this.onClose = onClose; + } + + @Override + public void close() { + onClose.close(); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java index f53e8edecd9e6..b9650a641ea48 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java @@ -176,7 +176,7 @@ private RecoverySourceHandler createRecoverySourceHandler(StartRecoveryRequest r final RemoteRecoveryTargetHandler recoveryTarget = new RemoteRecoveryTargetHandler(request.recoveryId(), request.shardId(), transportService, request.targetNode(), recoverySettings, throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime)); - handler = new RecoverySourceHandler(shard, recoveryTarget, request, + handler = new RecoverySourceHandler(shard, recoveryTarget, request, shard.getThreadPool().generic(), Math.toIntExact(recoverySettings.getChunkSize().getBytes()), recoverySettings.getMaxConcurrentFileChunks()); return handler; } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 3ae9598124b05..3270d1b1823d8 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -25,23 +25,19 @@ 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.IndexInput; import org.apache.lucene.store.RateLimiter; import org.apache.lucene.util.ArrayUtil; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.StepListener; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.CheckedSupplier; import org.elasticsearch.common.StopWatch; -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.Loggers; -import org.elasticsearch.common.lucene.store.InputStreamIndexInput; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; @@ -61,10 +57,10 @@ import org.elasticsearch.index.translog.Translog; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.RemoteTransportException; +import org.elasticsearch.transport.Transports; import java.io.Closeable; import java.io.IOException; -import java.io.InputStream; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -72,8 +68,9 @@ import java.util.Locale; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.Executor; +import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.IntSupplier; import java.util.stream.StreamSupport; @@ -103,10 +100,13 @@ public class RecoverySourceHandler { private final RecoveryTargetHandler recoveryTarget; private final int maxConcurrentFileChunks; private final CancellableThreads cancellableThreads = new CancellableThreads(); + private final List resources = new CopyOnWriteArrayList<>(); + private final Executor sendFileExecutor; - public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recoveryTarget, final StartRecoveryRequest request, - final int fileChunkSizeInBytes, final int maxConcurrentFileChunks) { + public RecoverySourceHandler(IndexShard shard, RecoveryTargetHandler recoveryTarget, StartRecoveryRequest request, + Executor sendFileExecutor, int fileChunkSizeInBytes, int maxConcurrentFileChunks) { this.shard = shard; + this.sendFileExecutor = sendFileExecutor; this.recoveryTarget = recoveryTarget; this.request = request; this.shardId = this.request.shardId().id(); @@ -123,7 +123,6 @@ public StartRecoveryRequest getRequest() { * performs the recovery from the local engine to the target */ public void recoverToTarget(ActionListener listener) { - final List resources = new CopyOnWriteArrayList<>(); final Closeable releaseResources = () -> IOUtils.close(resources); final ActionListener wrappedListener = ActionListener.notifyOnce(listener); try { @@ -404,13 +403,17 @@ void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps phase1FileNames.size(), new ByteSizeValue(totalSizeInBytes), phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSizeInBytes)); final StepListener sendFileInfoStep = new StepListener<>(); + final StepListener sendFileChunkStep = new StepListener<>(); final StepListener cleanFilesStep = new StepListener<>(); cancellableThreads.execute(() -> recoveryTarget.receiveFileInfo(phase1FileNames, phase1FileSizes, phase1ExistingFileNames, phase1ExistingFileSizes, translogOps.getAsInt(), sendFileInfoStep)); sendFileInfoStep.whenComplete(r -> { - sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps); + sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps, sendFileChunkStep); + }, listener::onFailure); + + sendFileChunkStep.whenComplete(r -> { cleanFiles(store, recoverySourceMetadata, translogOps, globalCheckpoint, cleanFilesStep); }, listener::onFailure); @@ -571,6 +574,7 @@ private void sendBatch( final long mappingVersionOnPrimary, final ActionListener listener) throws IOException { assert ThreadPool.assertCurrentMethodIsNotCalledRecursively(); + assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[send translog]"); final List operations = nextBatch.get(); // send the leftover operations or if no operations were sent, request the target to respond with its local checkpoint if (operations.isEmpty() == false || firstBatch) { @@ -669,56 +673,112 @@ public String toString() { '}'; } - void sendFiles(Store store, StoreFileMetaData[] files, IntSupplier translogOps) throws Exception { - ArrayUtil.timSort(files, Comparator.comparingLong(StoreFileMetaData::length)); // send smallest first - final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); - final AtomicReference> error = new AtomicReference<>(); - final byte[] buffer = new byte[chunkSizeInBytes]; - for (final StoreFileMetaData md : files) { - if (error.get() != null) { - break; - } - try (IndexInput indexInput = store.directory().openInput(md.name(), IOContext.READONCE); - InputStream in = new InputStreamIndexInput(indexInput, md.length())) { - long position = 0; - int bytesRead; - while ((bytesRead = in.read(buffer, 0, buffer.length)) != -1) { - final BytesArray content = new BytesArray(buffer, 0, bytesRead); - final boolean lastChunk = position + content.length() == md.length(); - final long requestSeqId = requestSeqIdTracker.generateSeqNo(); - cancellableThreads.execute( - () -> requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqId - maxConcurrentFileChunks)); - cancellableThreads.checkForCancel(); - if (error.get() != null) { - break; + /** + * File chunks are read/sent sequentially by at most one thread at any time. The sender, however, won't wait for the acknowledgement + * before reading/sending the next chunk to increase the recovery speed especially on secure/compressed or high latency communication. + *

+ * The sender can send up to {@code maxConcurrentFileChunks} file chunks without waiting for acknowledgments. Since the recovery target + * can receive file chunks out of order, it has to buffer those file chunks in memory and only flush to disk when there's no gap. + * To ensure the recover target never buffers more than {@code maxConcurrentFileChunks} file chunks, we allow the sender to send only up + * to {@code maxConcurrentFileChunks} file chunks from the last flushed (and acknowledged) file chunk. We leverage the local checkpoint + * tracker for this purpose. We generate a new sequence number and assign it to each file chunk before sending; then mark that sequence + * number as processed when we receive an acknowledgement for the corresponding file chunk request. With the local checkpoint tracker, + * we know the last acknowledged-flushed file-chunk is a file chunk whose {@code requestSeqId} equals to the local checkpoint because + * the recover target can flush all file chunks up to the local checkpoint. + *

+ * When the number of un-replied file chunk requests reaches the limit (i.e. the gap between the max_seq_no and the local checkpoint is + * greater than {@code maxConcurrentFileChunks}), the sending thread will abort its execution. The sending process will be resumed by + * one of the networking threads which receive/handle the acknowledgments of the current pending file chunk requests. This process will + * continue until all chunks are sent and acknowledged. + */ + private class MultiFileSender extends ActionRunnable implements Closeable { + private final Store store; + private final MultiFileReader multiFileReader; + private final IntSupplier translogOps; + private final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); + private final Semaphore semaphore = new Semaphore(0); + + private MultiFileSender(Store store, IntSupplier translogOps, StoreFileMetaData[] files, ActionListener listener) { + super(ActionListener.notifyOnce(listener)); + this.store = store; + this.translogOps = translogOps; + this.multiFileReader = new MultiFileReader(store, files, chunkSizeInBytes); + } + + @Override + protected void doRun() throws Exception { + assert ThreadPool.assertCurrentMethodIsNotCalledRecursively(); + assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[send file chunk]"); + for (; ; ) { + assert semaphore.availablePermits() == 0; + final MultiFileReader.FileChunk chunk; + try { + chunk = multiFileReader.readNextChunk(); + } catch (IOException e) { + handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{ multiFileReader.currentFile() }); + throw e; + } + if (chunk == null) { + semaphore.release(); // allow other threads respond if we are not done yet. + if (requestSeqIdTracker.getMaxSeqNo() == requestSeqIdTracker.getProcessedCheckpoint() && semaphore.tryAcquire()) { + listener.onResponse(null); } - final long requestFilePosition = position; - cancellableThreads.executeIO(() -> - recoveryTarget.writeFileChunk(md, requestFilePosition, content, lastChunk, translogOps.getAsInt(), - ActionListener.wrap( - r -> requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId), - e -> { - error.compareAndSet(null, Tuple.tuple(md, e)); - requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); + break; + } + final long requestSeqId = requestSeqIdTracker.generateSeqNo(); + cancellableThreads.execute(() -> + recoveryTarget.writeFileChunk(chunk.md, chunk.position, chunk.content, chunk.lastChunk, translogOps.getAsInt(), + ActionListener.wrap( + r -> { + chunk.close(); // release the buffer so we can reuse to reduce allocation + requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); + if (canSendMore() && semaphore.tryAcquire()) { + sendFileExecutor.execute(this); // fork off from the network thread } - ))); - position += content.length(); + }, + e -> ActionListener.completeWith(listener, () -> { + handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{ chunk.md }); + throw e; + }) + ) + ) + ); + if (canSendMore() == false) { + semaphore.release(); + // Here we have to retry before abort to avoid a race situation where the other threads have flipped `canSendMore` + // condition but they are not going to resume the sending process because this thread still holds the semaphore. + if (canSendMore() == false || semaphore.tryAcquire() == false) { + break; + } } - } catch (Exception e) { - error.compareAndSet(null, Tuple.tuple(md, e)); - break; } } - // When we terminate exceptionally, we don't wait for the outstanding requests as we don't use their results anyway. - // This allows us to end quickly and eliminate the complexity of handling requestSeqIds in case of error. - if (error.get() == null) { - cancellableThreads.execute(() -> requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqIdTracker.getMaxSeqNo())); + + private boolean canSendMore() { + return requestSeqIdTracker.getMaxSeqNo() - requestSeqIdTracker.getProcessedCheckpoint() < maxConcurrentFileChunks; } - if (error.get() != null) { - handleErrorOnSendFiles(store, error.get().v2(), new StoreFileMetaData[]{error.get().v1()}); + + @Override + public void close() throws IOException { + multiFileReader.close(); } } + void sendFiles(Store store, StoreFileMetaData[] files, IntSupplier translogOps, ActionListener listener) { + ArrayUtil.timSort(files, Comparator.comparingLong(StoreFileMetaData::length)); // send smallest first + StepListener wrappedListener = new StepListener<>(); + MultiFileSender multiFileSender = new MultiFileSender(store, translogOps, files, wrappedListener); + wrappedListener.whenComplete(r -> { + multiFileSender.close(); + listener.onResponse(null); + }, e -> { + IOUtils.closeWhileHandlingException(multiFileSender); + listener.onFailure(e); + }); + resources.add(multiFileSender); + multiFileSender.run(); + } + private void cleanFiles(Store store, Store.MetadataSnapshot sourceMetadata, IntSupplier translogOps, long globalCheckpoint, ActionListener listener) { // Send the CLEAN_FILES request, which takes all of the files that @@ -763,9 +823,8 @@ private void handleErrorOnSendFiles(Store store, Exception e, StoreFileMetaData[ shardId, request.targetNode(), mds), corruptIndexException); throw remoteException; } - } else { - throw e; } + throw e; } protected void failEngine(IOException cause) { diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 215bf475a0c9b..628b5c7c91bbb 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -34,6 +34,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; @@ -77,6 +78,7 @@ import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; +import org.elasticsearch.threadpool.FixedExecutorBuilder; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; @@ -93,10 +95,10 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.IntSupplier; import java.util.zip.CRC32; @@ -121,10 +123,18 @@ public class RecoverySourceHandlerTests extends ESTestCase { private final ClusterSettings service = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); private ThreadPool threadPool; + private String sendFileExecutor; @Before public void setUpThreadPool() { - threadPool = new TestThreadPool(getTestName()); + if (randomBoolean()) { + sendFileExecutor = "send_files"; + threadPool = new TestThreadPool(getTestName(), new FixedExecutorBuilder( + Settings.EMPTY, sendFileExecutor, randomIntBetween(1, 16), randomIntBetween(16, 128), sendFileExecutor)); + } else { + sendFileExecutor = ThreadPool.Names.GENERIC; + threadPool = new TestThreadPool(getTestName()); + } } @After @@ -185,9 +195,11 @@ public void close() throws IOException { } } }; - RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, + RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, threadPool.executor(sendFileExecutor), Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 5)); - handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0); + PlainActionFuture sendFileFuture = new PlainActionFuture<>(); + handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, sendFileFuture); + sendFileFuture.actionGet(); Store.MetadataSnapshot targetStoreMetadata = targetStore.getMetadata(null); Store.RecoveryDiff recoveryDiff = targetStoreMetadata.recoveryDiff(metadata); assertEquals(metas.size(), recoveryDiff.identical.size()); @@ -243,8 +255,8 @@ public void indexTranslogOperations(List operations, int tot checkpointOnTarget.set(randomLongBetween(checkpointOnTarget.get(), Long.MAX_VALUE)); listener.onResponse(checkpointOnTarget.get()); } }; - RecoverySourceHandler handler = new RecoverySourceHandler( - shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), request, fileChunkSizeInBytes, between(1, 10)); + RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), + request, threadPool.executor(sendFileExecutor), fileChunkSizeInBytes, between(1, 10)); PlainActionFuture future = new PlainActionFuture<>(); handler.phase2(startingSeqNo, endingSeqNo, newTranslogSnapshot(operations, Collections.emptyList()), randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, randomNonNegativeLong(), future); @@ -283,8 +295,9 @@ public void indexTranslogOperations(List operations, int tot } } }; - RecoverySourceHandler handler = new RecoverySourceHandler( - shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), request, fileChunkSizeInBytes, between(1, 10)); + RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), + request, threadPool.executor(sendFileExecutor), + fileChunkSizeInBytes, between(1, 10)); PlainActionFuture future = new PlainActionFuture<>(); final long startingSeqNo = randomLongBetween(0, ops.size() - 1L); final long endingSeqNo = randomLongBetween(startingSeqNo, ops.size() - 1L); @@ -372,7 +385,7 @@ public void close() throws IOException { } } }; - RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, + RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, threadPool.executor(sendFileExecutor), Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 8)) { @Override protected void failEngine(IOException cause) { @@ -380,13 +393,22 @@ protected void failEngine(IOException cause) { failedEngine.set(true); } }; - - try { - handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0); - fail("corrupted index"); - } catch (IOException ex) { - assertNotNull(ExceptionsHelper.unwrapCorruption(ex)); - } + SetOnce error = new SetOnce<>(); + CountDownLatch latch = new CountDownLatch(1); + handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, + ActionListener.wrap( + r -> { + error.set(null); + latch.countDown(); + }, + e -> { + error.set(e); + latch.countDown(); + } + )); + latch.await(); + assertNotNull(error.get()); + assertNotNull(ExceptionsHelper.unwrapCorruption(error.get())); assertTrue(failedEngine.get()); IOUtils.close(store, targetStore); } @@ -427,7 +449,7 @@ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference c } } }; - RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, + RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, threadPool.executor(sendFileExecutor), Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 10)) { @Override protected void failEngine(IOException cause) { @@ -436,7 +458,9 @@ protected void failEngine(IOException cause) { } }; try { - handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0); + PlainActionFuture sendFileFuture = new PlainActionFuture<>(); + handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, sendFileFuture); + sendFileFuture.actionGet(); fail("exception index"); } catch (RuntimeException ex) { final IOException unwrappedCorruption = ExceptionsHelper.unwrapCorruption(ex); @@ -447,8 +471,6 @@ protected void failEngine(IOException cause) { assertNull(unwrappedCorruption); assertEquals(ex.getMessage(), "boom"); } - } catch (CorruptIndexException ex) { - fail("not expected here"); } assertFalse(failedEngine.get()); IOUtils.close(store); @@ -471,9 +493,8 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE final AtomicBoolean phase2Called = new AtomicBoolean(); final RecoverySourceHandler handler = new RecoverySourceHandler( shard, - mock(RecoveryTargetHandler.class), - request, - Math.toIntExact(recoverySettings.getChunkSize().getBytes()), + mock(RecoveryTargetHandler.class), request, threadPool.executor(sendFileExecutor), + Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 8)) { @Override @@ -543,26 +564,30 @@ public void testSendFileChunksConcurrently() throws Exception { public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { final long chunkNumber = chunkNumberGenerator.getAndIncrement(); - logger.info("--> write chunk name={} seq={}, position={}", md.name(), chunkNumber, position); - unrepliedChunks.add(new FileChunkResponse(chunkNumber, listener)); - sentChunks.incrementAndGet(); + Runnable reply = () -> { + logger.info("--> write chunk name={} seq={}, position={}", md.name(), chunkNumber, position); + unrepliedChunks.add(new FileChunkResponse(chunkNumber, listener)); + sentChunks.incrementAndGet(); + }; + if (randomBoolean()) { + threadPool.executor(sendFileExecutor).execute(reply); + } else if (randomBoolean()) { + threadPool.schedule(reply, TimeValue.timeValueNanos(randomIntBetween(1, 100)), sendFileExecutor); + } else { + reply.run(); + } } }; final int maxConcurrentChunks = between(1, 8); final int chunkSize = between(1, 32); final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, getStartRecoveryRequest(), + threadPool.executor(sendFileExecutor), chunkSize, maxConcurrentChunks); Store store = newStore(createTempDir(), false); - List files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20)); + List files = generateFiles(store, between(1, 20), () -> between(1, chunkSize * 500)); int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum(); - Thread sender = new Thread(() -> { - try { - handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0); - } catch (Exception ex) { - throw new AssertionError(ex); - } - }); - sender.start(); + PlainActionFuture sendFileFuture = new PlainActionFuture<>(); + handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0, sendFileFuture); assertBusy(() -> { assertThat(sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks))); assertThat(unrepliedChunks, hasSize(sentChunks.get())); @@ -594,7 +619,7 @@ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference c assertThat(unrepliedChunks, hasSize(expectedUnrepliedChunks)); }); } - sender.join(); + sendFileFuture.actionGet(); store.close(); } @@ -609,27 +634,28 @@ public void testSendFileChunksStopOnError() throws Exception { public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { final long chunkNumber = chunkNumberGenerator.getAndIncrement(); - logger.info("--> write chunk name={} seq={}, position={}", md.name(), chunkNumber, position); - unrepliedChunks.add(new FileChunkResponse(chunkNumber, listener)); - sentChunks.incrementAndGet(); + final Runnable reply = () -> { + logger.info("--> write chunk name={} seq={}, position={}", md.name(), chunkNumber, position); + unrepliedChunks.add(new FileChunkResponse(chunkNumber, listener)); + sentChunks.incrementAndGet(); + }; + if (randomBoolean()) { + threadPool.executor(sendFileExecutor).execute(reply); + } else { + reply.run(); + } } }; final int maxConcurrentChunks = between(1, 4); final int chunkSize = between(1, 16); final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, getStartRecoveryRequest(), + threadPool.executor(sendFileExecutor), chunkSize, maxConcurrentChunks); Store store = newStore(createTempDir(), false); List files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20)); int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum(); - AtomicReference error = new AtomicReference<>(); - Thread sender = new Thread(() -> { - try { - handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0); - } catch (Exception ex) { - error.set(ex); - } - }); - sender.start(); + SetOnce error = new SetOnce<>(); + handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0, ActionListener.wrap(r -> error.set(null), error::set)); assertBusy(() -> assertThat(sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks)))); List failedChunks = randomSubsetOf(between(1, unrepliedChunks.size()), unrepliedChunks); failedChunks.forEach(c -> c.listener.onFailure(new RuntimeException("test chunk exception"))); @@ -646,15 +672,15 @@ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference c assertThat(error.get().getMessage(), containsString("test chunk exception")); }); assertThat("no more chunks should be sent", sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks))); - sender.join(); store.close(); } public void testVerifySeqNoStatsWhenRecoverWithSyncId() throws Exception { IndexShard shard = mock(IndexShard.class); when(shard.state()).thenReturn(IndexShardState.STARTED); - RecoverySourceHandler handler = new RecoverySourceHandler( - shard, new TestRecoveryTargetHandler(), getStartRecoveryRequest(), between(1, 16), between(1, 4)); + RecoverySourceHandler handler = new RecoverySourceHandler(shard, new TestRecoveryTargetHandler(), getStartRecoveryRequest(), + threadPool.executor(sendFileExecutor), + between(1, 16), between(1, 4)); String syncId = UUIDs.randomBase64UUID(); int numDocs = between(0, 1000); 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 47a8f73ef62c9..68496fc3cbb94 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 @@ -635,8 +635,8 @@ protected final void recoverUnstartedReplica(final IndexShard replica, final StartRecoveryRequest request = new StartRecoveryRequest(replica.shardId(), targetAllocationId, pNode, rNode, snapshot, replica.routingEntry().primary(), 0, startingSeqNo); final RecoverySourceHandler recovery = new RecoverySourceHandler(primary, - new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), - request, Math.toIntExact(ByteSizeUnit.MB.toBytes(1)), between(1, 8)); + new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), request, threadPool.generic(), + Math.toIntExact(ByteSizeUnit.MB.toBytes(1)), between(1, 8)); primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable); diff --git a/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java b/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java index afd2aa4e85888..6ef9725ccaf80 100644 --- a/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java +++ b/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java @@ -19,9 +19,7 @@ package org.elasticsearch.indices.recovery; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLeases; @@ -83,8 +81,6 @@ public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.Metada @Override public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { - // TODO: remove this clone once we send file chunk async - final BytesReference copy = new BytesArray(BytesRef.deepCopyOf(content.toBytesRef())); - executor.execute(() -> target.writeFileChunk(fileMetaData, position, copy, lastChunk, totalTranslogOps, listener)); + executor.execute(() -> target.writeFileChunk(fileMetaData, position, content, lastChunk, totalTranslogOps, listener)); } } From 782920c3750c9628462943c86f7b424a5c9c4aee Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 6 Jul 2019 15:01:46 -0400 Subject: [PATCH 02/27] read ahead --- .../recovery/RecoverySourceHandler.java | 26 +++++++++++++------ 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 3270d1b1823d8..97f7df7aaf53b 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -697,6 +697,7 @@ private class MultiFileSender extends ActionRunnable implements Closeable private final IntSupplier translogOps; private final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); private final Semaphore semaphore = new Semaphore(0); + private MultiFileReader.FileChunk currentChunk; private MultiFileSender(Store store, IntSupplier translogOps, StoreFileMetaData[] files, ActionListener listener) { super(ActionListener.notifyOnce(listener)); @@ -711,14 +712,8 @@ protected void doRun() throws Exception { assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[send file chunk]"); for (; ; ) { assert semaphore.availablePermits() == 0; - final MultiFileReader.FileChunk chunk; - try { - chunk = multiFileReader.readNextChunk(); - } catch (IOException e) { - handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{ multiFileReader.currentFile() }); - throw e; - } - if (chunk == null) { + final MultiFileReader.FileChunk chunk = readNextChunk(); + if (currentChunk == null) { semaphore.release(); // allow other threads respond if we are not done yet. if (requestSeqIdTracker.getMaxSeqNo() == requestSeqIdTracker.getProcessedCheckpoint() && semaphore.tryAcquire()) { listener.onResponse(null); @@ -726,6 +721,7 @@ protected void doRun() throws Exception { break; } final long requestSeqId = requestSeqIdTracker.generateSeqNo(); + currentChunk = null; cancellableThreads.execute(() -> recoveryTarget.writeFileChunk(chunk.md, chunk.position, chunk.content, chunk.lastChunk, translogOps.getAsInt(), ActionListener.wrap( @@ -744,6 +740,7 @@ protected void doRun() throws Exception { ) ); if (canSendMore() == false) { + readNextChunk(); // read ahead while we're waiting for acknowledgements semaphore.release(); // Here we have to retry before abort to avoid a race situation where the other threads have flipped `canSendMore` // condition but they are not going to resume the sending process because this thread still holds the semaphore. @@ -754,6 +751,19 @@ protected void doRun() throws Exception { } } + private MultiFileReader.FileChunk readNextChunk() throws Exception { + assert semaphore.availablePermits() == 0; + if (currentChunk == null) { + try { + currentChunk = multiFileReader.readNextChunk(); + } catch (IOException e) { + handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{multiFileReader.currentFile()}); + throw e; + } + } + return currentChunk; + } + private boolean canSendMore() { return requestSeqIdTracker.getMaxSeqNo() - requestSeqIdTracker.getProcessedCheckpoint() < maxConcurrentFileChunks; } From eb4b0979daaa1026b472a7e2dd54097eb5be5c05 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 6 Jul 2019 16:37:38 -0400 Subject: [PATCH 03/27] Revert "read ahead" This reverts commit 782920c3750c9628462943c86f7b424a5c9c4aee. --- .../recovery/RecoverySourceHandler.java | 26 ++++++------------- 1 file changed, 8 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 97f7df7aaf53b..3270d1b1823d8 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -697,7 +697,6 @@ private class MultiFileSender extends ActionRunnable implements Closeable private final IntSupplier translogOps; private final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); private final Semaphore semaphore = new Semaphore(0); - private MultiFileReader.FileChunk currentChunk; private MultiFileSender(Store store, IntSupplier translogOps, StoreFileMetaData[] files, ActionListener listener) { super(ActionListener.notifyOnce(listener)); @@ -712,8 +711,14 @@ protected void doRun() throws Exception { assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[send file chunk]"); for (; ; ) { assert semaphore.availablePermits() == 0; - final MultiFileReader.FileChunk chunk = readNextChunk(); - if (currentChunk == null) { + final MultiFileReader.FileChunk chunk; + try { + chunk = multiFileReader.readNextChunk(); + } catch (IOException e) { + handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{ multiFileReader.currentFile() }); + throw e; + } + if (chunk == null) { semaphore.release(); // allow other threads respond if we are not done yet. if (requestSeqIdTracker.getMaxSeqNo() == requestSeqIdTracker.getProcessedCheckpoint() && semaphore.tryAcquire()) { listener.onResponse(null); @@ -721,7 +726,6 @@ protected void doRun() throws Exception { break; } final long requestSeqId = requestSeqIdTracker.generateSeqNo(); - currentChunk = null; cancellableThreads.execute(() -> recoveryTarget.writeFileChunk(chunk.md, chunk.position, chunk.content, chunk.lastChunk, translogOps.getAsInt(), ActionListener.wrap( @@ -740,7 +744,6 @@ protected void doRun() throws Exception { ) ); if (canSendMore() == false) { - readNextChunk(); // read ahead while we're waiting for acknowledgements semaphore.release(); // Here we have to retry before abort to avoid a race situation where the other threads have flipped `canSendMore` // condition but they are not going to resume the sending process because this thread still holds the semaphore. @@ -751,19 +754,6 @@ protected void doRun() throws Exception { } } - private MultiFileReader.FileChunk readNextChunk() throws Exception { - assert semaphore.availablePermits() == 0; - if (currentChunk == null) { - try { - currentChunk = multiFileReader.readNextChunk(); - } catch (IOException e) { - handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{multiFileReader.currentFile()}); - throw e; - } - } - return currentChunk; - } - private boolean canSendMore() { return requestSeqIdTracker.getMaxSeqNo() - requestSeqIdTracker.getProcessedCheckpoint() < maxConcurrentFileChunks; } From 8d64abea5e46695366c086821d5634ede3897ed1 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 6 Jul 2019 19:27:28 -0400 Subject: [PATCH 04/27] style check --- .../org/elasticsearch/indices/recovery/MultiFileReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java index dba2b3a52b8d6..1b12687ce1298 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java @@ -89,7 +89,7 @@ public void close() throws IOException { final byte[] buffer = Objects.requireNonNullElseGet(recycledBuffers.pollFirst(), () -> new byte[chunkSizeInBytes]); final int bytesRead = currentInput.read(buffer); if (bytesRead == -1) { - throw new CorruptIndexException("file truncated, expected length= " + md.length() + " position=" + currentChunkPosition, md.name()); + throw new CorruptIndexException("file truncated; length=" + md.length() + " position=" + currentChunkPosition, md.name()); } final long chunkPosition = currentChunkPosition; currentChunkPosition += bytesRead; From 5c1977a2bf8e6f29c4f79ae5d8eae1d5162611a2 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sun, 7 Jul 2019 23:20:04 -0400 Subject: [PATCH 05/27] merge MultiFileReader --- .../indices/recovery/MultiFileReader.java | 133 ------------------ .../recovery/RecoverySourceHandler.java | 118 +++++++++++++--- .../recovery/RecoverySourceHandlerTests.java | 7 +- 3 files changed, 106 insertions(+), 152 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java deleted file mode 100644 index 1b12687ce1298..0000000000000 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileReader.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * 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.indices.recovery; - -import org.apache.lucene.index.CorruptIndexException; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexInput; -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.lucene.store.InputStreamIndexInput; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.store.StoreFileMetaData; - -import java.io.Closeable; -import java.io.IOException; -import java.util.Arrays; -import java.util.Deque; -import java.util.Iterator; -import java.util.Objects; - -/** - * A thread safe implementation that allows reading multiple files sequentially chunk by chunk. - */ -final class MultiFileReader implements Closeable { - private boolean closed = false; - private final Store store; - private final Iterator remainingFiles; - private StoreFileMetaData md; - private InputStreamIndexInput currentInput = null; - private long currentChunkPosition = 0; - private final int chunkSizeInBytes; - private final Deque recycledBuffers; - - MultiFileReader(Store store, StoreFileMetaData[] files, int chunkSizeInBytes) { - this.store = store; - this.remainingFiles = Arrays.asList(files).iterator(); - this.chunkSizeInBytes = chunkSizeInBytes; - this.recycledBuffers = ConcurrentCollections.newDeque(); - } - - synchronized StoreFileMetaData currentFile() { - assert md != null; - return md; - } - - /** - * Reads the next file chunk if available. This method returns {@code null} when all provided files are exhaustedly read. - * When the caller no longer needs the returned file chunk, it should call {@link FileChunk#close()} to release that file - * chunk so the reader can reuse the associated buffer to reduce allocation. - */ - synchronized FileChunk readNextChunk() throws IOException { - if (closed) { - throw new IllegalStateException("MultiFileReader was closed"); - } - if (currentInput == null) { - if (remainingFiles.hasNext() == false) { - return null; - } - currentChunkPosition = 0; - md = remainingFiles.next(); - final IndexInput indexInput = store.directory().openInput(md.name(), IOContext.READONCE); - currentInput = new InputStreamIndexInput(indexInput, md.length()) { - @Override - public void close() throws IOException { - indexInput.close(); //InputStreamIndexInput's close is noop - } - }; - } - final byte[] buffer = Objects.requireNonNullElseGet(recycledBuffers.pollFirst(), () -> new byte[chunkSizeInBytes]); - final int bytesRead = currentInput.read(buffer); - if (bytesRead == -1) { - throw new CorruptIndexException("file truncated; length=" + md.length() + " position=" + currentChunkPosition, md.name()); - } - final long chunkPosition = currentChunkPosition; - currentChunkPosition += bytesRead; - final boolean lastChunk = currentChunkPosition == md.length(); - final FileChunk chunk = new FileChunk( - md, new BytesArray(buffer, 0, bytesRead), chunkPosition, lastChunk, () -> recycledBuffers.addFirst(buffer)); - if (lastChunk) { - IOUtils.close(currentInput, () -> currentInput = null); - } - return chunk; - } - - @Override - public synchronized void close() throws IOException { - if (closed == false) { - closed = true; - IOUtils.close(currentInput, () -> currentInput = null); - } - } - - static final class FileChunk implements Releasable { - final StoreFileMetaData md; - final BytesReference content; - final long position; - final boolean lastChunk; - private final Releasable onClose; - - private FileChunk(StoreFileMetaData md, BytesReference content, long position, boolean lastChunk, Releasable onClose) { - this.md = md; - this.content = content; - this.position = position; - this.lastChunk = lastChunk; - this.onClose = onClose; - } - - @Override - public void close() { - onClose.close(); - } - } -} diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 3270d1b1823d8..14b13a44eda25 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -25,6 +25,8 @@ 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.IndexInput; import org.apache.lucene.store.RateLimiter; import org.apache.lucene.util.ArrayUtil; import org.elasticsearch.ExceptionsHelper; @@ -35,12 +37,16 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.CheckedSupplier; import org.elasticsearch.common.StopWatch; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.common.lucene.store.InputStreamIndexInput; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.Engine; @@ -62,10 +68,14 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Comparator; +import java.util.Deque; +import java.util.Iterator; import java.util.List; import java.util.Locale; +import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.Executor; @@ -693,31 +703,30 @@ public String toString() { */ private class MultiFileSender extends ActionRunnable implements Closeable { private final Store store; - private final MultiFileReader multiFileReader; private final IntSupplier translogOps; private final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); private final Semaphore semaphore = new Semaphore(0); - - private MultiFileSender(Store store, IntSupplier translogOps, StoreFileMetaData[] files, ActionListener listener) { + private boolean closed = false; + private final Iterator remainingFiles; + private StoreFileMetaData currentFile; + private InputStreamIndexInput currentInput = null; + private long currentChunkPosition = 0; + private final Deque recycledBuffers = ConcurrentCollections.newDeque(); + + MultiFileSender(Store store, IntSupplier translogOps, StoreFileMetaData[] files, ActionListener listener) { super(ActionListener.notifyOnce(listener)); this.store = store; this.translogOps = translogOps; - this.multiFileReader = new MultiFileReader(store, files, chunkSizeInBytes); + this.remainingFiles = Arrays.asList(files).iterator(); } @Override protected void doRun() throws Exception { assert ThreadPool.assertCurrentMethodIsNotCalledRecursively(); assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[send file chunk]"); - for (; ; ) { + while (true) { assert semaphore.availablePermits() == 0; - final MultiFileReader.FileChunk chunk; - try { - chunk = multiFileReader.readNextChunk(); - } catch (IOException e) { - handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{ multiFileReader.currentFile() }); - throw e; - } + final FileChunk chunk = readNextChunk(); if (chunk == null) { semaphore.release(); // allow other threads respond if we are not done yet. if (requestSeqIdTracker.getMaxSeqNo() == requestSeqIdTracker.getProcessedCheckpoint() && semaphore.tryAcquire()) { @@ -736,9 +745,16 @@ protected void doRun() throws Exception { sendFileExecutor.execute(this); // fork off from the network thread } }, - e -> ActionListener.completeWith(listener, () -> { - handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{ chunk.md }); - throw e; + // need to fork as `handleErrorOnSendFiles` might read some files which should not happen on the network thread + e -> sendFileExecutor.execute(new ActionRunnable<>(this.listener) { + @Override + protected void doRun() throws Exception { + cancellableThreads.execute(semaphore::acquire); + try (Releasable ignored = semaphore::release) { + handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{chunk.md}); + throw e; + } + } }) ) ) @@ -754,13 +770,79 @@ protected void doRun() throws Exception { } } - private boolean canSendMore() { + FileChunk readNextChunk() throws Exception { + assert semaphore.availablePermits() == 0; + if (closed) { + throw new IllegalStateException("MultiFileSender was closed"); + } + try { + if (currentInput == null) { + if (remainingFiles.hasNext() == false) { + return null; + } + currentChunkPosition = 0; + currentFile = remainingFiles.next(); + final IndexInput indexInput = store.directory().openInput(currentFile.name(), IOContext.READONCE); + currentInput = new InputStreamIndexInput(indexInput, currentFile.length()) { + @Override + public void close() throws IOException { + indexInput.close(); //InputStreamIndexInput's close is noop + } + }; + } + final byte[] buffer = Objects.requireNonNullElseGet(recycledBuffers.pollFirst(), () -> new byte[chunkSizeInBytes]); + final int bytesRead = currentInput.read(buffer); + if (bytesRead == -1) { + throw new CorruptIndexException("file truncated; " + + "length=" + currentFile.length() + " position=" + currentChunkPosition, currentFile.name()); + } + final long chunkPosition = currentChunkPosition; + currentChunkPosition += bytesRead; + final boolean lastChunk = currentChunkPosition == currentFile.length(); + final FileChunk chunk = new FileChunk(currentFile, new BytesArray(buffer, 0, bytesRead), chunkPosition, lastChunk, + () -> recycledBuffers.addFirst(buffer)); + if (lastChunk) { + IOUtils.close(currentInput, () -> currentInput = null); + } + return chunk; + } catch (IOException e) { + handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{currentFile}); + throw e; + } + } + + boolean canSendMore() { return requestSeqIdTracker.getMaxSeqNo() - requestSeqIdTracker.getProcessedCheckpoint() < maxConcurrentFileChunks; } @Override public void close() throws IOException { - multiFileReader.close(); + assert semaphore.availablePermits() == 0; + if (closed == false) { + closed = true; + IOUtils.close(recycledBuffers::clear, currentInput, () -> currentInput = null); + } + } + } + + private static class FileChunk implements Releasable { + final StoreFileMetaData md; + final BytesReference content; + final long position; + final boolean lastChunk; + final Releasable onClose; + + FileChunk(StoreFileMetaData md, BytesReference content, long position, boolean lastChunk, Releasable onClose) { + this.md = md; + this.content = content; + this.position = position; + this.lastChunk = lastChunk; + this.onClose = onClose; + } + + @Override + public void close() { + onClose.close(); } } @@ -800,6 +882,7 @@ private void cleanFiles(Store store, Store.MetadataSnapshot sourceMetadata, IntS private void handleErrorOnSendFiles(Store store, Exception e, StoreFileMetaData[] mds) throws Exception { final IOException corruptIndexException = ExceptionsHelper.unwrapCorruption(e); + assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[handle error on send/clean files]"); if (corruptIndexException != null) { Exception localException = null; for (StoreFileMetaData md : mds) { @@ -830,4 +913,5 @@ private void handleErrorOnSendFiles(Store store, Exception e, StoreFileMetaData[ protected void failEngine(IOException cause) { shard.failShard("recovery", cause); } + } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 628b5c7c91bbb..a06336031c864 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -107,6 +107,7 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.core.IsNull.notNullValue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; @@ -457,8 +458,8 @@ protected void failEngine(IOException cause) { failedEngine.set(true); } }; + PlainActionFuture sendFileFuture = new PlainActionFuture<>(); try { - PlainActionFuture sendFileFuture = new PlainActionFuture<>(); handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, sendFileFuture); sendFileFuture.actionGet(); fail("exception index"); @@ -472,6 +473,7 @@ protected void failEngine(IOException cause) { assertEquals(ex.getMessage(), "boom"); } } + IOUtils.close(() -> terminate(threadPool), () -> threadPool = null); assertFalse(failedEngine.get()); IOUtils.close(store); } @@ -671,7 +673,8 @@ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference c assertThat(error.get(), notNullValue()); assertThat(error.get().getMessage(), containsString("test chunk exception")); }); - assertThat("no more chunks should be sent", sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks))); + assertThat("no more chunks should be sent", sentChunks.get(), + lessThanOrEqualTo(Math.min(totalChunks, maxConcurrentChunks * 2))); store.close(); } From 93d45c49b897cf58f0a498b76175f07494d03b1b Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sun, 7 Jul 2019 23:28:44 -0400 Subject: [PATCH 06/27] inline --- .../indices/recovery/RecoverySourceHandler.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 14b13a44eda25..968e3e6a61986 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -419,13 +419,11 @@ void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps recoveryTarget.receiveFileInfo(phase1FileNames, phase1FileSizes, phase1ExistingFileNames, phase1ExistingFileSizes, translogOps.getAsInt(), sendFileInfoStep)); - sendFileInfoStep.whenComplete(r -> { - sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps, sendFileChunkStep); - }, listener::onFailure); + sendFileInfoStep.whenComplete(r -> + sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps, sendFileChunkStep), listener::onFailure); - sendFileChunkStep.whenComplete(r -> { - cleanFiles(store, recoverySourceMetadata, translogOps, globalCheckpoint, cleanFilesStep); - }, listener::onFailure); + sendFileChunkStep.whenComplete(r -> + cleanFiles(store, recoverySourceMetadata, translogOps, globalCheckpoint, cleanFilesStep), listener::onFailure); final long totalSize = totalSizeInBytes; final long existingTotalSize = existingTotalSizeInBytes; From 472bd703fa4aa7fc30ac157f4f49dd9e0f5d5d93 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 8 Jul 2019 10:11:38 -0400 Subject: [PATCH 07/27] report error in the loop --- .../recovery/RecoverySourceHandler.java | 33 ++++++++----------- .../recovery/RecoverySourceHandlerTests.java | 4 +-- 2 files changed, 15 insertions(+), 22 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 968e3e6a61986..d91274c99e77b 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -39,6 +39,7 @@ import org.elasticsearch.common.StopWatch; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.Loggers; @@ -81,6 +82,7 @@ import java.util.concurrent.Executor; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.IntSupplier; import java.util.stream.StreamSupport; @@ -704,12 +706,12 @@ private class MultiFileSender extends ActionRunnable implements Closeable private final IntSupplier translogOps; private final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); private final Semaphore semaphore = new Semaphore(0); - private boolean closed = false; private final Iterator remainingFiles; private StoreFileMetaData currentFile; private InputStreamIndexInput currentInput = null; private long currentChunkPosition = 0; private final Deque recycledBuffers = ConcurrentCollections.newDeque(); + private final AtomicReference> error = new AtomicReference<>(); MultiFileSender(Store store, IntSupplier translogOps, StoreFileMetaData[] files, ActionListener listener) { super(ActionListener.notifyOnce(listener)); @@ -724,6 +726,10 @@ protected void doRun() throws Exception { assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[send file chunk]"); while (true) { assert semaphore.availablePermits() == 0; + if (error.get() != null) { + handleErrorOnSendFiles(store, error.get().v2(), new StoreFileMetaData[]{error.get().v1()}); + throw error.get().v2(); + } final FileChunk chunk = readNextChunk(); if (chunk == null) { semaphore.release(); // allow other threads respond if we are not done yet. @@ -743,25 +749,20 @@ protected void doRun() throws Exception { sendFileExecutor.execute(this); // fork off from the network thread } }, - // need to fork as `handleErrorOnSendFiles` might read some files which should not happen on the network thread - e -> sendFileExecutor.execute(new ActionRunnable<>(this.listener) { - @Override - protected void doRun() throws Exception { - cancellableThreads.execute(semaphore::acquire); - try (Releasable ignored = semaphore::release) { - handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{chunk.md}); - throw e; - } + e -> { + if (error.compareAndSet(null, Tuple.tuple(chunk.md, e)) && semaphore.tryAcquire()) { + // have to fork as handleErrorOnSendFiles can read file which should not happen on the network thread. + sendFileExecutor.execute(this); } }) - ) ) ); if (canSendMore() == false) { semaphore.release(); // Here we have to retry before abort to avoid a race situation where the other threads have flipped `canSendMore` // condition but they are not going to resume the sending process because this thread still holds the semaphore. - if (canSendMore() == false || semaphore.tryAcquire() == false) { + final boolean changed = canSendMore() || error.get() != null; + if (changed == false || semaphore.tryAcquire() == false) { break; } } @@ -770,9 +771,6 @@ protected void doRun() throws Exception { FileChunk readNextChunk() throws Exception { assert semaphore.availablePermits() == 0; - if (closed) { - throw new IllegalStateException("MultiFileSender was closed"); - } try { if (currentInput == null) { if (remainingFiles.hasNext() == false) { @@ -816,10 +814,7 @@ boolean canSendMore() { @Override public void close() throws IOException { assert semaphore.availablePermits() == 0; - if (closed == false) { - closed = true; - IOUtils.close(recycledBuffers::clear, currentInput, () -> currentInput = null); - } + IOUtils.close(recycledBuffers::clear, currentInput, () -> currentInput = null); } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index a06336031c864..5678b5bb35725 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -107,7 +107,6 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.core.IsNull.notNullValue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; @@ -673,8 +672,7 @@ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference c assertThat(error.get(), notNullValue()); assertThat(error.get().getMessage(), containsString("test chunk exception")); }); - assertThat("no more chunks should be sent", sentChunks.get(), - lessThanOrEqualTo(Math.min(totalChunks, maxConcurrentChunks * 2))); + assertThat("no more chunks should be sent", sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks))); store.close(); } From 97b209994e6d084fcbd5d7eb24ab5af41fbbb51e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 8 Jul 2019 23:24:33 -0400 Subject: [PATCH 08/27] =?UTF-8?q?henning=E2=80=99s=20comment?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../recovery/RecoverySourceHandler.java | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index d91274c99e77b..685a89513f55b 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -415,16 +415,16 @@ void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps phase1FileNames.size(), new ByteSizeValue(totalSizeInBytes), phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSizeInBytes)); final StepListener sendFileInfoStep = new StepListener<>(); - final StepListener sendFileChunkStep = new StepListener<>(); + final StepListener sendFilesStep = new StepListener<>(); final StepListener cleanFilesStep = new StepListener<>(); cancellableThreads.execute(() -> recoveryTarget.receiveFileInfo(phase1FileNames, phase1FileSizes, phase1ExistingFileNames, phase1ExistingFileSizes, translogOps.getAsInt(), sendFileInfoStep)); sendFileInfoStep.whenComplete(r -> - sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps, sendFileChunkStep), listener::onFailure); + sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps, sendFilesStep), listener::onFailure); - sendFileChunkStep.whenComplete(r -> + sendFilesStep.whenComplete(r -> cleanFiles(store, recoverySourceMetadata, translogOps, globalCheckpoint, cleanFilesStep), listener::onFailure); final long totalSize = totalSizeInBytes; @@ -726,10 +726,20 @@ protected void doRun() throws Exception { assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[send file chunk]"); while (true) { assert semaphore.availablePermits() == 0; + cancellableThreads.checkForCancel(); if (error.get() != null) { handleErrorOnSendFiles(store, error.get().v2(), new StoreFileMetaData[]{error.get().v1()}); throw error.get().v2(); } + if (canSendMore() == false) { + semaphore.release(); + // Here we have to retry before abort to avoid a race situation where the other threads have flipped `canSendMore` + // condition but they are not going to resume the sending process because this thread still holds the semaphore. + final boolean changed = canSendMore() || error.get() != null; + if (changed == false || semaphore.tryAcquire() == false) { + break; + } + } final FileChunk chunk = readNextChunk(); if (chunk == null) { semaphore.release(); // allow other threads respond if we are not done yet. @@ -757,15 +767,6 @@ protected void doRun() throws Exception { }) ) ); - if (canSendMore() == false) { - semaphore.release(); - // Here we have to retry before abort to avoid a race situation where the other threads have flipped `canSendMore` - // condition but they are not going to resume the sending process because this thread still holds the semaphore. - final boolean changed = canSendMore() || error.get() != null; - if (changed == false || semaphore.tryAcquire() == false) { - break; - } - } } } From ec2194f775e74530b53ea9f2e59d419c06285ec1 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 8 Jul 2019 23:31:14 -0400 Subject: [PATCH 09/27] move checking error down --- .../indices/recovery/RecoverySourceHandler.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 685a89513f55b..4daac82f93139 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -727,10 +727,6 @@ protected void doRun() throws Exception { while (true) { assert semaphore.availablePermits() == 0; cancellableThreads.checkForCancel(); - if (error.get() != null) { - handleErrorOnSendFiles(store, error.get().v2(), new StoreFileMetaData[]{error.get().v1()}); - throw error.get().v2(); - } if (canSendMore() == false) { semaphore.release(); // Here we have to retry before abort to avoid a race situation where the other threads have flipped `canSendMore` @@ -740,6 +736,10 @@ protected void doRun() throws Exception { break; } } + if (error.get() != null) { + handleErrorOnSendFiles(store, error.get().v2(), new StoreFileMetaData[]{error.get().v1()}); + throw error.get().v2(); + } final FileChunk chunk = readNextChunk(); if (chunk == null) { semaphore.release(); // allow other threads respond if we are not done yet. From e8dfd756e743ed0ae6ea1f67a334482ccd7d9543 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 9 Jul 2019 12:34:20 -0400 Subject: [PATCH 10/27] integrate with AsyncIOProcessor --- .../recovery/PeerRecoverySourceService.java | 2 +- .../recovery/RecoverySourceHandler.java | 133 +++++++------ .../recovery/RemoteRecoveryTargetHandler.java | 2 +- .../recovery/RecoverySourceHandlerTests.java | 187 ++++++++---------- .../index/shard/IndexShardTestCase.java | 4 +- 5 files changed, 156 insertions(+), 172 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java index b9650a641ea48..f53e8edecd9e6 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java @@ -176,7 +176,7 @@ private RecoverySourceHandler createRecoverySourceHandler(StartRecoveryRequest r final RemoteRecoveryTargetHandler recoveryTarget = new RemoteRecoveryTargetHandler(request.recoveryId(), request.shardId(), transportService, request.targetNode(), recoverySettings, throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime)); - handler = new RecoverySourceHandler(shard, recoveryTarget, request, shard.getThreadPool().generic(), + handler = new RecoverySourceHandler(shard, recoveryTarget, request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), recoverySettings.getMaxConcurrentFileChunks()); return handler; } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 4daac82f93139..fb3f48f762234 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -31,7 +31,6 @@ import org.apache.lucene.util.ArrayUtil; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.StepListener; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; @@ -47,7 +46,7 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.AsyncIOProcessor; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.Engine; @@ -68,6 +67,7 @@ import java.io.Closeable; import java.io.IOException; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -79,10 +79,8 @@ import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.Executor; -import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.IntSupplier; import java.util.stream.StreamSupport; @@ -113,12 +111,10 @@ public class RecoverySourceHandler { private final int maxConcurrentFileChunks; private final CancellableThreads cancellableThreads = new CancellableThreads(); private final List resources = new CopyOnWriteArrayList<>(); - private final Executor sendFileExecutor; public RecoverySourceHandler(IndexShard shard, RecoveryTargetHandler recoveryTarget, StartRecoveryRequest request, - Executor sendFileExecutor, int fileChunkSizeInBytes, int maxConcurrentFileChunks) { + int fileChunkSizeInBytes, int maxConcurrentFileChunks) { this.shard = shard; - this.sendFileExecutor = sendFileExecutor; this.recoveryTarget = recoveryTarget; this.request = request; this.shardId = this.request.shardId().id(); @@ -701,77 +697,81 @@ public String toString() { * one of the networking threads which receive/handle the acknowledgments of the current pending file chunk requests. This process will * continue until all chunks are sent and acknowledged. */ - private class MultiFileSender extends ActionRunnable implements Closeable { + private class MultiFileSender extends AsyncIOProcessor implements Closeable { private final Store store; private final IntSupplier translogOps; + private final AtomicBoolean done = new AtomicBoolean(false); + private final ActionListener listener; private final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); - private final Semaphore semaphore = new Semaphore(0); private final Iterator remainingFiles; private StoreFileMetaData currentFile; private InputStreamIndexInput currentInput = null; private long currentChunkPosition = 0; - private final Deque recycledBuffers = ConcurrentCollections.newDeque(); - private final AtomicReference> error = new AtomicReference<>(); + private final Deque recycledBuffers = new ArrayDeque<>(); + private final FileChunkResponse INITIAL_RESPONSE = new FileChunkResponse(SequenceNumbers.UNASSIGNED_SEQ_NO, null, null); MultiFileSender(Store store, IntSupplier translogOps, StoreFileMetaData[] files, ActionListener listener) { - super(ActionListener.notifyOnce(listener)); + super(logger, maxConcurrentFileChunks * 2, shard.getThreadPool().getThreadContext()); this.store = store; this.translogOps = translogOps; this.remainingFiles = Arrays.asList(files).iterator(); + this.listener = ActionListener.wrap( + r -> { + if (done.compareAndSet(false, true)) { + listener.onResponse(r); + } + }, + e -> { + if (done.compareAndSet(false, true)) { + listener.onFailure(e); + } + }); + } + + void start() { + put(INITIAL_RESPONSE, e -> {}); } @Override - protected void doRun() throws Exception { - assert ThreadPool.assertCurrentMethodIsNotCalledRecursively(); - assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[send file chunk]"); - while (true) { - assert semaphore.availablePermits() == 0; - cancellableThreads.checkForCancel(); - if (canSendMore() == false) { - semaphore.release(); - // Here we have to retry before abort to avoid a race situation where the other threads have flipped `canSendMore` - // condition but they are not going to resume the sending process because this thread still holds the semaphore. - final boolean changed = canSendMore() || error.get() != null; - if (changed == false || semaphore.tryAcquire() == false) { - break; + protected void write(List>> responses) { + assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[send file chunks]"); + if (done.get()) { + return; + } + try { + for (Tuple> response : responses) { + if (response.v1() == INITIAL_RESPONSE) { + continue; // not an actual response, a marker to initialize the sending process. + } + requestSeqIdTracker.markSeqNoAsProcessed(response.v1().seqNo); + response.v1().chunk.close(); + if (response.v1().failure != null) { + handleErrorOnSendFiles(store, response.v1().failure, new StoreFileMetaData[]{response.v1().chunk.md}); + throw response.v1().failure; } } - if (error.get() != null) { - handleErrorOnSendFiles(store, error.get().v2(), new StoreFileMetaData[]{error.get().v1()}); - throw error.get().v2(); - } - final FileChunk chunk = readNextChunk(); - if (chunk == null) { - semaphore.release(); // allow other threads respond if we are not done yet. - if (requestSeqIdTracker.getMaxSeqNo() == requestSeqIdTracker.getProcessedCheckpoint() && semaphore.tryAcquire()) { - listener.onResponse(null); + while (requestSeqIdTracker.getMaxSeqNo() - requestSeqIdTracker.getProcessedCheckpoint() < maxConcurrentFileChunks) { + cancellableThreads.checkForCancel(); + final FileChunk chunk = readNextChunk(); + if (chunk == null) { + if (requestSeqIdTracker.getProcessedCheckpoint() == requestSeqIdTracker.getMaxSeqNo()) { + listener.onResponse(null); + } + return; } - break; + final long requestSeqId = requestSeqIdTracker.generateSeqNo(); + cancellableThreads.execute(() -> recoveryTarget.writeFileChunk(chunk.md, chunk.position, chunk.content, chunk.lastChunk, + translogOps.getAsInt(), ActionListener.wrap( + r -> this.put(new FileChunkResponse(requestSeqId, chunk, null), ignored -> {}), + e -> this.put(new FileChunkResponse(requestSeqId, chunk, e), ignored -> {}) + ))); } - final long requestSeqId = requestSeqIdTracker.generateSeqNo(); - cancellableThreads.execute(() -> - recoveryTarget.writeFileChunk(chunk.md, chunk.position, chunk.content, chunk.lastChunk, translogOps.getAsInt(), - ActionListener.wrap( - r -> { - chunk.close(); // release the buffer so we can reuse to reduce allocation - requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); - if (canSendMore() && semaphore.tryAcquire()) { - sendFileExecutor.execute(this); // fork off from the network thread - } - }, - e -> { - if (error.compareAndSet(null, Tuple.tuple(chunk.md, e)) && semaphore.tryAcquire()) { - // have to fork as handleErrorOnSendFiles can read file which should not happen on the network thread. - sendFileExecutor.execute(this); - } - }) - ) - ); + } catch (Exception e) { + listener.onFailure(e); } } - FileChunk readNextChunk() throws Exception { - assert semaphore.availablePermits() == 0; + private FileChunk readNextChunk() throws Exception { try { if (currentInput == null) { if (remainingFiles.hasNext() == false) { @@ -808,13 +808,8 @@ public void close() throws IOException { } } - boolean canSendMore() { - return requestSeqIdTracker.getMaxSeqNo() - requestSeqIdTracker.getProcessedCheckpoint() < maxConcurrentFileChunks; - } - @Override public void close() throws IOException { - assert semaphore.availablePermits() == 0; IOUtils.close(recycledBuffers::clear, currentInput, () -> currentInput = null); } } @@ -840,6 +835,18 @@ public void close() { } } + private static class FileChunkResponse { + final long seqNo; + final FileChunk chunk; + final Exception failure; + + FileChunkResponse(long seqNo, FileChunk chunk, Exception failure) { + this.seqNo = seqNo; + this.chunk = chunk; + this.failure = failure; + } + } + void sendFiles(Store store, StoreFileMetaData[] files, IntSupplier translogOps, ActionListener listener) { ArrayUtil.timSort(files, Comparator.comparingLong(StoreFileMetaData::length)); // send smallest first StepListener wrappedListener = new StepListener<>(); @@ -852,7 +859,7 @@ void sendFiles(Store store, StoreFileMetaData[] files, IntSupplier translogOps, listener.onFailure(e); }); resources.add(multiFileSender); - multiFileSender.run(); + multiFileSender.start(); } private void cleanFiles(Store store, Store.MetadataSnapshot sourceMetadata, IntSupplier translogOps, diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java index bb5457c1a3dca..f6f8baedd9d13 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java @@ -181,7 +181,7 @@ public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesR * would be in to restart file copy again (new deltas) if we have too many translog ops are piling up. */ throttleTimeInNanos), fileChunkRequestOptions, new ActionListenerResponseHandler<>( - ActionListener.map(listener, r -> null), in -> TransportResponse.Empty.INSTANCE)); + ActionListener.map(listener, r -> null), in -> TransportResponse.Empty.INSTANCE, ThreadPool.Names.GENERIC)); } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 5678b5bb35725..b2433bc5e5044 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -34,7 +34,6 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; -import org.apache.lucene.util.SetOnce; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; @@ -107,7 +106,6 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.core.IsNull.notNullValue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyObject; @@ -123,17 +121,18 @@ public class RecoverySourceHandlerTests extends ESTestCase { private final ClusterSettings service = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); private ThreadPool threadPool; - private String sendFileExecutor; + private String recoveryExecutor; @Before public void setUpThreadPool() { if (randomBoolean()) { - sendFileExecutor = "send_files"; - threadPool = new TestThreadPool(getTestName(), new FixedExecutorBuilder( - Settings.EMPTY, sendFileExecutor, randomIntBetween(1, 16), randomIntBetween(16, 128), sendFileExecutor)); - } else { - sendFileExecutor = ThreadPool.Names.GENERIC; + recoveryExecutor = ThreadPool.Names.GENERIC; threadPool = new TestThreadPool(getTestName()); + } else { + // verify that both sending and receiving files can be completed with a single thread + recoveryExecutor = "recovery_executor"; + threadPool = new TestThreadPool(getTestName(), + new FixedExecutorBuilder(Settings.EMPTY, recoveryExecutor, between(1, 16), between(16, 128), recoveryExecutor)); } } @@ -189,17 +188,19 @@ public void close() throws IOException { if (lastChunk) { out.close(); } - listener.onResponse(null); + threadPool.executor(recoveryExecutor).execute(() -> listener.onResponse(null)); } catch (Exception e) { - listener.onFailure(e); + threadPool.executor(recoveryExecutor).execute(() -> listener.onFailure(e)); } } }; - RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, threadPool.executor(sendFileExecutor), + IndexShard shard = mock(IndexShard.class); + when(shard.getThreadPool()).thenReturn(threadPool); + RecoverySourceHandler handler = new RecoverySourceHandler(shard, target, request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 5)); - PlainActionFuture sendFileFuture = new PlainActionFuture<>(); - handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, sendFileFuture); - sendFileFuture.actionGet(); + PlainActionFuture sendFilesFuture = new PlainActionFuture<>(); + handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, sendFilesFuture); + sendFilesFuture.actionGet(); Store.MetadataSnapshot targetStoreMetadata = targetStore.getMetadata(null); Store.RecoveryDiff recoveryDiff = targetStoreMetadata.recoveryDiff(metadata); assertEquals(metas.size(), recoveryDiff.identical.size()); @@ -255,8 +256,8 @@ public void indexTranslogOperations(List operations, int tot checkpointOnTarget.set(randomLongBetween(checkpointOnTarget.get(), Long.MAX_VALUE)); listener.onResponse(checkpointOnTarget.get()); } }; - RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), - request, threadPool.executor(sendFileExecutor), fileChunkSizeInBytes, between(1, 10)); + RecoverySourceHandler handler = new RecoverySourceHandler( + shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), request, fileChunkSizeInBytes, between(1, 10)); PlainActionFuture future = new PlainActionFuture<>(); handler.phase2(startingSeqNo, endingSeqNo, newTranslogSnapshot(operations, Collections.emptyList()), randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, randomNonNegativeLong(), future); @@ -295,9 +296,8 @@ public void indexTranslogOperations(List operations, int tot } } }; - RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), - request, threadPool.executor(sendFileExecutor), - fileChunkSizeInBytes, between(1, 10)); + RecoverySourceHandler handler = new RecoverySourceHandler( + shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), request, fileChunkSizeInBytes, between(1, 10)); PlainActionFuture future = new PlainActionFuture<>(); final long startingSeqNo = randomLongBetween(0, ops.size() - 1L); final long endingSeqNo = randomLongBetween(startingSeqNo, ops.size() - 1L); @@ -385,7 +385,9 @@ public void close() throws IOException { } } }; - RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, threadPool.executor(sendFileExecutor), + IndexShard shard = mock(IndexShard.class); + when(shard.getThreadPool()).thenReturn(threadPool); + RecoverySourceHandler handler = new RecoverySourceHandler(shard, target, request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 8)) { @Override protected void failEngine(IOException cause) { @@ -393,22 +395,11 @@ protected void failEngine(IOException cause) { failedEngine.set(true); } }; - SetOnce error = new SetOnce<>(); - CountDownLatch latch = new CountDownLatch(1); - handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, - ActionListener.wrap( - r -> { - error.set(null); - latch.countDown(); - }, - e -> { - error.set(e); - latch.countDown(); - } - )); - latch.await(); - assertNotNull(error.get()); - assertNotNull(ExceptionsHelper.unwrapCorruption(error.get())); + + PlainActionFuture sendFilesFuture = new PlainActionFuture<>(); + handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, sendFilesFuture); + Exception ex = expectThrows(Exception.class, sendFilesFuture::actionGet); + assertNotNull(ExceptionsHelper.unwrapCorruption(ex)); assertTrue(failedEngine.get()); IOUtils.close(store, targetStore); } @@ -442,14 +433,18 @@ public void testHandleExceptionOnSendFiles() throws Throwable { @Override public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { - if (throwCorruptedIndexException) { - listener.onFailure(new RuntimeException(new CorruptIndexException("foo", "bar"))); - } else { - listener.onFailure(new RuntimeException("boom")); - } + threadPool.executor(recoveryExecutor).execute(() -> { + if (throwCorruptedIndexException) { + listener.onFailure(new RuntimeException(new CorruptIndexException("foo", "bar"))); + } else { + listener.onFailure(new RuntimeException("boom")); + } + }); } }; - RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, threadPool.executor(sendFileExecutor), + IndexShard shard = mock(IndexShard.class); + when(shard.getThreadPool()).thenReturn(threadPool); + RecoverySourceHandler handler = new RecoverySourceHandler(shard, target, request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 10)) { @Override protected void failEngine(IOException cause) { @@ -457,22 +452,17 @@ protected void failEngine(IOException cause) { failedEngine.set(true); } }; - PlainActionFuture sendFileFuture = new PlainActionFuture<>(); - try { - handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, sendFileFuture); - sendFileFuture.actionGet(); - fail("exception index"); - } catch (RuntimeException ex) { - final IOException unwrappedCorruption = ExceptionsHelper.unwrapCorruption(ex); - if (throwCorruptedIndexException) { - assertNotNull(unwrappedCorruption); - assertEquals(ex.getMessage(), "[File corruption occurred on recovery but checksums are ok]"); - } else { - assertNull(unwrappedCorruption); - assertEquals(ex.getMessage(), "boom"); - } + PlainActionFuture sendFilesFuture = new PlainActionFuture<>(); + handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, sendFilesFuture); + Exception ex = expectThrows(Exception.class, sendFilesFuture::actionGet); + final IOException unwrappedCorruption = ExceptionsHelper.unwrapCorruption(ex); + if (throwCorruptedIndexException) { + assertNotNull(unwrappedCorruption); + assertEquals(ex.getMessage(), "[File corruption occurred on recovery but checksums are ok]"); + } else { + assertNull(unwrappedCorruption); + assertEquals(ex.getMessage(), "boom"); } - IOUtils.close(() -> terminate(threadPool), () -> threadPool = null); assertFalse(failedEngine.get()); IOUtils.close(store); } @@ -494,8 +484,9 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE final AtomicBoolean phase2Called = new AtomicBoolean(); final RecoverySourceHandler handler = new RecoverySourceHandler( shard, - mock(RecoveryTargetHandler.class), request, threadPool.executor(sendFileExecutor), - Math.toIntExact(recoverySettings.getChunkSize().getBytes()), + mock(RecoveryTargetHandler.class), + request, + Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 8)) { @Override @@ -557,6 +548,7 @@ public void testCancellationsDoesNotLeakPrimaryPermits() throws Exception { public void testSendFileChunksConcurrently() throws Exception { final IndexShard shard = mock(IndexShard.class); when(shard.state()).thenReturn(IndexShardState.STARTED); + when(shard.getThreadPool()).thenReturn(threadPool); final List unrepliedChunks = new CopyOnWriteArrayList<>(); final AtomicInteger sentChunks = new AtomicInteger(); final TestRecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() { @@ -565,30 +557,20 @@ public void testSendFileChunksConcurrently() throws Exception { public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { final long chunkNumber = chunkNumberGenerator.getAndIncrement(); - Runnable reply = () -> { - logger.info("--> write chunk name={} seq={}, position={}", md.name(), chunkNumber, position); - unrepliedChunks.add(new FileChunkResponse(chunkNumber, listener)); - sentChunks.incrementAndGet(); - }; - if (randomBoolean()) { - threadPool.executor(sendFileExecutor).execute(reply); - } else if (randomBoolean()) { - threadPool.schedule(reply, TimeValue.timeValueNanos(randomIntBetween(1, 100)), sendFileExecutor); - } else { - reply.run(); - } + logger.info("--> write chunk name={} seq={}, position={}", md.name(), chunkNumber, position); + unrepliedChunks.add(new FileChunkResponse(chunkNumber, listener)); + sentChunks.incrementAndGet(); } }; final int maxConcurrentChunks = between(1, 8); final int chunkSize = between(1, 32); final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, getStartRecoveryRequest(), - threadPool.executor(sendFileExecutor), chunkSize, maxConcurrentChunks); Store store = newStore(createTempDir(), false); - List files = generateFiles(store, between(1, 20), () -> between(1, chunkSize * 500)); + List files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20)); int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum(); - PlainActionFuture sendFileFuture = new PlainActionFuture<>(); - handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0, sendFileFuture); + PlainActionFuture sendFilesFuture = new PlainActionFuture<>(); + handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0, sendFilesFuture); assertBusy(() -> { assertThat(sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks))); assertThat(unrepliedChunks, hasSize(sentChunks.get())); @@ -614,19 +596,20 @@ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference c int expectedSentChunks = sentChunks.get() + chunksToSend; int expectedUnrepliedChunks = unrepliedChunks.size() + chunksToSend; - chunksToAck.forEach(c -> c.listener.onResponse(null)); + chunksToAck.forEach(c -> threadPool.executor(recoveryExecutor).execute(() -> c.listener.onResponse(null))); assertBusy(() -> { assertThat(sentChunks.get(), equalTo(expectedSentChunks)); assertThat(unrepliedChunks, hasSize(expectedUnrepliedChunks)); }); } - sendFileFuture.actionGet(); + sendFilesFuture.actionGet(); store.close(); } public void testSendFileChunksStopOnError() throws Exception { final IndexShard shard = mock(IndexShard.class); when(shard.state()).thenReturn(IndexShardState.STARTED); + when(shard.getThreadPool()).thenReturn(threadPool); final List unrepliedChunks = new CopyOnWriteArrayList<>(); final AtomicInteger sentChunks = new AtomicInteger(); final TestRecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() { @@ -635,43 +618,38 @@ public void testSendFileChunksStopOnError() throws Exception { public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { final long chunkNumber = chunkNumberGenerator.getAndIncrement(); - final Runnable reply = () -> { - logger.info("--> write chunk name={} seq={}, position={}", md.name(), chunkNumber, position); - unrepliedChunks.add(new FileChunkResponse(chunkNumber, listener)); - sentChunks.incrementAndGet(); - }; - if (randomBoolean()) { - threadPool.executor(sendFileExecutor).execute(reply); - } else { - reply.run(); - } + logger.info("--> write chunk name={} seq={}, position={}", md.name(), chunkNumber, position); + unrepliedChunks.add(new FileChunkResponse(chunkNumber, listener)); + sentChunks.incrementAndGet(); } }; final int maxConcurrentChunks = between(1, 4); final int chunkSize = between(1, 16); final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, getStartRecoveryRequest(), - threadPool.executor(sendFileExecutor), chunkSize, maxConcurrentChunks); Store store = newStore(createTempDir(), false); List files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20)); int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum(); - SetOnce error = new SetOnce<>(); - handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0, ActionListener.wrap(r -> error.set(null), error::set)); + PlainActionFuture sendFilesFuture = new PlainActionFuture<>(); + handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0, sendFilesFuture); assertBusy(() -> assertThat(sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks)))); List failedChunks = randomSubsetOf(between(1, unrepliedChunks.size()), unrepliedChunks); - failedChunks.forEach(c -> c.listener.onFailure(new RuntimeException("test chunk exception"))); + CountDownLatch latch = new CountDownLatch(1); + failedChunks.forEach(c -> threadPool.executor(recoveryExecutor).execute(() -> { + c.listener.onFailure(new RuntimeException("test chunk exception")); + latch.countDown(); + })); + latch.await(); unrepliedChunks.removeAll(failedChunks); - unrepliedChunks.forEach(c -> { - if (randomBoolean()) { - c.listener.onFailure(new RuntimeException("test")); - } else { - c.listener.onResponse(null); - } - }); - assertBusy(() -> { - assertThat(error.get(), notNullValue()); - assertThat(error.get().getMessage(), containsString("test chunk exception")); - }); + unrepliedChunks.forEach(c -> threadPool.executor(recoveryExecutor).execute(() -> { + if (randomBoolean()) { + c.listener.onFailure(new RuntimeException("test")); + } else { + c.listener.onResponse(null); + } + })); + Exception error = expectThrows(Exception.class, sendFilesFuture::actionGet); + assertThat(error.getMessage(), containsString("test chunk exception")); assertThat("no more chunks should be sent", sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks))); store.close(); } @@ -679,9 +657,8 @@ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference c public void testVerifySeqNoStatsWhenRecoverWithSyncId() throws Exception { IndexShard shard = mock(IndexShard.class); when(shard.state()).thenReturn(IndexShardState.STARTED); - RecoverySourceHandler handler = new RecoverySourceHandler(shard, new TestRecoveryTargetHandler(), getStartRecoveryRequest(), - threadPool.executor(sendFileExecutor), - between(1, 16), between(1, 4)); + RecoverySourceHandler handler = new RecoverySourceHandler( + shard, new TestRecoveryTargetHandler(), getStartRecoveryRequest(), between(1, 16), between(1, 4)); String syncId = UUIDs.randomBase64UUID(); int numDocs = between(0, 1000); 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 68496fc3cbb94..47a8f73ef62c9 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 @@ -635,8 +635,8 @@ protected final void recoverUnstartedReplica(final IndexShard replica, final StartRecoveryRequest request = new StartRecoveryRequest(replica.shardId(), targetAllocationId, pNode, rNode, snapshot, replica.routingEntry().primary(), 0, startingSeqNo); final RecoverySourceHandler recovery = new RecoverySourceHandler(primary, - new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), request, threadPool.generic(), - Math.toIntExact(ByteSizeUnit.MB.toBytes(1)), between(1, 8)); + new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), + request, Math.toIntExact(ByteSizeUnit.MB.toBytes(1)), between(1, 8)); primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable); From 60656728b5d8a693c0fa7694839beb3c207b6689 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 10 Jul 2019 13:55:16 -0400 Subject: [PATCH 11/27] use single buffer --- .../recovery/RecoverySourceHandler.java | 23 ++++--------------- .../indices/recovery/AsyncRecoveryTarget.java | 5 +++- 2 files changed, 9 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index fb3f48f762234..5ff6b72d01f11 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -67,16 +67,13 @@ import java.io.Closeable; import java.io.IOException; -import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; -import java.util.Deque; import java.util.Iterator; import java.util.List; import java.util.Locale; -import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicBoolean; @@ -707,8 +704,8 @@ private class MultiFileSender extends AsyncIOProcessor implem private StoreFileMetaData currentFile; private InputStreamIndexInput currentInput = null; private long currentChunkPosition = 0; - private final Deque recycledBuffers = new ArrayDeque<>(); private final FileChunkResponse INITIAL_RESPONSE = new FileChunkResponse(SequenceNumbers.UNASSIGNED_SEQ_NO, null, null); + private final byte[] buffer = new byte[chunkSizeInBytes]; MultiFileSender(Store store, IntSupplier translogOps, StoreFileMetaData[] files, ActionListener listener) { super(logger, maxConcurrentFileChunks * 2, shard.getThreadPool().getThreadContext()); @@ -744,7 +741,6 @@ protected void write(List>> respons continue; // not an actual response, a marker to initialize the sending process. } requestSeqIdTracker.markSeqNoAsProcessed(response.v1().seqNo); - response.v1().chunk.close(); if (response.v1().failure != null) { handleErrorOnSendFiles(store, response.v1().failure, new StoreFileMetaData[]{response.v1().chunk.md}); throw response.v1().failure; @@ -787,7 +783,6 @@ public void close() throws IOException { } }; } - final byte[] buffer = Objects.requireNonNullElseGet(recycledBuffers.pollFirst(), () -> new byte[chunkSizeInBytes]); final int bytesRead = currentInput.read(buffer); if (bytesRead == -1) { throw new CorruptIndexException("file truncated; " + @@ -796,8 +791,7 @@ public void close() throws IOException { final long chunkPosition = currentChunkPosition; currentChunkPosition += bytesRead; final boolean lastChunk = currentChunkPosition == currentFile.length(); - final FileChunk chunk = new FileChunk(currentFile, new BytesArray(buffer, 0, bytesRead), chunkPosition, lastChunk, - () -> recycledBuffers.addFirst(buffer)); + final FileChunk chunk = new FileChunk(currentFile, new BytesArray(buffer, 0, bytesRead), chunkPosition, lastChunk); if (lastChunk) { IOUtils.close(currentInput, () -> currentInput = null); } @@ -810,28 +804,21 @@ public void close() throws IOException { @Override public void close() throws IOException { - IOUtils.close(recycledBuffers::clear, currentInput, () -> currentInput = null); + IOUtils.close(currentInput, () -> currentInput = null); } } - private static class FileChunk implements Releasable { + private static class FileChunk { final StoreFileMetaData md; final BytesReference content; final long position; final boolean lastChunk; - final Releasable onClose; - FileChunk(StoreFileMetaData md, BytesReference content, long position, boolean lastChunk, Releasable onClose) { + FileChunk(StoreFileMetaData md, BytesReference content, long position, boolean lastChunk) { this.md = md; this.content = content; this.position = position; this.lastChunk = lastChunk; - this.onClose = onClose; - } - - @Override - public void close() { - onClose.close(); } } diff --git a/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java b/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java index 6ef9725ccaf80..1f4198d00a433 100644 --- a/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java +++ b/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java @@ -19,7 +19,9 @@ package org.elasticsearch.indices.recovery; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLeases; @@ -81,6 +83,7 @@ public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.Metada @Override public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { - executor.execute(() -> target.writeFileChunk(fileMetaData, position, content, lastChunk, totalTranslogOps, listener)); + final BytesReference copy = new BytesArray(BytesRef.deepCopyOf(content.toBytesRef())); + executor.execute(() -> target.writeFileChunk(fileMetaData, position, copy, lastChunk, totalTranslogOps, listener)); } } From 77d00ef45443bd4d58f4e666e061af2908fb32f5 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 10 Jul 2019 17:07:13 -0400 Subject: [PATCH 12/27] prepare for ccr --- .../indices/recovery/MultiFileTransfer.java | 170 ++++++++++++++++++ .../recovery/RecoverySourceHandler.java | 167 ++++------------- 2 files changed, 206 insertions(+), 131 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java new file mode 100644 index 0000000000000..324478683445e --- /dev/null +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -0,0 +1,170 @@ +/* + * 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.indices.recovery; + +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.util.concurrent.AsyncIOProcessor; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.index.seqno.LocalCheckpointTracker; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.store.StoreFileMetaData; + +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; + +import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; + +/** + * File chunks are sent/requested sequentially by at most one thread at any time. However, the sender/requestor won't wait for the response + * before processing the next file chunk request to reduce the recovery time especially on secure/compressed or high latency communication. + *

+ * The sender/requestor can send up to {@code maxConcurrentFileChunks} file chunk requests without waiting for responses. Since the recovery + * target can receive file chunks out of order, it has to buffer those file chunks in memory and only flush to disk when there's no gap. + * To ensure the recover target never buffers more than {@code maxConcurrentFileChunks} file chunks, we allow the sender/requestor to send + * only up to {@code maxConcurrentFileChunks} file chunk requests from the last flushed (and acknowledged) file chunk. We leverage the local + * checkpoint tracker for this purpose. We generate a new sequence number and assign it to each file chunk request before sending; then mark + * that sequence number as processed when we receive a response for the corresponding file chunk request. With the local checkpoint tracker, + * we know the last acknowledged-flushed file-chunk is a file chunk whose {@code requestSeqId} equals to the local checkpoint because the + * recover target can flush all file chunks up to the local checkpoint. + *

+ * When the number of un-replied file chunk requests reaches the limit (i.e. the gap between the max_seq_no and the local checkpoint is + * greater than {@code maxConcurrentFileChunks}), the sending/requesting thread will abort its execution. That process will be resumed by + * one of the networking threads which receive/handle the responses of the current pending file chunk requests. This process will continue + * until all chunk requests are sent/responded. + */ +public abstract class MultiFileTransfer { + private final AtomicBoolean done = new AtomicBoolean(false); + private final ActionListener listener; + private final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); + private final AsyncIOProcessor> processor; + private final int maxConcurrentFileChunks; + private long fileOffset = 0; + private StoreFileMetaData currentFile = null; + private final Iterator remainingFiles; + + protected MultiFileTransfer(Logger logger, ThreadContext threadContext, ActionListener listener, + int maxConcurrentFileChunks, List files) { + this.maxConcurrentFileChunks = maxConcurrentFileChunks; + this.listener = ActionListener.wrap( + r -> { + if (done.compareAndSet(false, true)) { + listener.onResponse(r); + } + }, + e -> { + if (done.compareAndSet(false, true)) { + listener.onFailure(e); + } + }); + this.processor = new AsyncIOProcessor<>(logger, maxConcurrentFileChunks * 2, threadContext) { + @Override + protected void write(List, Consumer>> items) { + handleItems(items); + } + }; + this.remainingFiles = files.iterator(); + } + + public final void start() { + // put an dummy item to start the processor + processor.put(new FileChunkResponseItem<>(SequenceNumbers.UNASSIGNED_PRIMARY_TERM, null, null, null), e -> {}); + } + + private void handleItems(List, Consumer>> items) { + if (done.get()) { + return; + } + try { + for (Tuple, Consumer> item : items) { + final FileChunkResponseItem resp = item.v1(); + if (resp.requestSeqId == SequenceNumbers.UNASSIGNED_SEQ_NO) { + continue; // not an actual item + } + requestSeqIdTracker.markSeqNoAsProcessed(resp.requestSeqId); + if (resp.failure != null) { + handleError(resp.md, resp.failure); + throw resp.failure; + } + } + while (requestSeqIdTracker.getMaxSeqNo() - requestSeqIdTracker.getProcessedCheckpoint() < maxConcurrentFileChunks) { + if (currentFile == null) { + if (remainingFiles.hasNext()) { + currentFile = remainingFiles.next(); + } else { + if (requestSeqIdTracker.getProcessedCheckpoint() == requestSeqIdTracker.getMaxSeqNo()) { + listener.onResponse(null); + } + return; + } + } + final Request request; + try { + request = prepareNextChunkRequest(currentFile, fileOffset); + } catch (Exception e) { + handleError(currentFile, e); + throw e; + } + final long requestSeqId = requestSeqIdTracker.generateSeqNo(); + final StoreFileMetaData md = this.currentFile; + sendChunkRequest(request, ActionListener.wrap( + r -> processor.put(new FileChunkResponseItem<>(requestSeqId, md, r, null), ignored -> {}), + e -> processor.put(new FileChunkResponseItem<>(requestSeqId, md, null, e), ignored -> {}) + )); + fileOffset += request.length; + if (fileOffset == this.currentFile.length()) { + fileOffset = 0; + this.currentFile = null; + } + } + } catch (Exception e) { + listener.onFailure(e); + } + } + + protected abstract Request prepareNextChunkRequest(StoreFileMetaData md, long offset) throws Exception; + + protected abstract void sendChunkRequest(Request request, ActionListener listener); + + protected abstract void handleError(StoreFileMetaData md, Exception e) throws Exception; + + private static class FileChunkResponseItem { + final long requestSeqId; + final StoreFileMetaData md; + final Resp response; + final Exception failure; + FileChunkResponseItem(long requestSeqId, StoreFileMetaData md, Resp response, Exception failure) { + this.requestSeqId = requestSeqId; + this.md = md; + this.response = response; + this.failure = failure; + } + } + + protected static abstract class ChunkRequest { + final long length; + protected ChunkRequest(long length) { + this.length = length; + } + } +} diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 5ff6b72d01f11..90e9a6510d871 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -38,7 +38,6 @@ import org.elasticsearch.common.StopWatch; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.Loggers; @@ -46,12 +45,10 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; -import org.elasticsearch.common.util.concurrent.AsyncIOProcessor; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.RecoveryEngineException; -import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.seqno.RetentionLeases; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -71,19 +68,15 @@ import java.util.Arrays; import java.util.Collections; import java.util.Comparator; -import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.IntSupplier; import java.util.stream.StreamSupport; -import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; - /** * RecoverySourceHandler handles the three phases of shard recovery, which is * everything relating to copying the segment files as well as sending translog @@ -676,130 +669,53 @@ public String toString() { '}'; } - /** - * File chunks are read/sent sequentially by at most one thread at any time. The sender, however, won't wait for the acknowledgement - * before reading/sending the next chunk to increase the recovery speed especially on secure/compressed or high latency communication. - *

- * The sender can send up to {@code maxConcurrentFileChunks} file chunks without waiting for acknowledgments. Since the recovery target - * can receive file chunks out of order, it has to buffer those file chunks in memory and only flush to disk when there's no gap. - * To ensure the recover target never buffers more than {@code maxConcurrentFileChunks} file chunks, we allow the sender to send only up - * to {@code maxConcurrentFileChunks} file chunks from the last flushed (and acknowledged) file chunk. We leverage the local checkpoint - * tracker for this purpose. We generate a new sequence number and assign it to each file chunk before sending; then mark that sequence - * number as processed when we receive an acknowledgement for the corresponding file chunk request. With the local checkpoint tracker, - * we know the last acknowledged-flushed file-chunk is a file chunk whose {@code requestSeqId} equals to the local checkpoint because - * the recover target can flush all file chunks up to the local checkpoint. - *

- * When the number of un-replied file chunk requests reaches the limit (i.e. the gap between the max_seq_no and the local checkpoint is - * greater than {@code maxConcurrentFileChunks}), the sending thread will abort its execution. The sending process will be resumed by - * one of the networking threads which receive/handle the acknowledgments of the current pending file chunk requests. This process will - * continue until all chunks are sent and acknowledged. - */ - private class MultiFileSender extends AsyncIOProcessor implements Closeable { + private class MultiFileSender extends MultiFileTransfer implements Closeable { private final Store store; private final IntSupplier translogOps; - private final AtomicBoolean done = new AtomicBoolean(false); - private final ActionListener listener; - private final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); - private final Iterator remainingFiles; - private StoreFileMetaData currentFile; private InputStreamIndexInput currentInput = null; - private long currentChunkPosition = 0; - private final FileChunkResponse INITIAL_RESPONSE = new FileChunkResponse(SequenceNumbers.UNASSIGNED_SEQ_NO, null, null); private final byte[] buffer = new byte[chunkSizeInBytes]; MultiFileSender(Store store, IntSupplier translogOps, StoreFileMetaData[] files, ActionListener listener) { - super(logger, maxConcurrentFileChunks * 2, shard.getThreadPool().getThreadContext()); + super(logger, shard.getThreadPool().getThreadContext(), listener, maxConcurrentFileChunks, Arrays.asList(files)); this.store = store; this.translogOps = translogOps; - this.remainingFiles = Arrays.asList(files).iterator(); - this.listener = ActionListener.wrap( - r -> { - if (done.compareAndSet(false, true)) { - listener.onResponse(r); - } - }, - e -> { - if (done.compareAndSet(false, true)) { - listener.onFailure(e); - } - }); - } - - void start() { - put(INITIAL_RESPONSE, e -> {}); } @Override - protected void write(List>> responses) { - assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[send file chunks]"); - if (done.get()) { - return; - } - try { - for (Tuple> response : responses) { - if (response.v1() == INITIAL_RESPONSE) { - continue; // not an actual response, a marker to initialize the sending process. + protected FileChunk prepareNextChunkRequest(StoreFileMetaData md, long offset) throws Exception { + assert Transports.assertNotTransportThread("read file chunk"); + cancellableThreads.checkForCancel(); + if (currentInput == null) { + assert offset == 0 : md + " offset=" + offset; + final IndexInput indexInput = store.directory().openInput(md.name(), IOContext.READONCE); + currentInput = new InputStreamIndexInput(indexInput, md.length()) { + @Override + public void close() throws IOException { + indexInput.close(); // InputStreamIndexInput's close is a noop } - requestSeqIdTracker.markSeqNoAsProcessed(response.v1().seqNo); - if (response.v1().failure != null) { - handleErrorOnSendFiles(store, response.v1().failure, new StoreFileMetaData[]{response.v1().chunk.md}); - throw response.v1().failure; - } - } - while (requestSeqIdTracker.getMaxSeqNo() - requestSeqIdTracker.getProcessedCheckpoint() < maxConcurrentFileChunks) { - cancellableThreads.checkForCancel(); - final FileChunk chunk = readNextChunk(); - if (chunk == null) { - if (requestSeqIdTracker.getProcessedCheckpoint() == requestSeqIdTracker.getMaxSeqNo()) { - listener.onResponse(null); - } - return; - } - final long requestSeqId = requestSeqIdTracker.generateSeqNo(); - cancellableThreads.execute(() -> recoveryTarget.writeFileChunk(chunk.md, chunk.position, chunk.content, chunk.lastChunk, - translogOps.getAsInt(), ActionListener.wrap( - r -> this.put(new FileChunkResponse(requestSeqId, chunk, null), ignored -> {}), - e -> this.put(new FileChunkResponse(requestSeqId, chunk, e), ignored -> {}) - ))); - } - } catch (Exception e) { - listener.onFailure(e); + }; } + final int bytesRead = currentInput.read(buffer); + if (bytesRead == -1) { + throw new CorruptIndexException("file truncated; length=" + md.length() + " offset=" + offset, md.name()); + } + final boolean lastChunk = offset + bytesRead == md.length(); + final FileChunk chunk = new FileChunk(md, new BytesArray(buffer, 0, bytesRead), offset, lastChunk); + if (lastChunk) { + IOUtils.close(currentInput, () -> currentInput = null); + } + return chunk; } - private FileChunk readNextChunk() throws Exception { - try { - if (currentInput == null) { - if (remainingFiles.hasNext() == false) { - return null; - } - currentChunkPosition = 0; - currentFile = remainingFiles.next(); - final IndexInput indexInput = store.directory().openInput(currentFile.name(), IOContext.READONCE); - currentInput = new InputStreamIndexInput(indexInput, currentFile.length()) { - @Override - public void close() throws IOException { - indexInput.close(); //InputStreamIndexInput's close is noop - } - }; - } - final int bytesRead = currentInput.read(buffer); - if (bytesRead == -1) { - throw new CorruptIndexException("file truncated; " + - "length=" + currentFile.length() + " position=" + currentChunkPosition, currentFile.name()); - } - final long chunkPosition = currentChunkPosition; - currentChunkPosition += bytesRead; - final boolean lastChunk = currentChunkPosition == currentFile.length(); - final FileChunk chunk = new FileChunk(currentFile, new BytesArray(buffer, 0, bytesRead), chunkPosition, lastChunk); - if (lastChunk) { - IOUtils.close(currentInput, () -> currentInput = null); - } - return chunk; - } catch (IOException e) { - handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{currentFile}); - throw e; - } + @Override + protected void sendChunkRequest(FileChunk fileChunk, ActionListener listener) { + cancellableThreads.execute(() -> recoveryTarget.writeFileChunk( + fileChunk.md, fileChunk.position, fileChunk.content, fileChunk.lastChunk, translogOps.getAsInt(), listener)); + } + + @Override + protected void handleError(StoreFileMetaData md, Exception e) throws Exception { + handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{md}); } @Override @@ -808,13 +724,14 @@ public void close() throws IOException { } } - private static class FileChunk { + private static class FileChunk extends MultiFileTransfer.ChunkRequest { final StoreFileMetaData md; final BytesReference content; final long position; final boolean lastChunk; FileChunk(StoreFileMetaData md, BytesReference content, long position, boolean lastChunk) { + super(content.length()); this.md = md; this.content = content; this.position = position; @@ -822,22 +739,10 @@ private static class FileChunk { } } - private static class FileChunkResponse { - final long seqNo; - final FileChunk chunk; - final Exception failure; - - FileChunkResponse(long seqNo, FileChunk chunk, Exception failure) { - this.seqNo = seqNo; - this.chunk = chunk; - this.failure = failure; - } - } - void sendFiles(Store store, StoreFileMetaData[] files, IntSupplier translogOps, ActionListener listener) { ArrayUtil.timSort(files, Comparator.comparingLong(StoreFileMetaData::length)); // send smallest first - StepListener wrappedListener = new StepListener<>(); - MultiFileSender multiFileSender = new MultiFileSender(store, translogOps, files, wrappedListener); + final StepListener wrappedListener = new StepListener<>(); + final MultiFileSender multiFileSender = new MultiFileSender(store, translogOps, files, wrappedListener); wrappedListener.whenComplete(r -> { multiFileSender.close(); listener.onResponse(null); From 3a398b4bd18777cbb674edc14744502f557e50bb Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 10 Jul 2019 19:00:30 -0400 Subject: [PATCH 13/27] naming --- .../elasticsearch/indices/recovery/MultiFileTransfer.java | 6 ++++-- .../indices/recovery/RecoverySourceHandler.java | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index 324478683445e..a7f5b008781bb 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -120,7 +120,7 @@ private void handleItems(List, Consumer, Consumer listener); @@ -153,6 +153,7 @@ private static class FileChunkResponseItem { final StoreFileMetaData md; final Resp response; final Exception failure; + FileChunkResponseItem(long requestSeqId, StoreFileMetaData md, Resp response, Exception failure) { this.requestSeqId = requestSeqId; this.md = md; @@ -163,6 +164,7 @@ private static class FileChunkResponseItem { protected static abstract class ChunkRequest { final long length; + protected ChunkRequest(long length) { this.length = length; } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 90e9a6510d871..b5d27aa9885c2 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -682,7 +682,7 @@ private class MultiFileSender extends MultiFileTransfer impleme } @Override - protected FileChunk prepareNextChunkRequest(StoreFileMetaData md, long offset) throws Exception { + protected FileChunk nextChunkRequest(StoreFileMetaData md, long offset) throws Exception { assert Transports.assertNotTransportThread("read file chunk"); cancellableThreads.checkForCancel(); if (currentInput == null) { From 32dccb3c0aaead18cd3019ece59fa2fa97dbdb4c Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 10 Jul 2019 22:55:23 -0400 Subject: [PATCH 14/27] style check --- .../indices/recovery/MultiFileTransfer.java | 13 ++++++------- .../indices/recovery/RecoverySourceHandler.java | 8 ++++++-- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index a7f5b008781bb..2c3255d5cc7ca 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -131,7 +131,7 @@ private void handleItems(List, Consumer processor.put(new FileChunkResponseItem<>(requestSeqId, md, r, null), ignored -> {}), e -> processor.put(new FileChunkResponseItem<>(requestSeqId, md, null, e), ignored -> {}) )); - fileOffset += request.length; + fileOffset += request.sizeInBytes(); if (fileOffset == this.currentFile.length()) { fileOffset = 0; this.currentFile = null; @@ -162,11 +162,10 @@ private static class FileChunkResponseItem { } } - protected static abstract class ChunkRequest { - final long length; - - protected ChunkRequest(long length) { - this.length = length; - } + protected interface ChunkRequest { + /** + * @return the number of bytes of the file chunk request + */ + long sizeInBytes(); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index b5d27aa9885c2..1160a3ce98040 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -724,19 +724,23 @@ public void close() throws IOException { } } - private static class FileChunk extends MultiFileTransfer.ChunkRequest { + private static class FileChunk implements MultiFileTransfer.ChunkRequest { final StoreFileMetaData md; final BytesReference content; final long position; final boolean lastChunk; FileChunk(StoreFileMetaData md, BytesReference content, long position, boolean lastChunk) { - super(content.length()); this.md = md; this.content = content; this.position = position; this.lastChunk = lastChunk; } + + @Override + public long sizeInBytes() { + return content.length(); + } } void sendFiles(Store store, StoreFileMetaData[] files, IntSupplier translogOps, ActionListener listener) { From 2d9aae86d983f9185697e2d6738e1c0d5b8c8406 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 10 Jul 2019 23:17:15 -0400 Subject: [PATCH 15/27] integrate with ccr --- .../indices/recovery/MultiFileTransfer.java | 10 +- .../recovery/RecoverySourceHandler.java | 5 + .../GetCcrRestoreFileChunkAction.java | 2 +- .../GetCcrRestoreFileChunkRequest.java | 6 +- .../xpack/ccr/repository/CcrRepository.java | 143 +++++++----------- 5 files changed, 70 insertions(+), 96 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index 2c3255d5cc7ca..5a52fca592c61 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -106,6 +106,12 @@ private void handleItems(List, Consumer, Consumer listener); + protected abstract void handleResponse(StoreFileMetaData md, Response resp) throws Exception; + protected abstract void handleError(StoreFileMetaData md, Exception e) throws Exception; private static class FileChunkResponseItem { @@ -162,7 +170,7 @@ private static class FileChunkResponseItem { } } - protected interface ChunkRequest { + public interface ChunkRequest { /** * @return the number of bytes of the file chunk request */ diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 1160a3ce98040..42673975d974e 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -713,6 +713,11 @@ protected void sendChunkRequest(FileChunk fileChunk, ActionListener listen fileChunk.md, fileChunk.position, fileChunk.content, fileChunk.lastChunk, translogOps.getAsInt(), listener)); } + @Override + protected void handleResponse(StoreFileMetaData md, Void resp) { + // noop + } + @Override protected void handleError(StoreFileMetaData md, Exception e) throws Exception { handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{md}); 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 5fb169dd3334c..ad64e148e58d1 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 @@ -60,7 +60,7 @@ public TransportGetCcrRestoreFileChunkAction(BigArrays bigArrays, TransportServi @Override protected void doExecute(Task task, GetCcrRestoreFileChunkRequest request, ActionListener listener) { - int bytesRequested = request.getSize(); + int bytesRequested = Math.toIntExact(request.sizeInBytes()); ByteArray array = bigArrays.newByteArray(bytesRequested, false); String fileName = request.getFileName(); String sessionUUID = request.getSessionUUID(); 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 5da0efcb372f5..4741174029f35 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 @@ -11,11 +11,12 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.indices.recovery.MultiFileTransfer; import org.elasticsearch.transport.RemoteClusterAwareRequest; import java.io.IOException; -public class GetCcrRestoreFileChunkRequest extends ActionRequest implements RemoteClusterAwareRequest { +public class GetCcrRestoreFileChunkRequest extends ActionRequest implements RemoteClusterAwareRequest, MultiFileTransfer.ChunkRequest { private final DiscoveryNode node; private final String sessionUUID; @@ -64,7 +65,8 @@ String getFileName() { return fileName; } - int getSize() { + @Override + public long sizeInBytes() { 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 0231681666f5a..93639ce7f699c 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 @@ -12,7 +12,6 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.IndexCommit; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; @@ -32,18 +31,15 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException; import org.elasticsearch.index.seqno.RetentionLeaseNotFoundException; import org.elasticsearch.index.shard.IndexShardRecoveryException; @@ -54,6 +50,7 @@ import org.elasticsearch.index.snapshots.blobstore.SnapshotFiles; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; +import org.elasticsearch.indices.recovery.MultiFileTransfer; import org.elasticsearch.indices.recovery.MultiFileWriter; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; @@ -88,12 +85,11 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.LongConsumer; import java.util.function.Supplier; +import java.util.stream.Collectors; import static org.elasticsearch.index.seqno.RetentionLeaseActions.RETAIN_ALL; -import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; import static org.elasticsearch.xpack.ccr.CcrRetentionLeases.retentionLeaseId; import static org.elasticsearch.xpack.ccr.CcrRetentionLeases.syncAddRetentionLease; import static org.elasticsearch.xpack.ccr.CcrRetentionLeases.syncRenewRetentionLease; @@ -476,98 +472,61 @@ void restoreFiles(Store store) throws IOException { restore(snapshotFiles, store); } - @Override - protected void restoreFiles(List filesToRecover, Store store) throws IOException { - logger.trace("[{}] starting CCR restore of {} files", shardId, filesToRecover); + private void restoreFiles(List files, Store store, ActionListener listener) { + final MultiFileWriter writer = new MultiFileWriter(store, recoveryState.getIndex(), "", logger, () -> { }); + MultiFileTransfer fetcher = + new MultiFileTransfer<>(logger, threadPool.getThreadContext(), + ActionListener.runAfter(listener, writer::close), ccrSettings.getMaxConcurrentFileChunks(), files) { + @Override + protected GetCcrRestoreFileChunkRequest nextChunkRequest(StoreFileMetaData md, long offset) { + final int bytesRequested = Math.toIntExact(Math.min(ccrSettings.getChunkSize().getBytes(), md.length() - offset)); + return new GetCcrRestoreFileChunkRequest(node, sessionUUID, md.name(), bytesRequested); + } - try (MultiFileWriter multiFileWriter = new MultiFileWriter(store, recoveryState.getIndex(), "", logger, () -> { - })) { - final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); - final AtomicReference> error = new AtomicReference<>(); - - for (FileInfo fileInfo : filesToRecover) { - final long fileLength = fileInfo.length(); - long offset = 0; - while (offset < fileLength && error.get() == null) { - final long requestSeqId = requestSeqIdTracker.generateSeqNo(); - try { - requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqId - ccrSettings.getMaxConcurrentFileChunks()); - - if (error.get() != null) { - requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); - break; - } + @Override + protected void sendChunkRequest(GetCcrRestoreFileChunkRequest request, + ActionListener listener) { + final TimeValue timeout = ccrSettings.getRecoveryActionTimeout(); + remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request, ListenerTimeouts.wrapWithTimeout( + threadPool, listener, timeout, ThreadPool.Names.GENERIC, GetCcrRestoreFileChunkAction.NAME)); + } - final int bytesRequested = Math.toIntExact( - Math.min(ccrSettings.getChunkSize().getBytes(), fileLength - offset)); - offset += bytesRequested; - - final GetCcrRestoreFileChunkRequest request = - new GetCcrRestoreFileChunkRequest(node, sessionUUID, fileInfo.name(), bytesRequested); - logger.trace("[{}] [{}] fetching chunk for file [{}], expected offset: {}, size: {}", shardId, snapshotId, - fileInfo.name(), offset, bytesRequested); - - TimeValue timeout = ccrSettings.getRecoveryActionTimeout(); - ActionListener listener = - ListenerTimeouts.wrapWithTimeout(threadPool, ActionListener.wrap( - r -> threadPool.generic().execute(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - error.compareAndSet(null, Tuple.tuple(fileInfo.metadata(), e)); - requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); - } - - @Override - protected void doRun() throws Exception { - final int actualChunkSize = r.getChunk().length(); - logger.trace("[{}] [{}] got response for file [{}], offset: {}, length: {}", shardId, - snapshotId, fileInfo.name(), r.getOffset(), actualChunkSize); - final long nanosPaused = ccrSettings.getRateLimiter().maybePause(actualChunkSize); - throttleListener.accept(nanosPaused); - final boolean lastChunk = r.getOffset() + actualChunkSize >= fileLength; - multiFileWriter.writeFileChunk(fileInfo.metadata(), r.getOffset(), r.getChunk(), lastChunk); - requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); - } - }), - e -> { - error.compareAndSet(null, Tuple.tuple(fileInfo.metadata(), e)); - requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); - } - ), timeout, ThreadPool.Names.GENERIC, GetCcrRestoreFileChunkAction.NAME); - remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request, listener); - } catch (Exception e) { - error.compareAndSet(null, Tuple.tuple(fileInfo.metadata(), e)); - requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); - } + @Override + protected void handleResponse(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); + final boolean lastChunk = r.getOffset() + actualChunkSize >= md.length(); + writer.writeFileChunk(md, r.getOffset(), r.getChunk(), lastChunk); } - } - - try { - requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqIdTracker.getMaxSeqNo()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new ElasticsearchException(e); - } - if (error.get() != null) { - handleError(store, error.get().v2()); - } - } - logger.trace("[{}] completed CCR restore", shardId); + @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 e; + } + }; + fetcher.start(); } - private void handleError(Store store, Exception e) throws IOException { - 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; - } else { - ExceptionsHelper.reThrowIfNotNull(e); - } + @Override + protected void restoreFiles(List filesToRecover, Store store) { + logger.trace("[{}] starting CCR restore of {} files", shardId, filesToRecover); + PlainActionFuture future = new PlainActionFuture<>(); + restoreFiles(filesToRecover.stream().map(FileInfo::metadata).collect(Collectors.toList()), store, future); + future.actionGet(); + logger.trace("[{}] completed CCR restore", shardId); } @Override From d49922f8d1fe6c5f5456462a8b1300744e37e8ea Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 10 Jul 2019 23:17:44 -0400 Subject: [PATCH 16/27] Revert "integrate with ccr" This reverts commit 2d9aae86d983f9185697e2d6738e1c0d5b8c8406. --- .../indices/recovery/MultiFileTransfer.java | 10 +- .../recovery/RecoverySourceHandler.java | 5 - .../GetCcrRestoreFileChunkAction.java | 2 +- .../GetCcrRestoreFileChunkRequest.java | 6 +- .../xpack/ccr/repository/CcrRepository.java | 143 +++++++++++------- 5 files changed, 96 insertions(+), 70 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index 5a52fca592c61..2c3255d5cc7ca 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -106,12 +106,6 @@ private void handleItems(List, Consumer, Consumer listener); - protected abstract void handleResponse(StoreFileMetaData md, Response resp) throws Exception; - protected abstract void handleError(StoreFileMetaData md, Exception e) throws Exception; private static class FileChunkResponseItem { @@ -170,7 +162,7 @@ private static class FileChunkResponseItem { } } - public interface ChunkRequest { + protected interface ChunkRequest { /** * @return the number of bytes of the file chunk request */ diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 42673975d974e..1160a3ce98040 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -713,11 +713,6 @@ protected void sendChunkRequest(FileChunk fileChunk, ActionListener listen fileChunk.md, fileChunk.position, fileChunk.content, fileChunk.lastChunk, translogOps.getAsInt(), listener)); } - @Override - protected void handleResponse(StoreFileMetaData md, Void resp) { - // noop - } - @Override protected void handleError(StoreFileMetaData md, Exception e) throws Exception { handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{md}); 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 ad64e148e58d1..5fb169dd3334c 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 @@ -60,7 +60,7 @@ public TransportGetCcrRestoreFileChunkAction(BigArrays bigArrays, TransportServi @Override protected void doExecute(Task task, GetCcrRestoreFileChunkRequest request, ActionListener listener) { - int bytesRequested = Math.toIntExact(request.sizeInBytes()); + int bytesRequested = request.getSize(); ByteArray array = bigArrays.newByteArray(bytesRequested, false); String fileName = request.getFileName(); String sessionUUID = request.getSessionUUID(); 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 4741174029f35..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 @@ -11,12 +11,11 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.indices.recovery.MultiFileTransfer; import org.elasticsearch.transport.RemoteClusterAwareRequest; import java.io.IOException; -public class GetCcrRestoreFileChunkRequest extends ActionRequest implements RemoteClusterAwareRequest, MultiFileTransfer.ChunkRequest { +public class GetCcrRestoreFileChunkRequest extends ActionRequest implements RemoteClusterAwareRequest { private final DiscoveryNode node; private final String sessionUUID; @@ -65,8 +64,7 @@ String getFileName() { return fileName; } - @Override - public long sizeInBytes() { + 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 93639ce7f699c..0231681666f5a 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 @@ -12,6 +12,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.IndexCommit; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; @@ -31,15 +32,18 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException; import org.elasticsearch.index.seqno.RetentionLeaseNotFoundException; import org.elasticsearch.index.shard.IndexShardRecoveryException; @@ -50,7 +54,6 @@ import org.elasticsearch.index.snapshots.blobstore.SnapshotFiles; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; -import org.elasticsearch.indices.recovery.MultiFileTransfer; import org.elasticsearch.indices.recovery.MultiFileWriter; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; @@ -85,11 +88,12 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.LongConsumer; import java.util.function.Supplier; -import java.util.stream.Collectors; import static org.elasticsearch.index.seqno.RetentionLeaseActions.RETAIN_ALL; +import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; import static org.elasticsearch.xpack.ccr.CcrRetentionLeases.retentionLeaseId; import static org.elasticsearch.xpack.ccr.CcrRetentionLeases.syncAddRetentionLease; import static org.elasticsearch.xpack.ccr.CcrRetentionLeases.syncRenewRetentionLease; @@ -472,63 +476,100 @@ void restoreFiles(Store store) throws IOException { restore(snapshotFiles, store); } - private void restoreFiles(List files, Store store, ActionListener listener) { - final MultiFileWriter writer = new MultiFileWriter(store, recoveryState.getIndex(), "", logger, () -> { }); - MultiFileTransfer fetcher = - new MultiFileTransfer<>(logger, threadPool.getThreadContext(), - ActionListener.runAfter(listener, writer::close), ccrSettings.getMaxConcurrentFileChunks(), files) { - @Override - protected GetCcrRestoreFileChunkRequest nextChunkRequest(StoreFileMetaData md, long offset) { - final int bytesRequested = Math.toIntExact(Math.min(ccrSettings.getChunkSize().getBytes(), md.length() - offset)); - return new GetCcrRestoreFileChunkRequest(node, sessionUUID, md.name(), bytesRequested); - } - - @Override - protected void sendChunkRequest(GetCcrRestoreFileChunkRequest request, - ActionListener listener) { - final TimeValue timeout = ccrSettings.getRecoveryActionTimeout(); - remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request, ListenerTimeouts.wrapWithTimeout( - threadPool, listener, timeout, ThreadPool.Names.GENERIC, GetCcrRestoreFileChunkAction.NAME)); - } - - @Override - protected void handleResponse(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); - final boolean lastChunk = r.getOffset() + actualChunkSize >= md.length(); - writer.writeFileChunk(md, r.getOffset(), r.getChunk(), lastChunk); - } + @Override + protected void restoreFiles(List filesToRecover, Store store) throws IOException { + logger.trace("[{}] starting CCR restore of {} files", shardId, filesToRecover); - @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); + try (MultiFileWriter multiFileWriter = new MultiFileWriter(store, recoveryState.getIndex(), "", logger, () -> { + })) { + final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); + final AtomicReference> error = new AtomicReference<>(); + + for (FileInfo fileInfo : filesToRecover) { + final long fileLength = fileInfo.length(); + long offset = 0; + while (offset < fileLength && error.get() == null) { + final long requestSeqId = requestSeqIdTracker.generateSeqNo(); + try { + requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqId - ccrSettings.getMaxConcurrentFileChunks()); + + if (error.get() != null) { + requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); + break; } - throw corruptIndexException; + + final int bytesRequested = Math.toIntExact( + Math.min(ccrSettings.getChunkSize().getBytes(), fileLength - offset)); + offset += bytesRequested; + + final GetCcrRestoreFileChunkRequest request = + new GetCcrRestoreFileChunkRequest(node, sessionUUID, fileInfo.name(), bytesRequested); + logger.trace("[{}] [{}] fetching chunk for file [{}], expected offset: {}, size: {}", shardId, snapshotId, + fileInfo.name(), offset, bytesRequested); + + TimeValue timeout = ccrSettings.getRecoveryActionTimeout(); + ActionListener listener = + ListenerTimeouts.wrapWithTimeout(threadPool, ActionListener.wrap( + r -> threadPool.generic().execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + error.compareAndSet(null, Tuple.tuple(fileInfo.metadata(), e)); + requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); + } + + @Override + protected void doRun() throws Exception { + final int actualChunkSize = r.getChunk().length(); + logger.trace("[{}] [{}] got response for file [{}], offset: {}, length: {}", shardId, + snapshotId, fileInfo.name(), r.getOffset(), actualChunkSize); + final long nanosPaused = ccrSettings.getRateLimiter().maybePause(actualChunkSize); + throttleListener.accept(nanosPaused); + final boolean lastChunk = r.getOffset() + actualChunkSize >= fileLength; + multiFileWriter.writeFileChunk(fileInfo.metadata(), r.getOffset(), r.getChunk(), lastChunk); + requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); + } + }), + e -> { + error.compareAndSet(null, Tuple.tuple(fileInfo.metadata(), e)); + requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); + } + ), timeout, ThreadPool.Names.GENERIC, GetCcrRestoreFileChunkAction.NAME); + remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request, listener); + } catch (Exception e) { + error.compareAndSet(null, Tuple.tuple(fileInfo.metadata(), e)); + requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId); } - throw e; } - }; - fetcher.start(); - } + } + + try { + requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqIdTracker.getMaxSeqNo()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ElasticsearchException(e); + } + if (error.get() != null) { + handleError(store, error.get().v2()); + } + } - @Override - protected void restoreFiles(List filesToRecover, Store store) { - logger.trace("[{}] starting CCR restore of {} files", shardId, filesToRecover); - PlainActionFuture future = new PlainActionFuture<>(); - restoreFiles(filesToRecover.stream().map(FileInfo::metadata).collect(Collectors.toList()), store, future); - future.actionGet(); logger.trace("[{}] completed CCR restore", shardId); } + private void handleError(Store store, Exception e) throws IOException { + 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; + } else { + ExceptionsHelper.reThrowIfNotNull(e); + } + } + @Override protected InputStream fileInputStream(FileInfo fileInfo) { throw new UnsupportedOperationException(); From a41592c188de4d5fe58f960f61747b4fade16d56 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 11 Jul 2019 08:37:26 -0400 Subject: [PATCH 17/27] =?UTF-8?q?Henning=E2=80=99s=20comments?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../indices/recovery/MultiFileTransfer.java | 74 +++---- .../recovery/PeerRecoverySourceService.java | 2 +- .../recovery/RecoverySourceHandler.java | 127 ++++++------ .../recovery/RecoverySourceHandlerTests.java | 181 +++++++----------- .../index/shard/IndexShardTestCase.java | 2 +- 5 files changed, 173 insertions(+), 213 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index 2c3255d5cc7ca..97a3250af67bf 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -24,16 +24,18 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.util.concurrent.AsyncIOProcessor; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.seqno.LocalCheckpointTracker; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.store.StoreFileMetaData; +import java.io.Closeable; +import java.io.IOException; import java.util.Iterator; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; /** * File chunks are sent/requested sequentially by at most one thread at any time. However, the sender/requestor won't wait for the response @@ -53,31 +55,20 @@ * one of the networking threads which receive/handle the responses of the current pending file chunk requests. This process will continue * until all chunk requests are sent/responded. */ -public abstract class MultiFileTransfer { - private final AtomicBoolean done = new AtomicBoolean(false); +public abstract class MultiFileTransfer implements Closeable { + private boolean done = false; private final ActionListener listener; private final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); private final AsyncIOProcessor> processor; private final int maxConcurrentFileChunks; - private long fileOffset = 0; private StoreFileMetaData currentFile = null; private final Iterator remainingFiles; protected MultiFileTransfer(Logger logger, ThreadContext threadContext, ActionListener listener, int maxConcurrentFileChunks, List files) { this.maxConcurrentFileChunks = maxConcurrentFileChunks; - this.listener = ActionListener.wrap( - r -> { - if (done.compareAndSet(false, true)) { - listener.onResponse(r); - } - }, - e -> { - if (done.compareAndSet(false, true)) { - listener.onFailure(e); - } - }); - this.processor = new AsyncIOProcessor<>(logger, maxConcurrentFileChunks * 2, threadContext) { + this.listener = listener; + this.processor = new AsyncIOProcessor<>(logger, maxConcurrentFileChunks, threadContext) { @Override protected void write(List, Consumer>> items) { handleItems(items); @@ -87,18 +78,21 @@ protected void write(List, Consumer(SequenceNumbers.UNASSIGNED_PRIMARY_TERM, null, null, null), e -> {}); + addItem(UNASSIGNED_SEQ_NO, null, null, null); // put an dummy item to start the processor + } + + private void addItem(long requestSeqId, StoreFileMetaData md, Response response, Exception failure) { + processor.put(new FileChunkResponseItem<>(requestSeqId, md, response, failure), e -> { assert e == null : e; }); } private void handleItems(List, Consumer>> items) { - if (done.get()) { + if (done) { return; } try { for (Tuple, Consumer> item : items) { final FileChunkResponseItem resp = item.v1(); - if (resp.requestSeqId == SequenceNumbers.UNASSIGNED_SEQ_NO) { + if (resp.requestSeqId == UNASSIGNED_SEQ_NO) { continue; // not an actual item } requestSeqIdTracker.markSeqNoAsProcessed(resp.requestSeqId); @@ -111,16 +105,17 @@ private void handleItems(List, Consumer, Consumer processor.put(new FileChunkResponseItem<>(requestSeqId, md, r, null), ignored -> {}), - e -> processor.put(new FileChunkResponseItem<>(requestSeqId, md, null, e), ignored -> {}) - )); - fileOffset += request.sizeInBytes(); - if (fileOffset == this.currentFile.length()) { - fileOffset = 0; + r -> addItem(requestSeqId, md, r, null), + e -> addItem(requestSeqId, md, null, e))); + if (request.lastChunk()) { this.currentFile = null; } } } catch (Exception e) { - listener.onFailure(e); + onCompleted(e); } } - protected abstract Request nextChunkRequest(StoreFileMetaData md, long offset) throws Exception; + private void onCompleted(Exception failure) { + if (done == false) { + done = true; + ActionListener.completeWith(listener, () -> { + IOUtils.close(failure, this); + return null; + }); + } + } + + /** + * This method is called when starting sending/requesting a new file. Subclasses should override + * this method to reset the file offset of close the previous file and open a new file if needed. + */ + protected abstract void onNewFile(StoreFileMetaData md) throws IOException; + + protected abstract Request nextChunkRequest(StoreFileMetaData md) throws Exception; protected abstract void sendChunkRequest(Request request, ActionListener listener); @@ -164,8 +172,8 @@ private static class FileChunkResponseItem { protected interface ChunkRequest { /** - * @return the number of bytes of the file chunk request + * @return {@code true} if this chunk request is the last chunk of the current file */ - long sizeInBytes(); + boolean lastChunk(); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java index f53e8edecd9e6..686ccc799eef0 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java @@ -176,7 +176,7 @@ private RecoverySourceHandler createRecoverySourceHandler(StartRecoveryRequest r final RemoteRecoveryTargetHandler recoveryTarget = new RemoteRecoveryTargetHandler(request.recoveryId(), request.shardId(), transportService, request.targetNode(), recoverySettings, throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime)); - handler = new RecoverySourceHandler(shard, recoveryTarget, request, + handler = new RecoverySourceHandler(shard, recoveryTarget, shard.getThreadPool(), request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), recoverySettings.getMaxConcurrentFileChunks()); return handler; } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 1160a3ce98040..300b17596704a 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -46,6 +46,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.FutureUtils; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.RecoveryEngineException; @@ -99,13 +100,15 @@ public class RecoverySourceHandler { private final int chunkSizeInBytes; private final RecoveryTargetHandler recoveryTarget; private final int maxConcurrentFileChunks; + private final ThreadPool threadPool; private final CancellableThreads cancellableThreads = new CancellableThreads(); private final List resources = new CopyOnWriteArrayList<>(); - public RecoverySourceHandler(IndexShard shard, RecoveryTargetHandler recoveryTarget, StartRecoveryRequest request, - int fileChunkSizeInBytes, int maxConcurrentFileChunks) { + public RecoverySourceHandler(IndexShard shard, RecoveryTargetHandler recoveryTarget, ThreadPool threadPool, + StartRecoveryRequest request, int fileChunkSizeInBytes, int maxConcurrentFileChunks) { this.shard = shard; this.recoveryTarget = recoveryTarget; + this.threadPool = threadPool; this.request = request; this.shardId = this.request.shardId().id(); this.logger = Loggers.getLogger(getClass(), request.shardId(), "recover to " + request.targetNode().getName()); @@ -669,61 +672,6 @@ public String toString() { '}'; } - private class MultiFileSender extends MultiFileTransfer implements Closeable { - private final Store store; - private final IntSupplier translogOps; - private InputStreamIndexInput currentInput = null; - private final byte[] buffer = new byte[chunkSizeInBytes]; - - MultiFileSender(Store store, IntSupplier translogOps, StoreFileMetaData[] files, ActionListener listener) { - super(logger, shard.getThreadPool().getThreadContext(), listener, maxConcurrentFileChunks, Arrays.asList(files)); - this.store = store; - this.translogOps = translogOps; - } - - @Override - protected FileChunk nextChunkRequest(StoreFileMetaData md, long offset) throws Exception { - assert Transports.assertNotTransportThread("read file chunk"); - cancellableThreads.checkForCancel(); - if (currentInput == null) { - assert offset == 0 : md + " offset=" + offset; - final IndexInput indexInput = store.directory().openInput(md.name(), IOContext.READONCE); - currentInput = new InputStreamIndexInput(indexInput, md.length()) { - @Override - public void close() throws IOException { - indexInput.close(); // InputStreamIndexInput's close is a noop - } - }; - } - final int bytesRead = currentInput.read(buffer); - if (bytesRead == -1) { - throw new CorruptIndexException("file truncated; length=" + md.length() + " offset=" + offset, md.name()); - } - final boolean lastChunk = offset + bytesRead == md.length(); - final FileChunk chunk = new FileChunk(md, new BytesArray(buffer, 0, bytesRead), offset, lastChunk); - if (lastChunk) { - IOUtils.close(currentInput, () -> currentInput = null); - } - return chunk; - } - - @Override - protected void sendChunkRequest(FileChunk fileChunk, ActionListener listener) { - cancellableThreads.execute(() -> recoveryTarget.writeFileChunk( - fileChunk.md, fileChunk.position, fileChunk.content, fileChunk.lastChunk, translogOps.getAsInt(), listener)); - } - - @Override - protected void handleError(StoreFileMetaData md, Exception e) throws Exception { - handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{md}); - } - - @Override - public void close() throws IOException { - IOUtils.close(currentInput, () -> currentInput = null); - } - } - private static class FileChunk implements MultiFileTransfer.ChunkRequest { final StoreFileMetaData md; final BytesReference content; @@ -738,22 +686,64 @@ private static class FileChunk implements MultiFileTransfer.ChunkRequest { } @Override - public long sizeInBytes() { - return content.length(); + public boolean lastChunk() { + return lastChunk; } } void sendFiles(Store store, StoreFileMetaData[] files, IntSupplier translogOps, ActionListener listener) { ArrayUtil.timSort(files, Comparator.comparingLong(StoreFileMetaData::length)); // send smallest first - final StepListener wrappedListener = new StepListener<>(); - final MultiFileSender multiFileSender = new MultiFileSender(store, translogOps, files, wrappedListener); - wrappedListener.whenComplete(r -> { - multiFileSender.close(); - listener.onResponse(null); - }, e -> { - IOUtils.closeWhileHandlingException(multiFileSender); - listener.onFailure(e); - }); + + final MultiFileTransfer multiFileSender = + new MultiFileTransfer<>(logger, threadPool.getThreadContext(), listener, maxConcurrentFileChunks, Arrays.asList(files)) { + + final byte[] buffer = new byte[chunkSizeInBytes]; + InputStreamIndexInput currentInput = null; + long offset = 0; + + @Override + protected void onNewFile(StoreFileMetaData md) throws IOException { + offset = 0; + IOUtils.close(currentInput, () -> currentInput = null); + final IndexInput indexInput = store.directory().openInput(md.name(), IOContext.READONCE); + currentInput = new InputStreamIndexInput(indexInput, md.length()) { + @Override + public void close() throws IOException { + indexInput.close(); // InputStreamIndexInput's close is a noop + } + }; + } + + @Override + protected FileChunk nextChunkRequest(StoreFileMetaData md) throws Exception { + assert Transports.assertNotTransportThread("read file chunk"); + cancellableThreads.checkForCancel(); + final int bytesRead = currentInput.read(buffer); + if (bytesRead == -1) { + throw new CorruptIndexException("file truncated; length=" + md.length() + " offset=" + offset, md.name()); + } + final boolean lastChunk = offset + bytesRead == md.length(); + final FileChunk chunk = new FileChunk(md, new BytesArray(buffer, 0, bytesRead), offset, lastChunk); + offset += bytesRead; + return chunk; + } + + @Override + protected void sendChunkRequest(FileChunk request, ActionListener listener) { + cancellableThreads.execute(() -> recoveryTarget.writeFileChunk( + request.md, request.position, request.content, request.lastChunk, translogOps.getAsInt(), listener)); + } + + @Override + protected void handleError(StoreFileMetaData md, Exception e) throws Exception { + handleErrorOnSendFiles(store, e, new StoreFileMetaData[]{md}); + } + + @Override + public void close() throws IOException { + IOUtils.close(currentInput, () -> currentInput = null); + } + }; resources.add(multiFileSender); multiFileSender.start(); } @@ -810,5 +800,4 @@ private void handleErrorOnSendFiles(Store store, Exception e, StoreFileMetaData[ protected void failEngine(IOException cause) { shard.failShard("recovery", cause); } - } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index b2433bc5e5044..dcd3db44dc594 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -32,11 +32,11 @@ import org.apache.lucene.store.BaseDirectoryWrapper; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.BytesRefIterator; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.LatchedActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -95,6 +95,7 @@ import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -106,6 +107,7 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.instanceOf; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyObject; @@ -121,18 +123,18 @@ public class RecoverySourceHandlerTests extends ESTestCase { private final ClusterSettings service = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); private ThreadPool threadPool; - private String recoveryExecutor; + private Executor recoveryExecutor; @Before public void setUpThreadPool() { if (randomBoolean()) { - recoveryExecutor = ThreadPool.Names.GENERIC; threadPool = new TestThreadPool(getTestName()); + recoveryExecutor = threadPool.generic(); } else { // verify that both sending and receiving files can be completed with a single thread - recoveryExecutor = "recovery_executor"; threadPool = new TestThreadPool(getTestName(), - new FixedExecutorBuilder(Settings.EMPTY, recoveryExecutor, between(1, 16), between(16, 128), recoveryExecutor)); + new FixedExecutorBuilder(Settings.EMPTY, "recovery_executor", between(1, 16), between(16, 128), "recovery_executor")); + recoveryExecutor = threadPool.executor("recovery_executor"); } } @@ -165,39 +167,19 @@ public void testSendFiles() throws Throwable { metas.add(md); } Store targetStore = newStore(createTempDir()); + MultiFileWriter multiFileWriter = new MultiFileWriter(targetStore, mock(RecoveryState.Index.class), "", logger, () -> {}); RecoveryTargetHandler target = new TestRecoveryTargetHandler() { - IndexOutputOutputStream out; @Override public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { - try { - if (position == 0) { - out = new IndexOutputOutputStream(targetStore.createVerifyingOutput(md.name(), md, IOContext.DEFAULT)) { - @Override - public void close() throws IOException { - super.close(); - targetStore.directory().sync(Collections.singleton(md.name())); // sync otherwise MDW will mess with it - } - }; - } - final BytesRefIterator iterator = content.iterator(); - BytesRef scratch; - while ((scratch = iterator.next()) != null) { - out.write(scratch.bytes, scratch.offset, scratch.length); - } - if (lastChunk) { - out.close(); - } - threadPool.executor(recoveryExecutor).execute(() -> listener.onResponse(null)); - } catch (Exception e) { - threadPool.executor(recoveryExecutor).execute(() -> listener.onFailure(e)); - } + ActionListener.completeWith(listener, () -> { + multiFileWriter.writeFileChunk(md, position, content, lastChunk); + return null; + }); } }; - IndexShard shard = mock(IndexShard.class); - when(shard.getThreadPool()).thenReturn(threadPool); - RecoverySourceHandler handler = new RecoverySourceHandler(shard, target, request, - Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 5)); + RecoverySourceHandler handler = new RecoverySourceHandler(null, new AsyncRecoveryTarget(target, recoveryExecutor), + threadPool, request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 5)); PlainActionFuture sendFilesFuture = new PlainActionFuture<>(); handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, sendFilesFuture); sendFilesFuture.actionGet(); @@ -208,7 +190,7 @@ public void close() throws IOException { assertEquals(0, recoveryDiff.missing.size()); IndexReader reader = DirectoryReader.open(targetStore.directory()); assertEquals(numDocs, reader.maxDoc()); - IOUtils.close(reader, store, targetStore); + IOUtils.close(reader, store, multiFileWriter, targetStore); } public StartRecoveryRequest getStartRecoveryRequest() throws IOException { @@ -254,10 +236,11 @@ public void indexTranslogOperations(List operations, int tot RetentionLeases retentionLeases, long mappingVersion, ActionListener listener) { shippedOps.addAll(operations); checkpointOnTarget.set(randomLongBetween(checkpointOnTarget.get(), Long.MAX_VALUE)); - listener.onResponse(checkpointOnTarget.get()); } + listener.onResponse(checkpointOnTarget.get()); + } }; - RecoverySourceHandler handler = new RecoverySourceHandler( - shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), request, fileChunkSizeInBytes, between(1, 10)); + RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), + threadPool, request, fileChunkSizeInBytes, between(1, 10)); PlainActionFuture future = new PlainActionFuture<>(); handler.phase2(startingSeqNo, endingSeqNo, newTranslogSnapshot(operations, Collections.emptyList()), randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, randomNonNegativeLong(), future); @@ -296,8 +279,8 @@ public void indexTranslogOperations(List operations, int tot } } }; - RecoverySourceHandler handler = new RecoverySourceHandler( - shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), request, fileChunkSizeInBytes, between(1, 10)); + RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), + threadPool, request, fileChunkSizeInBytes, between(1, 10)); PlainActionFuture future = new PlainActionFuture<>(); final long startingSeqNo = randomLongBetween(0, ops.size() - 1L); final long endingSeqNo = randomLongBetween(startingSeqNo, ops.size() - 1L); @@ -356,52 +339,36 @@ public void testHandleCorruptedIndexOnSendSendFiles() throws Throwable { (p.getFileName().toString().equals("write.lock") || p.getFileName().toString().startsWith("extra")) == false)); Store targetStore = newStore(createTempDir(), false); + MultiFileWriter multiFileWriter = new MultiFileWriter(targetStore, mock(RecoveryState.Index.class), "", logger, () -> {}); RecoveryTargetHandler target = new TestRecoveryTargetHandler() { - IndexOutputOutputStream out; @Override public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { - try { - if (position == 0) { - out = new IndexOutputOutputStream(targetStore.createVerifyingOutput(md.name(), md, IOContext.DEFAULT)) { - @Override - public void close() throws IOException { - super.close(); - targetStore.directory().sync(Collections.singleton(md.name())); // sync otherwise MDW will mess with it - } - }; - } - final BytesRefIterator iterator = content.iterator(); - BytesRef scratch; - while ((scratch = iterator.next()) != null) { - out.write(scratch.bytes, scratch.offset, scratch.length); - } - if (lastChunk) { - out.close(); - } - listener.onResponse(null); - } catch (Exception e) { - IOUtils.closeWhileHandlingException(out, () -> listener.onFailure(e)); - } + ActionListener.completeWith(listener, () -> { + multiFileWriter.writeFileChunk(md, position, content, lastChunk); + return null; + }); } }; - IndexShard shard = mock(IndexShard.class); - when(shard.getThreadPool()).thenReturn(threadPool); - RecoverySourceHandler handler = new RecoverySourceHandler(shard, target, request, - Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 8)) { + RecoverySourceHandler handler = new RecoverySourceHandler(null, new AsyncRecoveryTarget(target, recoveryExecutor), threadPool, + request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 8)) { @Override protected void failEngine(IOException cause) { assertFalse(failedEngine.get()); failedEngine.set(true); } }; - - PlainActionFuture sendFilesFuture = new PlainActionFuture<>(); - handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, sendFilesFuture); - Exception ex = expectThrows(Exception.class, sendFilesFuture::actionGet); - assertNotNull(ExceptionsHelper.unwrapCorruption(ex)); + SetOnce sendFilesError = new SetOnce<>(); + CountDownLatch latch = new CountDownLatch(1); + handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0, + new LatchedActionListener<>(ActionListener.wrap(r -> sendFilesError.set(null), e -> sendFilesError.set(e)), latch)); + latch.await(); + assertThat(sendFilesError.get(), instanceOf(IOException.class)); + assertNotNull(ExceptionsHelper.unwrapCorruption(sendFilesError.get())); assertTrue(failedEngine.get()); - IOUtils.close(store, targetStore); + // ensure all chunk requests have been completed; otherwise some files on the target are left open. + IOUtils.close(() -> terminate(threadPool), () -> threadPool = null); + IOUtils.close(store, multiFileWriter, targetStore); } @@ -433,19 +400,15 @@ public void testHandleExceptionOnSendFiles() throws Throwable { @Override public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { - threadPool.executor(recoveryExecutor).execute(() -> { - if (throwCorruptedIndexException) { - listener.onFailure(new RuntimeException(new CorruptIndexException("foo", "bar"))); - } else { - listener.onFailure(new RuntimeException("boom")); - } - }); + if (throwCorruptedIndexException) { + listener.onFailure(new RuntimeException(new CorruptIndexException("foo", "bar"))); + } else { + listener.onFailure(new RuntimeException("boom")); + } } }; - IndexShard shard = mock(IndexShard.class); - when(shard.getThreadPool()).thenReturn(threadPool); - RecoverySourceHandler handler = new RecoverySourceHandler(shard, target, request, - Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 10)) { + RecoverySourceHandler handler = new RecoverySourceHandler(null, new AsyncRecoveryTarget(target, recoveryExecutor), threadPool, + request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 10)) { @Override protected void failEngine(IOException cause) { assertFalse(failedEngine.get()); @@ -485,6 +448,7 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE final RecoverySourceHandler handler = new RecoverySourceHandler( shard, mock(RecoveryTargetHandler.class), + threadPool, request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 8)) { @@ -548,7 +512,6 @@ public void testCancellationsDoesNotLeakPrimaryPermits() throws Exception { public void testSendFileChunksConcurrently() throws Exception { final IndexShard shard = mock(IndexShard.class); when(shard.state()).thenReturn(IndexShardState.STARTED); - when(shard.getThreadPool()).thenReturn(threadPool); final List unrepliedChunks = new CopyOnWriteArrayList<>(); final AtomicInteger sentChunks = new AtomicInteger(); final TestRecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() { @@ -564,7 +527,7 @@ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference c }; final int maxConcurrentChunks = between(1, 8); final int chunkSize = between(1, 32); - final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, getStartRecoveryRequest(), + final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, threadPool, getStartRecoveryRequest(), chunkSize, maxConcurrentChunks); Store store = newStore(createTempDir(), false); List files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20)); @@ -596,7 +559,7 @@ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference c int expectedSentChunks = sentChunks.get() + chunksToSend; int expectedUnrepliedChunks = unrepliedChunks.size() + chunksToSend; - chunksToAck.forEach(c -> threadPool.executor(recoveryExecutor).execute(() -> c.listener.onResponse(null))); + chunksToAck.forEach(c -> c.listener.onResponse(null)); assertBusy(() -> { assertThat(sentChunks.get(), equalTo(expectedSentChunks)); assertThat(unrepliedChunks, hasSize(expectedUnrepliedChunks)); @@ -607,9 +570,6 @@ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference c } public void testSendFileChunksStopOnError() throws Exception { - final IndexShard shard = mock(IndexShard.class); - when(shard.state()).thenReturn(IndexShardState.STARTED); - when(shard.getThreadPool()).thenReturn(threadPool); final List unrepliedChunks = new CopyOnWriteArrayList<>(); final AtomicInteger sentChunks = new AtomicInteger(); final TestRecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() { @@ -625,31 +585,34 @@ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference c }; final int maxConcurrentChunks = between(1, 4); final int chunkSize = between(1, 16); - final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, getStartRecoveryRequest(), - chunkSize, maxConcurrentChunks); + final RecoverySourceHandler handler = new RecoverySourceHandler(null, new AsyncRecoveryTarget(recoveryTarget, recoveryExecutor), + threadPool, getStartRecoveryRequest(), chunkSize, maxConcurrentChunks); Store store = newStore(createTempDir(), false); List files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20)); int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum(); - PlainActionFuture sendFilesFuture = new PlainActionFuture<>(); - handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0, sendFilesFuture); + SetOnce sendFilesError = new SetOnce<>(); + CountDownLatch sendFilesLatch = new CountDownLatch(1); + handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0, + new LatchedActionListener<>(ActionListener.wrap(r -> sendFilesError.set(null), e -> sendFilesError.set(e)), sendFilesLatch)); assertBusy(() -> assertThat(sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks)))); List failedChunks = randomSubsetOf(between(1, unrepliedChunks.size()), unrepliedChunks); - CountDownLatch latch = new CountDownLatch(1); - failedChunks.forEach(c -> threadPool.executor(recoveryExecutor).execute(() -> { - c.listener.onFailure(new RuntimeException("test chunk exception")); - latch.countDown(); - })); - latch.await(); + CountDownLatch replyLatch = new CountDownLatch(failedChunks.size()); + failedChunks.forEach(c -> { + c.listener.onFailure(new IllegalStateException("test chunk exception")); + replyLatch.countDown(); + }); + replyLatch.await(); unrepliedChunks.removeAll(failedChunks); - unrepliedChunks.forEach(c -> threadPool.executor(recoveryExecutor).execute(() -> { - if (randomBoolean()) { - c.listener.onFailure(new RuntimeException("test")); - } else { - c.listener.onResponse(null); - } - })); - Exception error = expectThrows(Exception.class, sendFilesFuture::actionGet); - assertThat(error.getMessage(), containsString("test chunk exception")); + unrepliedChunks.forEach(c -> { + if (randomBoolean()) { + c.listener.onFailure(new RuntimeException("test")); + } else { + c.listener.onResponse(null); + } + }); + sendFilesLatch.await(); + assertThat(sendFilesError.get(), instanceOf(IllegalStateException.class)); + assertThat(sendFilesError.get().getMessage(), containsString("test chunk exception")); assertThat("no more chunks should be sent", sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks))); store.close(); } @@ -658,7 +621,7 @@ public void testVerifySeqNoStatsWhenRecoverWithSyncId() throws Exception { IndexShard shard = mock(IndexShard.class); when(shard.state()).thenReturn(IndexShardState.STARTED); RecoverySourceHandler handler = new RecoverySourceHandler( - shard, new TestRecoveryTargetHandler(), getStartRecoveryRequest(), between(1, 16), between(1, 4)); + shard, new TestRecoveryTargetHandler(), threadPool, getStartRecoveryRequest(), between(1, 16), between(1, 4)); String syncId = UUIDs.randomBase64UUID(); int numDocs = between(0, 1000); 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 47a8f73ef62c9..e36f5e3999076 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 @@ -635,7 +635,7 @@ protected final void recoverUnstartedReplica(final IndexShard replica, final StartRecoveryRequest request = new StartRecoveryRequest(replica.shardId(), targetAllocationId, pNode, rNode, snapshot, replica.routingEntry().primary(), 0, startingSeqNo); final RecoverySourceHandler recovery = new RecoverySourceHandler(primary, - new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), + new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), threadPool, request, Math.toIntExact(ByteSizeUnit.MB.toBytes(1)), between(1, 8)); primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable); From 28e266412ead25bc421f43f30026cef8b633bacc Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 11 Jul 2019 16:51:24 -0400 Subject: [PATCH 18/27] unused settings --- .../elasticsearch/indices/recovery/RecoverySourceHandler.java | 1 - .../indices/recovery/RecoverySourceHandlerTests.java | 4 +--- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 300b17596704a..efdcd42aa2ec9 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -46,7 +46,6 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.FutureUtils; -import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.RecoveryEngineException; diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index dcd3db44dc594..fdef75bb1df8e 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -144,9 +144,7 @@ public void tearDownThreadPool() { } public void testSendFiles() throws Throwable { - Settings settings = Settings.builder().put("indices.recovery.concurrent_streams", 1). - put("indices.recovery.concurrent_small_file_streams", 1).build(); - final RecoverySettings recoverySettings = new RecoverySettings(settings, service); + final RecoverySettings recoverySettings = new RecoverySettings(Settings.EMPTY, service); final StartRecoveryRequest request = getStartRecoveryRequest(); Store store = newStore(createTempDir()); Directory dir = store.directory(); From 28cface71ed39016019ed59d5f4fa0a214cd4130 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 11 Jul 2019 16:56:19 -0400 Subject: [PATCH 19/27] wording --- .../org/elasticsearch/indices/recovery/MultiFileTransfer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index 97a3250af67bf..602b7c82d5788 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -146,7 +146,7 @@ private void onCompleted(Exception failure) { /** * This method is called when starting sending/requesting a new file. Subclasses should override - * this method to reset the file offset of close the previous file and open a new file if needed. + * this method to reset the file offset or close the previous file and open a new file if needed. */ protected abstract void onNewFile(StoreFileMetaData md) throws IOException; From 78a8a4e55fbc05a721dcf9b39dd4d015c5a566e2 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 12 Jul 2019 09:17:21 -0400 Subject: [PATCH 20/27] remove not used resp now --- .../indices/recovery/MultiFileTransfer.java | 43 +++++++++---------- .../recovery/RecoverySourceHandler.java | 2 +- 2 files changed, 21 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index 602b7c82d5788..9ae657439aad7 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -55,11 +55,11 @@ * one of the networking threads which receive/handle the responses of the current pending file chunk requests. This process will continue * until all chunk requests are sent/responded. */ -public abstract class MultiFileTransfer implements Closeable { +public abstract class MultiFileTransfer implements Closeable { private boolean done = false; private final ActionListener listener; private final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); - private final AsyncIOProcessor> processor; + private final AsyncIOProcessor processor; private final int maxConcurrentFileChunks; private StoreFileMetaData currentFile = null; private final Iterator remainingFiles; @@ -70,7 +70,7 @@ protected MultiFileTransfer(Logger logger, ThreadContext threadContext, ActionLi this.listener = listener; this.processor = new AsyncIOProcessor<>(logger, maxConcurrentFileChunks, threadContext) { @Override - protected void write(List, Consumer>> items) { + protected void write(List>> items) { handleItems(items); } }; @@ -78,20 +78,20 @@ protected void write(List, Consumer(requestSeqId, md, response, failure), e -> { assert e == null : e; }); + private void addItem(long requestSeqId, StoreFileMetaData md, Exception failure) { + processor.put(new FileChunkResponseItem(requestSeqId, md, failure), e -> { assert e == null : e; }); } - private void handleItems(List, Consumer>> items) { + private void handleItems(List>> items) { if (done) { return; } try { - for (Tuple, Consumer> item : items) { - final FileChunkResponseItem resp = item.v1(); + for (Tuple> item : items) { + final FileChunkResponseItem resp = item.v1(); if (resp.requestSeqId == UNASSIGNED_SEQ_NO) { continue; // not an actual item } @@ -123,8 +123,8 @@ private void handleItems(List, Consumer addItem(requestSeqId, md, r, null), - e -> addItem(requestSeqId, md, null, e))); + r -> addItem(requestSeqId, md, null), + e -> addItem(requestSeqId, md, e))); if (request.lastChunk()) { this.currentFile = null; } @@ -135,13 +135,12 @@ private void handleItems(List, Consumer { - IOUtils.close(failure, this); - return null; - }); - } + assert done == false; + done = true; + ActionListener.completeWith(listener, () -> { + IOUtils.close(failure, this); + return null; + }); } /** @@ -152,20 +151,18 @@ private void onCompleted(Exception failure) { protected abstract Request nextChunkRequest(StoreFileMetaData md) throws Exception; - protected abstract void sendChunkRequest(Request request, ActionListener listener); + protected abstract void sendChunkRequest(Request request, ActionListener listener); protected abstract void handleError(StoreFileMetaData md, Exception e) throws Exception; - private static class FileChunkResponseItem { + private static class FileChunkResponseItem { final long requestSeqId; final StoreFileMetaData md; - final Resp response; final Exception failure; - FileChunkResponseItem(long requestSeqId, StoreFileMetaData md, Resp response, Exception failure) { + FileChunkResponseItem(long requestSeqId, StoreFileMetaData md, Exception failure) { this.requestSeqId = requestSeqId; this.md = md; - this.response = response; this.failure = failure; } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index efdcd42aa2ec9..1041491b40645 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -693,7 +693,7 @@ public boolean lastChunk() { void sendFiles(Store store, StoreFileMetaData[] files, IntSupplier translogOps, ActionListener listener) { ArrayUtil.timSort(files, Comparator.comparingLong(StoreFileMetaData::length)); // send smallest first - final MultiFileTransfer multiFileSender = + final MultiFileTransfer multiFileSender = new MultiFileTransfer<>(logger, threadPool.getThreadContext(), listener, maxConcurrentFileChunks, Arrays.asList(files)) { final byte[] buffer = new byte[chunkSizeInBytes]; From 6f8f7a457174643de32c2f77684a1473c99c7c1e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 12 Jul 2019 09:45:52 -0400 Subject: [PATCH 21/27] add status enum --- .../indices/recovery/MultiFileTransfer.java | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index 9ae657439aad7..c15f4ee85b262 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -20,6 +20,7 @@ package org.elasticsearch.indices.recovery; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.util.concurrent.AsyncIOProcessor; @@ -55,8 +56,9 @@ * one of the networking threads which receive/handle the responses of the current pending file chunk requests. This process will continue * until all chunk requests are sent/responded. */ -public abstract class MultiFileTransfer implements Closeable { - private boolean done = false; +abstract class MultiFileTransfer implements Closeable { + private Status status = Status.PROCESSING; + private final Logger logger; private final ActionListener listener; private final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); private final AsyncIOProcessor processor; @@ -66,6 +68,7 @@ public abstract class MultiFileTransfer listener, int maxConcurrentFileChunks, List files) { + this.logger = logger; this.maxConcurrentFileChunks = maxConcurrentFileChunks; this.listener = listener; this.processor = new AsyncIOProcessor<>(logger, maxConcurrentFileChunks, threadContext) { @@ -86,7 +89,11 @@ private void addItem(long requestSeqId, StoreFileMetaData md, Exception failure) } private void handleItems(List>> items) { - if (done) { + if (status != Status.PROCESSING) { + assert status == Status.FAILED : "must not receive any response after the transfer was completed"; + // These exceptions will be ignored as we record only the first failure, log them for debugging purpose + items.stream().filter(item -> item.v1().failure != null).forEach(item -> + logger.debug(new ParameterizedMessage("failed to transfer file chunk request {}", item.v1().md), item.v1().failure)); return; } try { @@ -135,8 +142,8 @@ private void handleItems(List>> } private void onCompleted(Exception failure) { - assert done == false; - done = true; + assert status == Status.PROCESSING; + status = failure == null ? Status.SUCCESS : Status.FAILED; ActionListener.completeWith(listener, () -> { IOUtils.close(failure, this); return null; @@ -173,4 +180,10 @@ protected interface ChunkRequest { */ boolean lastChunk(); } + + private enum Status { + PROCESSING, + SUCCESS, + FAILED + } } From c2b3fdfb551b2fa90f0a7295bca334c5c972313f Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 12 Jul 2019 12:18:56 -0400 Subject: [PATCH 22/27] read ahead --- .../indices/recovery/MultiFileTransfer.java | 65 ++++++++++++------- .../recovery/RecoverySourceHandler.java | 2 +- 2 files changed, 41 insertions(+), 26 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index c15f4ee85b262..6c22b05203e1d 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -65,6 +65,7 @@ abstract class MultiFileTransfer private final int maxConcurrentFileChunks; private StoreFileMetaData currentFile = null; private final Iterator remainingFiles; + private Tuple readAheadRequest = null; protected MultiFileTransfer(Logger logger, ThreadContext threadContext, ActionListener listener, int maxConcurrentFileChunks, List files) { @@ -91,7 +92,7 @@ private void addItem(long requestSeqId, StoreFileMetaData md, Exception failure) private void handleItems(List>> items) { if (status != Status.PROCESSING) { assert status == Status.FAILED : "must not receive any response after the transfer was completed"; - // These exceptions will be ignored as we record only the first failure, log them for debugging purpose + // These exceptions will be ignored as we record only the first failure, log them for debugging purpose. items.stream().filter(item -> item.v1().failure != null).forEach(item -> logger.debug(new ParameterizedMessage("failed to transfer file chunk request {}", item.v1().md), item.v1().failure)); return; @@ -109,32 +110,24 @@ private void handleItems(List>> } } while (requestSeqIdTracker.getMaxSeqNo() - requestSeqIdTracker.getProcessedCheckpoint() < maxConcurrentFileChunks) { - if (currentFile == null) { - if (remainingFiles.hasNext()) { - currentFile = remainingFiles.next(); - onNewFile(currentFile); - } else { - if (requestSeqIdTracker.getProcessedCheckpoint() == requestSeqIdTracker.getMaxSeqNo()) { - onCompleted(null); - } - return; + final Tuple request = readAheadRequest != null ? readAheadRequest : getNextRequest(); + readAheadRequest = null; + if (request == null) { + assert currentFile == null && remainingFiles.hasNext() == false; + if (requestSeqIdTracker.getMaxSeqNo() == requestSeqIdTracker.getProcessedCheckpoint()) { + onCompleted(null); } - } - final Request request; - try { - request = nextChunkRequest(currentFile); - } catch (Exception e) { - handleError(currentFile, e); - throw e; + return; } final long requestSeqId = requestSeqIdTracker.generateSeqNo(); - final StoreFileMetaData md = this.currentFile; - sendChunkRequest(request, ActionListener.wrap( - r -> addItem(requestSeqId, md, null), - e -> addItem(requestSeqId, md, e))); - if (request.lastChunk()) { - this.currentFile = null; - } + sendChunkRequest(request.v2(), ActionListener.wrap( + r -> addItem(requestSeqId, request.v1(), null), + e -> addItem(requestSeqId, request.v1(), e))); + } + // While we are waiting for the responses, we can prepare the next request in advance + // so we can send it immediately when the responses arrive to reduce the transfer time. + if (readAheadRequest == null) { + readAheadRequest = getNextRequest(); } } catch (Exception e) { onCompleted(e); @@ -150,13 +143,35 @@ private void onCompleted(Exception failure) { }); } + private Tuple getNextRequest() throws Exception { + try { + if (currentFile == null) { + if (remainingFiles.hasNext()) { + currentFile = remainingFiles.next(); + onNewFile(currentFile); + } else { + return null; + } + } + final StoreFileMetaData md = currentFile; + final Request request = nextChunkRequest(md); + if (request.lastChunk()) { + currentFile = null; + } + return Tuple.tuple(md, request); + } catch (Exception e) { + handleError(currentFile, e); + throw e; + } + } + /** * This method is called when starting sending/requesting a new file. Subclasses should override * this method to reset the file offset or close the previous file and open a new file if needed. */ protected abstract void onNewFile(StoreFileMetaData md) throws IOException; - protected abstract Request nextChunkRequest(StoreFileMetaData md) throws Exception; + protected abstract Request nextChunkRequest(StoreFileMetaData md) throws IOException; protected abstract void sendChunkRequest(Request request, ActionListener listener); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 1041491b40645..922ee15d6730a 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -714,7 +714,7 @@ public void close() throws IOException { } @Override - protected FileChunk nextChunkRequest(StoreFileMetaData md) throws Exception { + protected FileChunk nextChunkRequest(StoreFileMetaData md) throws IOException { assert Transports.assertNotTransportThread("read file chunk"); cancellableThreads.checkForCancel(); final int bytesRead = currentInput.read(buffer); From d582b0993c3427383f0db2da46d597bb87a327f9 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 12 Jul 2019 17:02:13 -0400 Subject: [PATCH 23/27] add status to assertion --- .../org/elasticsearch/indices/recovery/MultiFileTransfer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index 6c22b05203e1d..dc836e811527a 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -135,7 +135,7 @@ private void handleItems(List>> } private void onCompleted(Exception failure) { - assert status == Status.PROCESSING; + assert status == Status.PROCESSING : "status [" + status + "] failure [" + failure + "]"; status = failure == null ? Status.SUCCESS : Status.FAILED; ActionListener.completeWith(listener, () -> { IOUtils.close(failure, this); From fb8bb6e9a47afebca8fc0880330e293e2a15ba08 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 12 Jul 2019 22:49:10 -0400 Subject: [PATCH 24/27] notify once --- .../indices/recovery/MultiFileTransfer.java | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index dc836e811527a..8d6a9b280d62b 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -82,7 +82,7 @@ protected void write(List>> ite } public final void start() { - addItem(UNASSIGNED_SEQ_NO, null, null); // put an dummy item to start the processor + addItem(UNASSIGNED_SEQ_NO, null, null); // put a dummy item to start the processor } private void addItem(long requestSeqId, StoreFileMetaData md, Exception failure) { @@ -91,10 +91,10 @@ private void addItem(long requestSeqId, StoreFileMetaData md, Exception failure) private void handleItems(List>> items) { if (status != Status.PROCESSING) { - assert status == Status.FAILED : "must not receive any response after the transfer was completed"; + assert status != Status.SUCCESS : "must not receive any response after the transfer was completed"; // These exceptions will be ignored as we record only the first failure, log them for debugging purpose. items.stream().filter(item -> item.v1().failure != null).forEach(item -> - logger.debug(new ParameterizedMessage("failed to transfer file chunk request {}", item.v1().md), item.v1().failure)); + logger.debug(new ParameterizedMessage("failed to transfer a file chunk request {}", item.v1().md), item.v1().failure)); return; } try { @@ -137,10 +137,17 @@ private void handleItems(List>> private void onCompleted(Exception failure) { assert status == Status.PROCESSING : "status [" + status + "] failure [" + failure + "]"; status = failure == null ? Status.SUCCESS : Status.FAILED; - ActionListener.completeWith(listener, () -> { - IOUtils.close(failure, this); - return null; - }); + try { + try { + IOUtils.close(failure, this); + } catch (Exception e) { + listener.onFailure(e); + return; + } + listener.onResponse(null); + } catch (Exception ignored) { + // we can safely this exception as it happens after we have released the resource and notified the caller. + } } private Tuple getNextRequest() throws Exception { From 8cf85d50103f6ea59bc202458ce1ff803e0a26de Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 13 Jul 2019 09:17:04 -0400 Subject: [PATCH 25/27] missing word in comment --- .../org/elasticsearch/indices/recovery/MultiFileTransfer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index 8d6a9b280d62b..de3db0537a748 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -146,7 +146,7 @@ private void onCompleted(Exception failure) { } listener.onResponse(null); } catch (Exception ignored) { - // we can safely this exception as it happens after we have released the resource and notified the caller. + // we can safely ignore this exception as it happens after we have released the resource and notified the caller. } } From 6d768948069f004c67d98b4e050e4811018605dc Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 15 Jul 2019 08:35:03 -0400 Subject: [PATCH 26/27] let bubble up exception --- .../indices/recovery/MultiFileTransfer.java | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java index de3db0537a748..09366a38a9957 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileTransfer.java @@ -21,6 +21,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.Assertions; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.util.concurrent.AsyncIOProcessor; @@ -91,7 +92,7 @@ private void addItem(long requestSeqId, StoreFileMetaData md, Exception failure) private void handleItems(List>> items) { if (status != Status.PROCESSING) { - assert status != Status.SUCCESS : "must not receive any response after the transfer was completed"; + assert status == Status.FAILED : "must not receive any response after the transfer was completed"; // These exceptions will be ignored as we record only the first failure, log them for debugging purpose. items.stream().filter(item -> item.v1().failure != null).forEach(item -> logger.debug(new ParameterizedMessage("failed to transfer a file chunk request {}", item.v1().md), item.v1().failure)); @@ -135,19 +136,17 @@ private void handleItems(List>> } private void onCompleted(Exception failure) { - assert status == Status.PROCESSING : "status [" + status + "] failure [" + failure + "]"; + if (Assertions.ENABLED && status != Status.PROCESSING) { + throw new AssertionError("invalid status: expected [" + Status.PROCESSING + "] actual [" + status + "]", failure); + } status = failure == null ? Status.SUCCESS : Status.FAILED; try { - try { - IOUtils.close(failure, this); - } catch (Exception e) { - listener.onFailure(e); - return; - } - listener.onResponse(null); - } catch (Exception ignored) { - // we can safely ignore this exception as it happens after we have released the resource and notified the caller. + IOUtils.close(failure, this); + } catch (Exception e) { + listener.onFailure(e); + return; } + listener.onResponse(null); } private Tuple getNextRequest() throws Exception { From 21c2a0a776d57ebac26eeaeefe50c2cbba9a13d2 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 15 Jul 2019 08:41:01 -0400 Subject: [PATCH 27/27] super close --- .../elasticsearch/indices/recovery/RecoverySourceHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 922ee15d6730a..e1353ecb52fac 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -708,7 +708,7 @@ protected void onNewFile(StoreFileMetaData md) throws IOException { currentInput = new InputStreamIndexInput(indexInput, md.length()) { @Override public void close() throws IOException { - indexInput.close(); // InputStreamIndexInput's close is a noop + IOUtils.close(indexInput, super::close); // InputStreamIndexInput's close is a noop } }; }