|
@@ -22,16 +22,15 @@ package org.elasticsearch.index.snapshots.blobstore;
|
|
|
import com.google.common.collect.ImmutableMap;
|
|
|
import com.google.common.collect.Iterables;
|
|
|
import com.google.common.collect.Lists;
|
|
|
+import com.google.common.io.ByteStreams;
|
|
|
import org.apache.lucene.index.CorruptIndexException;
|
|
|
import org.apache.lucene.store.*;
|
|
|
import org.apache.lucene.util.BytesRef;
|
|
|
-import org.apache.lucene.util.IOUtils;
|
|
|
import org.elasticsearch.ExceptionsHelper;
|
|
|
import org.elasticsearch.cluster.metadata.SnapshotId;
|
|
|
import org.elasticsearch.common.blobstore.*;
|
|
|
import org.elasticsearch.common.component.AbstractComponent;
|
|
|
import org.elasticsearch.common.inject.Inject;
|
|
|
-import org.elasticsearch.common.io.stream.BytesStreamInput;
|
|
|
import org.elasticsearch.common.lucene.Lucene;
|
|
|
import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
|
|
|
import org.elasticsearch.common.settings.Settings;
|
|
@@ -48,14 +47,12 @@ import org.elasticsearch.indices.IndicesService;
|
|
|
import org.elasticsearch.indices.recovery.RecoveryState;
|
|
|
import org.elasticsearch.repositories.RepositoryName;
|
|
|
|
|
|
-import java.io.ByteArrayOutputStream;
|
|
|
import java.io.FilterInputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.io.InputStream;
|
|
|
+import java.io.OutputStream;
|
|
|
import java.util.*;
|
|
|
import java.util.concurrent.CopyOnWriteArrayList;
|
|
|
-import java.util.concurrent.CountDownLatch;
|
|
|
-import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
import static com.google.common.collect.Lists.newArrayList;
|
|
|
|
|
@@ -64,6 +61,7 @@ import static com.google.common.collect.Lists.newArrayList;
|
|
|
*/
|
|
|
public class BlobStoreIndexShardRepository extends AbstractComponent implements IndexShardRepository {
|
|
|
|
|
|
+ private static final int BUFFER_SIZE = 4096;
|
|
|
private BlobStore blobStore;
|
|
|
|
|
|
private BlobPath basePath;
|
|
@@ -144,8 +142,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
*/
|
|
|
@Override
|
|
|
public void restore(SnapshotId snapshotId, ShardId shardId, ShardId snapshotShardId, RecoveryState recoveryState) {
|
|
|
- RestoreContext snapshotContext = new RestoreContext(snapshotId, shardId, snapshotShardId, recoveryState);
|
|
|
-
|
|
|
+ final RestoreContext snapshotContext = new RestoreContext(snapshotId, shardId, snapshotShardId, recoveryState);
|
|
|
try {
|
|
|
recoveryState.getIndex().startTime(System.currentTimeMillis());
|
|
|
snapshotContext.restore();
|
|
@@ -205,24 +202,25 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
* Serializes snapshot to JSON
|
|
|
*
|
|
|
* @param snapshot snapshot
|
|
|
- * @return JSON representation of the snapshot
|
|
|
- * @throws IOException
|
|
|
+ * @param stream the stream to output the snapshot JSON represetation to
|
|
|
+ * @throws IOException if an IOException occurs
|
|
|
*/
|
|
|
- public static byte[] writeSnapshot(BlobStoreIndexShardSnapshot snapshot) throws IOException {
|
|
|
- XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON).prettyPrint();
|
|
|
+ public static void writeSnapshot(BlobStoreIndexShardSnapshot snapshot, OutputStream stream) throws IOException {
|
|
|
+ XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream).prettyPrint();
|
|
|
BlobStoreIndexShardSnapshot.toXContent(snapshot, builder, ToXContent.EMPTY_PARAMS);
|
|
|
- return builder.bytes().toBytes();
|
|
|
+ builder.flush();
|
|
|
+ builder.close();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Parses JSON representation of a snapshot
|
|
|
*
|
|
|
- * @param data JSON
|
|
|
+ * @param stream JSON
|
|
|
* @return snapshot
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- public static BlobStoreIndexShardSnapshot readSnapshot(byte[] data) throws IOException {
|
|
|
- try (XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(data)) {
|
|
|
+ * @throws IOException if an IOException occurs
|
|
|
+ * */
|
|
|
+ public static BlobStoreIndexShardSnapshot readSnapshot(InputStream stream) throws IOException {
|
|
|
+ try (XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(stream)) {
|
|
|
parser.nextToken();
|
|
|
return BlobStoreIndexShardSnapshot.fromXContent(parser);
|
|
|
}
|
|
@@ -237,7 +235,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
|
|
|
protected final ShardId shardId;
|
|
|
|
|
|
- protected final ImmutableBlobContainer blobContainer;
|
|
|
+ protected final BlobContainer blobContainer;
|
|
|
|
|
|
public Context(SnapshotId snapshotId, ShardId shardId) {
|
|
|
this(snapshotId, shardId, shardId);
|
|
@@ -246,7 +244,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
public Context(SnapshotId snapshotId, ShardId shardId, ShardId snapshotShardId) {
|
|
|
this.snapshotId = snapshotId;
|
|
|
this.shardId = shardId;
|
|
|
- blobContainer = blobStore.immutableBlobContainer(basePath.add("indices").add(snapshotShardId.getIndex()).add(Integer.toString(snapshotShardId.getId())));
|
|
|
+ blobContainer = blobStore.blobContainer(basePath.add("indices").add(snapshotShardId.getIndex()).add(Integer.toString(snapshotShardId.getId())));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -286,8 +284,8 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
*/
|
|
|
public BlobStoreIndexShardSnapshot loadSnapshot() {
|
|
|
BlobStoreIndexShardSnapshot snapshot;
|
|
|
- try {
|
|
|
- snapshot = readSnapshot(blobContainer.readBlobFully(snapshotBlobName(snapshotId)));
|
|
|
+ try (InputStream stream = blobContainer.openInput(snapshotBlobName(snapshotId))) {
|
|
|
+ snapshot = readSnapshot(stream);
|
|
|
} catch (IOException ex) {
|
|
|
throw new IndexShardRestoreFailedException(shardId, "failed to read shard snapshot file", ex);
|
|
|
}
|
|
@@ -362,8 +360,8 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
List<BlobStoreIndexShardSnapshot> snapshots = Lists.newArrayList();
|
|
|
for (String name : blobs.keySet()) {
|
|
|
if (name.startsWith(SNAPSHOT_PREFIX)) {
|
|
|
- try {
|
|
|
- snapshots.add(readSnapshot(blobContainer.readBlobFully(name)));
|
|
|
+ try (InputStream stream = blobContainer.openInput(name)) {
|
|
|
+ snapshots.add(readSnapshot(stream));
|
|
|
} catch (IOException e) {
|
|
|
logger.warn("failed to read commit point [{}]", e, name);
|
|
|
}
|
|
@@ -469,28 +467,15 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
|
|
|
snapshotStatus.updateStage(IndexShardSnapshotStatus.Stage.STARTED);
|
|
|
|
|
|
- final CountDownLatch indexLatch = new CountDownLatch(filesToSnapshot.size());
|
|
|
-
|
|
|
for (FileInfo snapshotFileInfo : filesToSnapshot) {
|
|
|
try {
|
|
|
- snapshotFile(snapshotFileInfo, indexLatch, failures);
|
|
|
+ snapshotFile(snapshotFileInfo);
|
|
|
} catch (IOException e) {
|
|
|
- failures.add(e);
|
|
|
+ throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
snapshotStatus.indexVersion(snapshotIndexCommit.getGeneration());
|
|
|
-
|
|
|
- try {
|
|
|
- indexLatch.await();
|
|
|
- } catch (InterruptedException e) {
|
|
|
- failures.add(e);
|
|
|
- Thread.currentThread().interrupt();
|
|
|
- }
|
|
|
- if (!failures.isEmpty()) {
|
|
|
- throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", failures.get(0));
|
|
|
- }
|
|
|
-
|
|
|
// now create and write the commit point
|
|
|
snapshotStatus.updateStage(IndexShardSnapshotStatus.Stage.FINALIZE);
|
|
|
|
|
@@ -501,9 +486,10 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
System.currentTimeMillis() - snapshotStatus.startTime(), indexNumberOfFiles, indexTotalFilesSize);
|
|
|
//TODO: The time stored in snapshot doesn't include cleanup time.
|
|
|
try {
|
|
|
- byte[] snapshotData = writeSnapshot(snapshot);
|
|
|
logger.trace("[{}] [{}] writing shard snapshot file", shardId, snapshotId);
|
|
|
- blobContainer.writeBlob(commitPointName, new BytesStreamInput(snapshotData, false), snapshotData.length);
|
|
|
+ try (OutputStream output = blobContainer.createOutput(commitPointName)) {
|
|
|
+ writeSnapshot(snapshot, output);
|
|
|
+ }
|
|
|
} catch (IOException e) {
|
|
|
throw new IndexShardSnapshotFailedException(shardId, "Failed to write commit point", e);
|
|
|
}
|
|
@@ -529,98 +515,32 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
* added to the {@code failures} list
|
|
|
*
|
|
|
* @param fileInfo file to be snapshotted
|
|
|
- * @param latch latch that should be counted down once file is snapshoted
|
|
|
- * @param failures thread-safe list of failures
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- private void snapshotFile(final BlobStoreIndexShardSnapshot.FileInfo fileInfo, final CountDownLatch latch, final List<Throwable> failures) throws IOException {
|
|
|
+ private void snapshotFile(final BlobStoreIndexShardSnapshot.FileInfo fileInfo) throws IOException {
|
|
|
final String file = fileInfo.physicalName();
|
|
|
- IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata());
|
|
|
- writeBlob(indexInput, fileInfo, 0, latch, failures);
|
|
|
- }
|
|
|
-
|
|
|
- private class BlobPartWriter implements ImmutableBlobContainer.WriterListener {
|
|
|
-
|
|
|
- private final int part;
|
|
|
-
|
|
|
- private final FileInfo fileInfo;
|
|
|
-
|
|
|
- private final List<Throwable> failures;
|
|
|
-
|
|
|
- private final CountDownLatch latch;
|
|
|
-
|
|
|
- private final IndexInput indexInput;
|
|
|
-
|
|
|
- private final InputStream inputStream;
|
|
|
-
|
|
|
- private final InputStreamIndexInput inputStreamIndexInput;
|
|
|
-
|
|
|
- private BlobPartWriter(IndexInput indexInput, FileInfo fileInfo, int part, CountDownLatch latch, List<Throwable> failures) throws IOException {
|
|
|
- this.indexInput = indexInput;
|
|
|
- this.part = part;
|
|
|
- this.fileInfo = fileInfo;
|
|
|
- this.failures = failures;
|
|
|
- this.latch = latch;
|
|
|
- inputStreamIndexInput = new InputStreamIndexInput(indexInput, fileInfo.partBytes());
|
|
|
- InputStream inputStream = inputStreamIndexInput;
|
|
|
- if (snapshotRateLimiter != null) {
|
|
|
- inputStream = new RateLimitingInputStream(inputStream, snapshotRateLimiter, snapshotThrottleListener);
|
|
|
- }
|
|
|
- inputStream = new AbortableInputStream(inputStream, fileInfo.physicalName());
|
|
|
- this.inputStream = inputStream;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onCompleted() {
|
|
|
- int nextPart = part + 1;
|
|
|
- if (nextPart < fileInfo.numberOfParts()) {
|
|
|
- try {
|
|
|
- // We have more parts to go
|
|
|
- writeBlob(indexInput, fileInfo, nextPart, latch, failures);
|
|
|
- } catch (Throwable t) {
|
|
|
- onFailure(t);
|
|
|
- }
|
|
|
- } else {
|
|
|
- // Last part - verify checksum
|
|
|
- try {
|
|
|
- Store.verify(indexInput);
|
|
|
- indexInput.close();
|
|
|
- snapshotStatus.addProcessedFile(fileInfo.length());
|
|
|
- } catch (Throwable t) {
|
|
|
- onFailure(t);
|
|
|
- return;
|
|
|
+ try (IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata())) {
|
|
|
+ for (int i = 0; i < fileInfo.numberOfParts(); i++) {
|
|
|
+ final InputStreamIndexInput inputStreamIndexInput = new InputStreamIndexInput(indexInput, fileInfo.partBytes());
|
|
|
+ InputStream inputStream = snapshotRateLimiter == null ? inputStreamIndexInput : new RateLimitingInputStream(inputStreamIndexInput, snapshotRateLimiter, snapshotThrottleListener);
|
|
|
+ inputStream = new AbortableInputStream(inputStream, fileInfo.physicalName());
|
|
|
+ try (OutputStream output = blobContainer.createOutput(fileInfo.partName(i))) {
|
|
|
+ int len;
|
|
|
+ final byte[] buffer = new byte[BUFFER_SIZE];
|
|
|
+ while ((len = inputStream.read(buffer)) > 0) {
|
|
|
+ output.write(buffer, 0, len);
|
|
|
+ }
|
|
|
}
|
|
|
- latch.countDown();
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onFailure(Throwable t) {
|
|
|
- cleanupFailedSnapshot(t, indexInput, latch, failures);
|
|
|
- }
|
|
|
-
|
|
|
- public void writeBlobPart() throws IOException {
|
|
|
- blobContainer.writeBlob(fileInfo.partName(part), inputStream, inputStreamIndexInput.actualSizeToRead(), this);
|
|
|
- }
|
|
|
-
|
|
|
- }
|
|
|
-
|
|
|
- private void writeBlob(IndexInput indexInput, FileInfo fileInfo, int part, CountDownLatch latch, List<Throwable> failures) {
|
|
|
- try {
|
|
|
- new BlobPartWriter(indexInput, fileInfo, part, latch, failures).writeBlobPart();
|
|
|
+ Store.verify(indexInput);
|
|
|
+ snapshotStatus.addProcessedFile(fileInfo.length());
|
|
|
} catch (Throwable t) {
|
|
|
- cleanupFailedSnapshot(t, indexInput, latch, failures);
|
|
|
+ failStoreIfCorrupted(t);
|
|
|
+ snapshotStatus.addProcessedFile(0);
|
|
|
+ throw t;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void cleanupFailedSnapshot(Throwable t, IndexInput indexInput, CountDownLatch latch, List<Throwable> failures) {
|
|
|
- IOUtils.closeWhileHandlingException(indexInput);
|
|
|
- failStoreIfCorrupted(t);
|
|
|
- snapshotStatus.addProcessedFile(0);
|
|
|
- failures.add(t);
|
|
|
- latch.countDown();
|
|
|
- }
|
|
|
-
|
|
|
private void failStoreIfCorrupted(Throwable t) {
|
|
|
if (t instanceof CorruptIndexException) {
|
|
|
try {
|
|
@@ -693,69 +613,39 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
* The new logic for StoreFileMetaData reads the entire <tt>.si</tt> and <tt>segments.n</tt> files to strengthen the
|
|
|
* comparison of the files on a per-segment / per-commit level.
|
|
|
*/
|
|
|
- private static final void maybeRecalculateMetadataHash(final ImmutableBlobContainer blobContainer, final FileInfo fileInfo, Store.MetadataSnapshot snapshot) throws Throwable {
|
|
|
+ private static final void maybeRecalculateMetadataHash(final BlobContainer blobContainer, final FileInfo fileInfo, Store.MetadataSnapshot snapshot) throws Throwable {
|
|
|
final StoreFileMetaData metadata;
|
|
|
if (fileInfo != null && (metadata = snapshot.get(fileInfo.physicalName())) != null) {
|
|
|
if (metadata.hash().length > 0 && fileInfo.metadata().hash().length == 0) {
|
|
|
// we have a hash - check if our repo has a hash too otherwise we have
|
|
|
// to calculate it.
|
|
|
- final ByteArrayOutputStream out = new ByteArrayOutputStream();
|
|
|
- final CountDownLatch latch = new CountDownLatch(1);
|
|
|
- final CopyOnWriteArrayList<Throwable> failures = new CopyOnWriteArrayList<>();
|
|
|
// we might have multiple parts even though the file is small... make sure we read all of it.
|
|
|
- // TODO this API should really support a stream!
|
|
|
- blobContainer.readBlob(fileInfo.partName(0), new BlobContainer.ReadBlobListener() {
|
|
|
- final AtomicInteger partIndex = new AtomicInteger();
|
|
|
- @Override
|
|
|
- public synchronized void onPartial(byte[] data, int offset, int size) throws IOException {
|
|
|
- out.write(data, offset, size);
|
|
|
- }
|
|
|
+ try (final InputStream stream = new PartSliceStream(blobContainer, fileInfo)) {
|
|
|
+ final byte[] bytes = ByteStreams.toByteArray(stream);
|
|
|
+ assert bytes != null;
|
|
|
+ assert bytes.length == fileInfo.length() : bytes.length + " != " + fileInfo.length();
|
|
|
+ final BytesRef spare = new BytesRef(bytes);
|
|
|
+ Store.MetadataSnapshot.hashFile(fileInfo.metadata().hash(), spare);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- @Override
|
|
|
- public synchronized void onCompleted() {
|
|
|
- boolean countDown = true;
|
|
|
- try {
|
|
|
- final int part = partIndex.incrementAndGet();
|
|
|
- if (part < fileInfo.numberOfParts()) {
|
|
|
- final String partName = fileInfo.partName(part);
|
|
|
- // continue with the new part
|
|
|
- blobContainer.readBlob(partName, this);
|
|
|
- countDown = false;
|
|
|
- return;
|
|
|
- }
|
|
|
- } finally {
|
|
|
- if (countDown) {
|
|
|
- latch.countDown();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
+ private static final class PartSliceStream extends SlicedInputStream {
|
|
|
|
|
|
- @Override
|
|
|
- public void onFailure(Throwable t) {
|
|
|
- try {
|
|
|
- failures.add(t);
|
|
|
- } finally {
|
|
|
- latch.countDown();
|
|
|
- }
|
|
|
- }
|
|
|
- });
|
|
|
+ private final BlobContainer container;
|
|
|
+ private final FileInfo info;
|
|
|
|
|
|
- try {
|
|
|
- latch.await();
|
|
|
- } catch (InterruptedException e) {
|
|
|
- Thread.interrupted();
|
|
|
- }
|
|
|
+ public PartSliceStream(BlobContainer container, FileInfo info) {
|
|
|
+ super(info.numberOfParts());
|
|
|
+ this.info = info;
|
|
|
+ this.container = container;
|
|
|
+ }
|
|
|
|
|
|
- if (!failures.isEmpty()) {
|
|
|
- ExceptionsHelper.rethrowAndSuppress(failures);
|
|
|
- }
|
|
|
+ @Override
|
|
|
+ protected InputStream openSlice(long slice) throws IOException {
|
|
|
+ return container.openInput(info.partName(slice));
|
|
|
|
|
|
- final byte[] bytes = out.toByteArray();
|
|
|
- assert bytes != null;
|
|
|
- assert bytes.length == fileInfo.length() : bytes.length + " != " + fileInfo.length();
|
|
|
- final BytesRef spare = new BytesRef(bytes);
|
|
|
- Store.MetadataSnapshot.hashFile(fileInfo.metadata().hash(), spare);
|
|
|
- }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -864,25 +754,14 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
if (logger.isTraceEnabled()) {
|
|
|
logger.trace("[{}] [{}] recovering_files [{}] with total_size [{}], reusing_files [{}] with reused_size [{}]", shardId, snapshotId, numberOfFiles, new ByteSizeValue(totalSize), numberOfReusedFiles, new ByteSizeValue(reusedTotalSize));
|
|
|
}
|
|
|
-
|
|
|
- final CountDownLatch latch = new CountDownLatch(filesToRecover.size());
|
|
|
- final CopyOnWriteArrayList<Throwable> failures = new CopyOnWriteArrayList<>();
|
|
|
-
|
|
|
- for (final FileInfo fileToRecover : filesToRecover) {
|
|
|
- logger.trace("[{}] [{}] restoring file [{}]", shardId, snapshotId, fileToRecover.name());
|
|
|
- restoreFile(fileToRecover, latch, failures);
|
|
|
- }
|
|
|
-
|
|
|
try {
|
|
|
- latch.await();
|
|
|
- } catch (InterruptedException e) {
|
|
|
- Thread.currentThread().interrupt();
|
|
|
- }
|
|
|
-
|
|
|
- if (!failures.isEmpty()) {
|
|
|
- throw new IndexShardRestoreFailedException(shardId, "Failed to recover index", failures.get(0));
|
|
|
+ for (final FileInfo fileToRecover : filesToRecover) {
|
|
|
+ logger.trace("[{}] [{}] restoring file [{}]", shardId, snapshotId, fileToRecover.name());
|
|
|
+ restoreFile(fileToRecover);
|
|
|
+ }
|
|
|
+ } catch (IOException ex) {
|
|
|
+ throw new IndexShardRestoreFailedException(shardId, "Failed to recover index", ex);
|
|
|
}
|
|
|
-
|
|
|
// read the snapshot data persisted
|
|
|
long version = -1;
|
|
|
try {
|
|
@@ -919,107 +798,52 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|
|
* added to the {@code failures} list
|
|
|
*
|
|
|
* @param fileInfo file to be restored
|
|
|
- * @param latch latch that should be counted down once file is snapshoted
|
|
|
- * @param failures thread-safe list of failures
|
|
|
*/
|
|
|
- private void restoreFile(final FileInfo fileInfo, final CountDownLatch latch, final List<Throwable> failures) {
|
|
|
- final IndexOutput indexOutput;
|
|
|
- try {
|
|
|
- // we create an output with no checksum, this is because the pure binary data of the file is not
|
|
|
- // the checksum (because of seek). We will create the checksum file once copying is done
|
|
|
- indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), IOContext.DEFAULT, fileInfo.metadata());
|
|
|
- } catch (IOException e) {
|
|
|
- try {
|
|
|
- failures.add(e);
|
|
|
- } finally {
|
|
|
- latch.countDown();
|
|
|
- }
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- String firstFileToRecover = fileInfo.partName(0);
|
|
|
- final AtomicInteger partIndex = new AtomicInteger();
|
|
|
+ private void restoreFile(final FileInfo fileInfo) throws IOException {
|
|
|
boolean success = false;
|
|
|
- try {
|
|
|
- blobContainer.readBlob(firstFileToRecover, new BlobContainer.ReadBlobListener() {
|
|
|
- @Override
|
|
|
- public synchronized void onPartial(byte[] data, int offset, int size) throws IOException {
|
|
|
- recoveryState.getIndex().addRecoveredByteCount(size);
|
|
|
- RecoveryState.File file = recoveryState.getIndex().file(fileInfo.name());
|
|
|
+ RecoveryState.File file = recoveryState.getIndex().file(fileInfo.name());
|
|
|
+ try (InputStream stream = new PartSliceStream(blobContainer, fileInfo)) {
|
|
|
+ try (final IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), IOContext.DEFAULT, fileInfo.metadata())) {
|
|
|
+ final byte[] buffer = new byte[BUFFER_SIZE];
|
|
|
+ int length;
|
|
|
+ while((length=stream.read(buffer))>0){
|
|
|
+ indexOutput.writeBytes(buffer,0,length);
|
|
|
if (file != null) {
|
|
|
- file.updateRecovered(size);
|
|
|
+ file.updateRecovered(length);
|
|
|
}
|
|
|
- indexOutput.writeBytes(data, offset, size);
|
|
|
if (restoreRateLimiter != null) {
|
|
|
- rateLimiterListener.onRestorePause(restoreRateLimiter.pause(size));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public synchronized void onCompleted() {
|
|
|
- int part = partIndex.incrementAndGet();
|
|
|
- if (part < fileInfo.numberOfParts()) {
|
|
|
- String partName = fileInfo.partName(part);
|
|
|
- // continue with the new part
|
|
|
- blobContainer.readBlob(partName, this);
|
|
|
- return;
|
|
|
- } else {
|
|
|
- // we are done...
|
|
|
- try {
|
|
|
- Store.verify(indexOutput);
|
|
|
- indexOutput.close();
|
|
|
- // write the checksum
|
|
|
- if (fileInfo.metadata().hasLegacyChecksum()) {
|
|
|
- Store.LegacyChecksums legacyChecksums = new Store.LegacyChecksums();
|
|
|
- legacyChecksums.add(fileInfo.metadata());
|
|
|
- legacyChecksums.write(store);
|
|
|
-
|
|
|
- }
|
|
|
- store.directory().sync(Collections.singleton(fileInfo.physicalName()));
|
|
|
- recoveryState.getIndex().addRecoveredFileCount(1);
|
|
|
- } catch (IOException e) {
|
|
|
- onFailure(e);
|
|
|
- return;
|
|
|
- }
|
|
|
+ rateLimiterListener.onRestorePause(restoreRateLimiter.pause(length));
|
|
|
}
|
|
|
- latch.countDown();
|
|
|
}
|
|
|
+ Store.verify(indexOutput);
|
|
|
+ indexOutput.close();
|
|
|
+ // write the checksum
|
|
|
+ if (fileInfo.metadata().hasLegacyChecksum()) {
|
|
|
+ Store.LegacyChecksums legacyChecksums = new Store.LegacyChecksums();
|
|
|
+ legacyChecksums.add(fileInfo.metadata());
|
|
|
+ legacyChecksums.write(store);
|
|
|
|
|
|
- @Override
|
|
|
- public void onFailure(Throwable t) {
|
|
|
- try {
|
|
|
- failures.add(t);
|
|
|
- IOUtils.closeWhileHandlingException(indexOutput);
|
|
|
- if (t instanceof CorruptIndexException) {
|
|
|
- try {
|
|
|
- store.markStoreCorrupted((CorruptIndexException) t);
|
|
|
- } catch (IOException e) {
|
|
|
- logger.warn("store cannot be marked as corrupted", e);
|
|
|
- }
|
|
|
- }
|
|
|
- store.deleteQuiet(fileInfo.physicalName());
|
|
|
- } finally {
|
|
|
- latch.countDown();
|
|
|
- }
|
|
|
}
|
|
|
- });
|
|
|
- success = true;
|
|
|
- } finally {
|
|
|
- if (!success) {
|
|
|
+ store.directory().sync(Collections.singleton(fileInfo.physicalName()));
|
|
|
+ recoveryState.getIndex().addRecoveredFileCount(1);
|
|
|
+ success = true;
|
|
|
+ } catch (CorruptIndexException ex) {
|
|
|
try {
|
|
|
- IOUtils.closeWhileHandlingException(indexOutput);
|
|
|
+ store.markStoreCorrupted(ex);
|
|
|
+ } catch (IOException e) {
|
|
|
+ logger.warn("store cannot be marked as corrupted", e);
|
|
|
+ }
|
|
|
+ throw ex;
|
|
|
+ } finally {
|
|
|
+ if (success == false) {
|
|
|
store.deleteQuiet(fileInfo.physicalName());
|
|
|
- } finally {
|
|
|
- latch.countDown();
|
|
|
}
|
|
|
-
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
}
|
|
|
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
public interface RateLimiterListener {
|
|
|
void onRestorePause(long nanos);
|
|
|
|