more debug statr on recovery from gateway

This commit is contained in:
kimchy 2010-07-11 21:51:45 +03:00
parent e7a8da8236
commit 52ac24fa23
5 changed files with 32 additions and 14 deletions

View File

@ -213,19 +213,27 @@ public interface IndexShardGateway extends IndexShardComponent, CloseableIndexCo
} }
public static class Translog { public static class Translog {
private int numberOfOperations; public static final Translog EMPTY = new Translog(0, TimeValue.timeValueMillis(0));
public Translog(int numberOfOperations) { private int numberOfOperations;
private TimeValue took;
public Translog(int numberOfOperations, TimeValue took) {
this.numberOfOperations = numberOfOperations; this.numberOfOperations = numberOfOperations;
this.took = took;
} }
public int numberOfOperations() { public int numberOfOperations() {
return numberOfOperations; return numberOfOperations;
} }
public TimeValue took() {
return this.took;
}
} }
public static class Index { public static class Index {
public static final Index EMPTY = new Index(-1, 0, new ByteSizeValue(0), 0, new ByteSizeValue(0), timeValueMillis(0)); public static final Index EMPTY = new Index(-1, 0, new ByteSizeValue(0), 0, new ByteSizeValue(0), timeValueMillis(0), timeValueMillis(0));
private long version; private long version;
private int numberOfFiles; private int numberOfFiles;
@ -233,16 +241,18 @@ public interface IndexShardGateway extends IndexShardComponent, CloseableIndexCo
private int numberOfExistingFiles; private int numberOfExistingFiles;
private ByteSizeValue existingTotalSize; private ByteSizeValue existingTotalSize;
private TimeValue throttlingWaitTime; private TimeValue throttlingWaitTime;
private TimeValue took;
public Index(long version, int numberOfFiles, ByteSizeValue totalSize, public Index(long version, int numberOfFiles, ByteSizeValue totalSize,
int numberOfExistingFiles, ByteSizeValue existingTotalSize, int numberOfExistingFiles, ByteSizeValue existingTotalSize,
TimeValue throttlingWaitTime) { TimeValue throttlingWaitTime, TimeValue took) {
this.version = version; this.version = version;
this.numberOfFiles = numberOfFiles; this.numberOfFiles = numberOfFiles;
this.totalSize = totalSize; this.totalSize = totalSize;
this.numberOfExistingFiles = numberOfExistingFiles; this.numberOfExistingFiles = numberOfExistingFiles;
this.existingTotalSize = existingTotalSize; this.existingTotalSize = existingTotalSize;
this.throttlingWaitTime = throttlingWaitTime; this.throttlingWaitTime = throttlingWaitTime;
this.took = took;
} }
public long version() { public long version() {
@ -268,6 +278,10 @@ public interface IndexShardGateway extends IndexShardComponent, CloseableIndexCo
public TimeValue throttlingWaitTime() { public TimeValue throttlingWaitTime() {
return throttlingWaitTime; return throttlingWaitTime;
} }
public TimeValue took() {
return this.took;
}
} }
} }
} }

View File

@ -144,9 +144,9 @@ public class IndexShardGatewayService extends AbstractIndexShardComponent implem
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
StringBuilder sb = new StringBuilder(); StringBuilder sb = new StringBuilder();
sb.append("recovery completed from ").append(shardGateway).append(", took [").append(stopWatch.totalTime()).append("], throttling_wait [").append(throttlingWaitTime.totalTime()).append("]\n"); sb.append("recovery completed from ").append(shardGateway).append(", took [").append(stopWatch.totalTime()).append("], throttling_wait [").append(throttlingWaitTime.totalTime()).append("]\n");
sb.append(" index : recovered_files [").append(recoveryStatus.index().numberOfFiles()).append("] with total_size [").append(recoveryStatus.index().totalSize()).append("], throttling_wait [").append(recoveryStatus.index().throttlingWaitTime()).append("]\n"); sb.append(" index : recovered_files [").append(recoveryStatus.index().numberOfFiles()).append("] with total_size [").append(recoveryStatus.index().totalSize()).append("], took [").append(recoveryStatus.index().took()).append("], throttling_wait [").append(recoveryStatus.index().throttlingWaitTime()).append("]\n");
sb.append(" : reusing_files [").append(recoveryStatus.index().numberOfExistingFiles()).append("] with total_size [").append(recoveryStatus.index().existingTotalSize()).append("]\n"); sb.append(" : reusing_files [").append(recoveryStatus.index().numberOfExistingFiles()).append("] with total_size [").append(recoveryStatus.index().existingTotalSize()).append("]\n");
sb.append(" translog : number_of_operations [").append(recoveryStatus.translog().numberOfOperations()).append("]"); sb.append(" translog : number_of_operations [").append(recoveryStatus.translog().numberOfOperations()).append("], took [").append(recoveryStatus.translog().took()).append("]");
logger.debug(sb.toString()); logger.debug(sb.toString());
} }
// refresh the shard // refresh the shard

View File

@ -26,6 +26,7 @@ import org.apache.lucene.store.IndexOutput;
import org.elasticsearch.ElasticSearchException; import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.common.Digest; import org.elasticsearch.common.Digest;
import org.elasticsearch.common.Hex; import org.elasticsearch.common.Hex;
import org.elasticsearch.common.StopWatch;
import org.elasticsearch.common.blobstore.*; import org.elasticsearch.common.blobstore.*;
import org.elasticsearch.common.blobstore.support.PlainBlobMetaData; import org.elasticsearch.common.blobstore.support.PlainBlobMetaData;
import org.elasticsearch.common.collect.ImmutableMap; import org.elasticsearch.common.collect.ImmutableMap;
@ -367,16 +368,18 @@ public abstract class BlobStoreIndexShardGateway extends AbstractIndexShardCompo
} catch (FileNotFoundException e) { } catch (FileNotFoundException e) {
// no index, that fine // no index, that fine
indexShard.start(); indexShard.start();
return new RecoveryStatus.Translog(0); return RecoveryStatus.Translog.EMPTY;
} catch (IOException e) { } catch (IOException e) {
throw new IndexShardGatewayRecoveryException(shardId, "Failed to recovery translog, can't read current index version", e); throw new IndexShardGatewayRecoveryException(shardId, "Failed to recovery translog, can't read current index version", e);
} }
if (!translogContainer.blobExists("translog-" + translogId)) { if (!translogContainer.blobExists("translog-" + translogId)) {
// no recovery file found, start the shard and bail // no recovery file found, start the shard and bail
indexShard.start(); indexShard.start();
return new RecoveryStatus.Translog(0); return RecoveryStatus.Translog.EMPTY;
} }
StopWatch timer = new StopWatch().start();
try { try {
indexShard.performRecoveryPrepareForTranslog(); indexShard.performRecoveryPrepareForTranslog();
@ -450,13 +453,14 @@ public abstract class BlobStoreIndexShardGateway extends AbstractIndexShardCompo
indexShard.performRecoveryFinalization(true); indexShard.performRecoveryFinalization(true);
return new RecoveryStatus.Translog(totalOperations.get()); return new RecoveryStatus.Translog(totalOperations.get(), timer.stop().totalTime());
} catch (Throwable e) { } catch (Throwable e) {
throw new IndexShardGatewayRecoveryException(shardId, "Failed to recovery translog", e); throw new IndexShardGatewayRecoveryException(shardId, "Failed to recovery translog", e);
} }
} }
private RecoveryStatus.Index recoverIndex() throws IndexShardGatewayRecoveryException { private RecoveryStatus.Index recoverIndex() throws IndexShardGatewayRecoveryException {
StopWatch timer = new StopWatch().start();
final ImmutableMap<String, BlobMetaData> indicesBlobs; final ImmutableMap<String, BlobMetaData> indicesBlobs;
try { try {
indicesBlobs = indexContainer.listBlobs(); indicesBlobs = indexContainer.listBlobs();
@ -562,7 +566,7 @@ public abstract class BlobStoreIndexShardGateway extends AbstractIndexShardCompo
// ignore // ignore
} }
return new RecoveryStatus.Index(version, numberOfFiles, new ByteSizeValue(totalSize), numberOfExistingFiles, new ByteSizeValue(existingTotalSize), TimeValue.timeValueMillis(throttlingWaitTime.get())); return new RecoveryStatus.Index(version, numberOfFiles, new ByteSizeValue(totalSize), numberOfExistingFiles, new ByteSizeValue(existingTotalSize), TimeValue.timeValueMillis(throttlingWaitTime.get()), timer.stop().totalTime());
} }
private void recoverFile(final BlobMetaData fileToRecover, final ImmutableMap<String, BlobMetaData> blobs, final CountDownLatch latch, final List<Throwable> failures) { private void recoverFile(final BlobMetaData fileToRecover, final ImmutableMap<String, BlobMetaData> blobs, final CountDownLatch latch, final List<Throwable> failures) {

View File

@ -53,7 +53,7 @@ public class NoneIndexShardGateway extends AbstractIndexShardComponent implement
logger.warn("failed to clean store before starting shard", e); logger.warn("failed to clean store before starting shard", e);
} }
indexShard.start(); indexShard.start();
return new RecoveryStatus(RecoveryStatus.Index.EMPTY, new RecoveryStatus.Translog(0)); return new RecoveryStatus(RecoveryStatus.Index.EMPTY, RecoveryStatus.Translog.EMPTY);
} }
@Override public String type() { @Override public String type() {

View File

@ -397,7 +397,7 @@ public class CloudIndexShardGateway extends AbstractIndexShardComponent implemen
throw new IndexShardGatewayRecoveryException(shardId(), "Failed to fetch index version after copying it over", e); throw new IndexShardGatewayRecoveryException(shardId(), "Failed to fetch index version after copying it over", e);
} }
return new RecoveryStatus.Index(version, filesMetaDatas.size(), new ByteSizeValue(totalSize), 0, new ByteSizeValue(0), TimeValue.timeValueMillis(throttlingWaitTime.get())); return new RecoveryStatus.Index(version, filesMetaDatas.size(), new ByteSizeValue(totalSize), 0, new ByteSizeValue(0), TimeValue.timeValueMillis(throttlingWaitTime.get()), TimeValue.timeValueMillis(-1));
} }
private RecoveryStatus.Translog recoverTranslog() throws IndexShardGatewayRecoveryException { private RecoveryStatus.Translog recoverTranslog() throws IndexShardGatewayRecoveryException {
@ -415,7 +415,7 @@ public class CloudIndexShardGateway extends AbstractIndexShardComponent implemen
if (latestTranslogId == -1) { if (latestTranslogId == -1) {
// no recovery file found, start the shard and bail // no recovery file found, start the shard and bail
indexShard.start(); indexShard.start();
return new RecoveryStatus.Translog(0); return new RecoveryStatus.Translog(0, TimeValue.timeValueMillis(0));
} }
@ -446,7 +446,7 @@ public class CloudIndexShardGateway extends AbstractIndexShardComponent implemen
indexShard.performRecoveryPrepareForTranslog(); indexShard.performRecoveryPrepareForTranslog();
indexShard.performRecoveryFinalization(true); indexShard.performRecoveryFinalization(true);
return new RecoveryStatus.Translog(operations.size()); return new RecoveryStatus.Translog(operations.size(), TimeValue.timeValueMillis(-1));
} catch (Exception e) { } catch (Exception e) {
throw new IndexShardGatewayRecoveryException(shardId(), "Failed to perform recovery of translog", e); throw new IndexShardGatewayRecoveryException(shardId(), "Failed to perform recovery of translog", e);
} }