|
@@ -32,12 +32,15 @@ import org.apache.lucene.index.Term;
|
|
|
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.elasticsearch.ExceptionsHelper;
|
|
|
import org.elasticsearch.Version;
|
|
|
import org.elasticsearch.action.ActionListener;
|
|
|
import org.elasticsearch.action.support.PlainActionFuture;
|
|
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
|
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
|
+import org.elasticsearch.common.Numbers;
|
|
|
import org.elasticsearch.common.UUIDs;
|
|
|
import org.elasticsearch.common.bytes.BytesArray;
|
|
|
import org.elasticsearch.common.bytes.BytesReference;
|
|
@@ -59,6 +62,7 @@ import org.elasticsearch.index.mapper.ParseContext;
|
|
|
import org.elasticsearch.index.mapper.ParsedDocument;
|
|
|
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
|
|
|
import org.elasticsearch.index.mapper.Uid;
|
|
|
+import org.elasticsearch.index.seqno.ReplicationTracker;
|
|
|
import org.elasticsearch.index.seqno.SeqNoStats;
|
|
|
import org.elasticsearch.index.seqno.SequenceNumbers;
|
|
|
import org.elasticsearch.index.shard.IndexShard;
|
|
@@ -75,19 +79,29 @@ import org.elasticsearch.test.IndexSettingsModule;
|
|
|
import org.mockito.ArgumentCaptor;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.io.OutputStream;
|
|
|
import java.nio.file.Path;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collections;
|
|
|
import java.util.Comparator;
|
|
|
import java.util.List;
|
|
|
+import java.util.concurrent.CopyOnWriteArrayList;
|
|
|
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.function.Supplier;
|
|
|
import java.util.stream.Collectors;
|
|
|
+import java.util.zip.CRC32;
|
|
|
|
|
|
import static java.util.Collections.emptyMap;
|
|
|
import static java.util.Collections.emptySet;
|
|
|
+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;
|
|
@@ -109,8 +123,6 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|
|
final RecoverySettings recoverySettings = new RecoverySettings(settings, service);
|
|
|
final StartRecoveryRequest request = getStartRecoveryRequest();
|
|
|
Store store = newStore(createTempDir());
|
|
|
- RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request,
|
|
|
- recoverySettings.getChunkSize().bytesAsInt());
|
|
|
Directory dir = store.directory();
|
|
|
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig());
|
|
|
int numDocs = randomIntBetween(10, 100);
|
|
@@ -129,19 +141,38 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|
|
metas.add(md);
|
|
|
}
|
|
|
Store targetStore = newStore(createTempDir());
|
|
|
- handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), (md) -> {
|
|
|
- try {
|
|
|
- return 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
|
|
|
+ RecoveryTargetHandler target = new TestRecoveryTargetHandler() {
|
|
|
+ IndexOutputOutputStream out;
|
|
|
+ @Override
|
|
|
+ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk,
|
|
|
+ int totalTranslogOps, ActionListener<Void> 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);
|
|
|
}
|
|
|
- };
|
|
|
- } catch (IOException e) {
|
|
|
- throw new RuntimeException(e);
|
|
|
+ if (lastChunk) {
|
|
|
+ out.close();
|
|
|
+ }
|
|
|
+ listener.onResponse(null);
|
|
|
+ } catch (Exception e) {
|
|
|
+ listener.onFailure(e);
|
|
|
+ }
|
|
|
}
|
|
|
- });
|
|
|
+ };
|
|
|
+ RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request,
|
|
|
+ Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 5));
|
|
|
+ handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0);
|
|
|
Store.MetadataSnapshot targetStoreMetadata = targetStore.getMetadata(null);
|
|
|
Store.RecoveryDiff recoveryDiff = targetStoreMetadata.recoveryDiff(metadata);
|
|
|
assertEquals(metas.size(), recoveryDiff.identical.size());
|
|
@@ -176,7 +207,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|
|
when(shard.state()).thenReturn(IndexShardState.STARTED);
|
|
|
final RecoveryTargetHandler recoveryTarget = mock(RecoveryTargetHandler.class);
|
|
|
final RecoverySourceHandler handler =
|
|
|
- new RecoverySourceHandler(shard, recoveryTarget, request, fileChunkSizeInBytes);
|
|
|
+ new RecoverySourceHandler(shard, recoveryTarget, request, fileChunkSizeInBytes, between(1, 10));
|
|
|
final List<Translog.Operation> operations = new ArrayList<>();
|
|
|
final int initialNumberOfDocs = randomIntBetween(16, 64);
|
|
|
for (int i = 0; i < initialNumberOfDocs; i++) {
|
|
@@ -283,14 +314,6 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|
|
Path tempDir = createTempDir();
|
|
|
Store store = newStore(tempDir, false);
|
|
|
AtomicBoolean failedEngine = new AtomicBoolean(false);
|
|
|
- RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request,
|
|
|
- recoverySettings.getChunkSize().bytesAsInt()) {
|
|
|
- @Override
|
|
|
- protected void failEngine(IOException cause) {
|
|
|
- assertFalse(failedEngine.get());
|
|
|
- failedEngine.set(true);
|
|
|
- }
|
|
|
- };
|
|
|
Directory dir = store.directory();
|
|
|
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig());
|
|
|
int numDocs = randomIntBetween(10, 100);
|
|
@@ -313,20 +336,46 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|
|
(p.getFileName().toString().equals("write.lock") ||
|
|
|
p.getFileName().toString().startsWith("extra")) == false));
|
|
|
Store targetStore = newStore(createTempDir(), false);
|
|
|
- try {
|
|
|
- handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), (md) -> {
|
|
|
+ RecoveryTargetHandler target = new TestRecoveryTargetHandler() {
|
|
|
+ IndexOutputOutputStream out;
|
|
|
+ @Override
|
|
|
+ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk,
|
|
|
+ int totalTranslogOps, ActionListener<Void> listener) {
|
|
|
try {
|
|
|
- return new IndexOutputOutputStream(targetStore.createVerifyingOutput(md.name(), md, IOContext.DEFAULT)) {
|
|
|
- @Override
|
|
|
- public void close() throws IOException {
|
|
|
- super.close();
|
|
|
- store.directory().sync(Collections.singleton(md.name())); // sync otherwise MDW will mess with it
|
|
|
- }
|
|
|
- };
|
|
|
- } catch (IOException e) {
|
|
|
- throw new RuntimeException(e);
|
|
|
+ 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));
|
|
|
}
|
|
|
- });
|
|
|
+ }
|
|
|
+ };
|
|
|
+ RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request,
|
|
|
+ Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 8)) {
|
|
|
+ @Override
|
|
|
+ protected void failEngine(IOException cause) {
|
|
|
+ assertFalse(failedEngine.get());
|
|
|
+ failedEngine.set(true);
|
|
|
+ }
|
|
|
+ };
|
|
|
+
|
|
|
+ try {
|
|
|
+ handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0);
|
|
|
fail("corrupted index");
|
|
|
} catch (IOException ex) {
|
|
|
assertNotNull(ExceptionsHelper.unwrapCorruption(ex));
|
|
@@ -342,14 +391,6 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|
|
Path tempDir = createTempDir();
|
|
|
Store store = newStore(tempDir, false);
|
|
|
AtomicBoolean failedEngine = new AtomicBoolean(false);
|
|
|
- RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request,
|
|
|
- recoverySettings.getChunkSize().bytesAsInt()) {
|
|
|
- @Override
|
|
|
- protected void failEngine(IOException cause) {
|
|
|
- assertFalse(failedEngine.get());
|
|
|
- failedEngine.set(true);
|
|
|
- }
|
|
|
- };
|
|
|
Directory dir = store.directory();
|
|
|
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig());
|
|
|
int numDocs = randomIntBetween(10, 100);
|
|
@@ -368,15 +409,27 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|
|
metas.add(md);
|
|
|
}
|
|
|
final boolean throwCorruptedIndexException = randomBoolean();
|
|
|
- Store targetStore = newStore(createTempDir(), false);
|
|
|
- try {
|
|
|
- handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), (md) -> {
|
|
|
+ RecoveryTargetHandler target = new TestRecoveryTargetHandler() {
|
|
|
+ @Override
|
|
|
+ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk,
|
|
|
+ int totalTranslogOps, ActionListener<Void> listener) {
|
|
|
if (throwCorruptedIndexException) {
|
|
|
- throw new RuntimeException(new CorruptIndexException("foo", "bar"));
|
|
|
+ listener.onFailure(new RuntimeException(new CorruptIndexException("foo", "bar")));
|
|
|
} else {
|
|
|
- throw new RuntimeException("boom");
|
|
|
+ listener.onFailure(new RuntimeException("boom"));
|
|
|
}
|
|
|
- });
|
|
|
+ }
|
|
|
+ };
|
|
|
+ RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request,
|
|
|
+ Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 10)) {
|
|
|
+ @Override
|
|
|
+ protected void failEngine(IOException cause) {
|
|
|
+ assertFalse(failedEngine.get());
|
|
|
+ failedEngine.set(true);
|
|
|
+ }
|
|
|
+ };
|
|
|
+ try {
|
|
|
+ handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0);
|
|
|
fail("exception index");
|
|
|
} catch (RuntimeException ex) {
|
|
|
assertNull(ExceptionsHelper.unwrapCorruption(ex));
|
|
@@ -389,7 +442,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|
|
fail("not expected here");
|
|
|
}
|
|
|
assertFalse(failedEngine.get());
|
|
|
- IOUtils.close(store, targetStore);
|
|
|
+ IOUtils.close(store);
|
|
|
}
|
|
|
|
|
|
public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOException {
|
|
@@ -411,7 +464,8 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|
|
shard,
|
|
|
mock(RecoveryTargetHandler.class),
|
|
|
request,
|
|
|
- recoverySettings.getChunkSize().bytesAsInt()) {
|
|
|
+ Math.toIntExact(recoverySettings.getChunkSize().getBytes()),
|
|
|
+ between(1, 8)) {
|
|
|
|
|
|
@Override
|
|
|
public SendFileResult phase1(final IndexCommit snapshot, final Supplier<Integer> translogOps) {
|
|
@@ -468,9 +522,128 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|
|
assertBusy(() -> assertTrue(freed.get()));
|
|
|
}
|
|
|
|
|
|
+ public void testSendFileChunksConcurrently() throws Exception {
|
|
|
+ final IndexShard shard = mock(IndexShard.class);
|
|
|
+ when(shard.state()).thenReturn(IndexShardState.STARTED);
|
|
|
+ final List<FileChunkResponse> unrepliedChunks = new CopyOnWriteArrayList<>();
|
|
|
+ final AtomicInteger sentChunks = new AtomicInteger();
|
|
|
+ final TestRecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() {
|
|
|
+ final AtomicLong chunkNumberGenerator = new AtomicLong();
|
|
|
+ @Override
|
|
|
+ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk,
|
|
|
+ int totalTranslogOps, ActionListener<Void> 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 int maxConcurrentChunks = between(1, 8);
|
|
|
+ final int chunkSize = between(1, 32);
|
|
|
+ final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, getStartRecoveryRequest(),
|
|
|
+ chunkSize, maxConcurrentChunks);
|
|
|
+ Store store = newStore(createTempDir(), false);
|
|
|
+ List<StoreFileMetaData> files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20));
|
|
|
+ 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();
|
|
|
+ assertBusy(() -> {
|
|
|
+ assertThat(sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks)));
|
|
|
+ assertThat(unrepliedChunks, hasSize(sentChunks.get()));
|
|
|
+ });
|
|
|
+
|
|
|
+ List<FileChunkResponse> ackedChunks = new ArrayList<>();
|
|
|
+ while (sentChunks.get() < totalChunks || unrepliedChunks.isEmpty() == false) {
|
|
|
+ List<FileChunkResponse> chunksToAck = randomSubsetOf(between(1, unrepliedChunks.size()), unrepliedChunks);
|
|
|
+ unrepliedChunks.removeAll(chunksToAck);
|
|
|
+ ackedChunks.addAll(chunksToAck);
|
|
|
+ ackedChunks.sort(Comparator.comparing(c -> c.chunkNumber));
|
|
|
+ int checkpoint = -1;
|
|
|
+ for (int i = 0; i < ackedChunks.size(); i++) {
|
|
|
+ if (i != ackedChunks.get(i).chunkNumber) {
|
|
|
+ break;
|
|
|
+ } else {
|
|
|
+ checkpoint = i;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ int chunksToSend = Math.min(
|
|
|
+ totalChunks - sentChunks.get(), // limited by the remaining chunks
|
|
|
+ maxConcurrentChunks - (sentChunks.get() - 1 - checkpoint)); // limited by the buffering chunks
|
|
|
+
|
|
|
+ int expectedSentChunks = sentChunks.get() + chunksToSend;
|
|
|
+ int expectedUnrepliedChunks = unrepliedChunks.size() + chunksToSend;
|
|
|
+ chunksToAck.forEach(c -> c.listener.onResponse(null));
|
|
|
+ assertBusy(() -> {
|
|
|
+ assertThat(sentChunks.get(), equalTo(expectedSentChunks));
|
|
|
+ assertThat(unrepliedChunks, hasSize(expectedUnrepliedChunks));
|
|
|
+ });
|
|
|
+ }
|
|
|
+ sender.join();
|
|
|
+ store.close();
|
|
|
+ }
|
|
|
+
|
|
|
+ public void testSendFileChunksStopOnError() throws Exception {
|
|
|
+ final IndexShard shard = mock(IndexShard.class);
|
|
|
+ when(shard.state()).thenReturn(IndexShardState.STARTED);
|
|
|
+ final List<FileChunkResponse> unrepliedChunks = new CopyOnWriteArrayList<>();
|
|
|
+ final AtomicInteger sentChunks = new AtomicInteger();
|
|
|
+ final TestRecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() {
|
|
|
+ final AtomicLong chunkNumberGenerator = new AtomicLong();
|
|
|
+ @Override
|
|
|
+ public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk,
|
|
|
+ int totalTranslogOps, ActionListener<Void> 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 int maxConcurrentChunks = between(1, 4);
|
|
|
+ final int chunkSize = between(1, 16);
|
|
|
+ final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, getStartRecoveryRequest(),
|
|
|
+ chunkSize, maxConcurrentChunks);
|
|
|
+ Store store = newStore(createTempDir(), false);
|
|
|
+ List<StoreFileMetaData> files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20));
|
|
|
+ int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum();
|
|
|
+ AtomicReference<Exception> 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();
|
|
|
+ assertBusy(() -> assertThat(sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks))));
|
|
|
+ List<FileChunkResponse> failedChunks = randomSubsetOf(between(1, unrepliedChunks.size()), unrepliedChunks);
|
|
|
+ failedChunks.forEach(c -> c.listener.onFailure(new RuntimeException("test chunk exception")));
|
|
|
+ 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"));
|
|
|
+ });
|
|
|
+ assertThat("no more chunks should be sent", sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks)));
|
|
|
+ sender.join();
|
|
|
+ store.close();
|
|
|
+ }
|
|
|
+
|
|
|
private Store newStore(Path path) throws IOException {
|
|
|
return newStore(path, true);
|
|
|
}
|
|
|
+
|
|
|
private Store newStore(Path path, boolean checkIndex) throws IOException {
|
|
|
BaseDirectoryWrapper baseDirectoryWrapper = RecoverySourceHandlerTests.newFSDirectory(path);
|
|
|
if (checkIndex == false) {
|
|
@@ -479,5 +652,68 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|
|
return new Store(shardId, INDEX_SETTINGS, baseDirectoryWrapper, new DummyShardLock(shardId));
|
|
|
}
|
|
|
|
|
|
+ static final class FileChunkResponse {
|
|
|
+ final long chunkNumber;
|
|
|
+ final ActionListener<Void> listener;
|
|
|
+
|
|
|
+ FileChunkResponse(long chunkNumber, ActionListener<Void> listener) {
|
|
|
+ this.chunkNumber = chunkNumber;
|
|
|
+ this.listener = listener;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private List<StoreFileMetaData> generateFiles(Store store, int numFiles, IntSupplier fileSizeSupplier) throws IOException {
|
|
|
+ List<StoreFileMetaData> files = new ArrayList<>();
|
|
|
+ for (int i = 0; i < numFiles; i++) {
|
|
|
+ byte[] buffer = randomByteArrayOfLength(fileSizeSupplier.getAsInt());
|
|
|
+ CRC32 digest = new CRC32();
|
|
|
+ digest.update(buffer, 0, buffer.length);
|
|
|
+ StoreFileMetaData md = new StoreFileMetaData("test-" + i, buffer.length + 8,
|
|
|
+ Store.digestToString(digest.getValue()), org.apache.lucene.util.Version.LATEST);
|
|
|
+ try (OutputStream out = new IndexOutputOutputStream(store.createVerifyingOutput(md.name(), md, IOContext.DEFAULT))) {
|
|
|
+ out.write(buffer);
|
|
|
+ out.write(Numbers.longToBytes(digest.getValue()));
|
|
|
+ }
|
|
|
+ store.directory().sync(Collections.singleton(md.name()));
|
|
|
+ files.add(md);
|
|
|
+ }
|
|
|
+ return files;
|
|
|
+ }
|
|
|
+
|
|
|
+ class TestRecoveryTargetHandler implements RecoveryTargetHandler {
|
|
|
+ @Override
|
|
|
+ public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTranslogOps) {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void finalizeRecovery(long globalCheckpoint) {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void ensureClusterStateVersion(long clusterStateVersion) {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void handoffPrimaryContext(ReplicationTracker.PrimaryContext primaryContext) {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps, long timestamp, long msu) {
|
|
|
+ return 0;
|
|
|
+ }
|
|
|
|
|
|
+ @Override
|
|
|
+ public void receiveFileInfo(List<String> phase1FileNames, List<Long> phase1FileSizes, List<String> phase1ExistingFileNames,
|
|
|
+ List<Long> phase1ExistingFileSizes, int totalTranslogOps) {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaData) {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content, boolean lastChunk,
|
|
|
+ int totalTranslogOps, ActionListener<Void> listener) {
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|