better logging message for reused vs. recovered from gateway
This commit is contained in:
parent
5bd37f6f47
commit
62cc4d554a
|
@ -212,7 +212,7 @@ public class TransportIndicesStatusAction extends TransportBroadcastOperationAct
|
|||
stage = GatewayRecoveryStatus.Stage.INIT;
|
||||
}
|
||||
shardStatus.gatewayRecoveryStatus = new GatewayRecoveryStatus(stage, gatewayRecoveryStatus.startTime(), gatewayRecoveryStatus.time(),
|
||||
gatewayRecoveryStatus.index().totalSize(), gatewayRecoveryStatus.index().existingTotalSize(), gatewayRecoveryStatus.index().currentFilesSize(), gatewayRecoveryStatus.translog().currentTranslogOperations());
|
||||
gatewayRecoveryStatus.index().totalSize(), gatewayRecoveryStatus.index().reusedTotalSize(), gatewayRecoveryStatus.index().currentFilesSize(), gatewayRecoveryStatus.translog().currentTranslogOperations());
|
||||
}
|
||||
|
||||
SnapshotStatus snapshotStatus = gatewayService.snapshotStatus();
|
||||
|
|
|
@ -167,8 +167,9 @@ public class IndexShardGatewayService extends AbstractIndexShardComponent implem
|
|||
if (logger.isDebugEnabled()) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("recovery completed from ").append(shardGateway).append(", took [").append(timeValueMillis(recoveryStatus.time())).append("]\n");
|
||||
sb.append(" index : recovered_files [").append(recoveryStatus.index().numberOfFiles()).append("] with total_size [").append(new ByteSizeValue(recoveryStatus.index().totalSize())).append("], took [").append(TimeValue.timeValueMillis(recoveryStatus.index().time())).append("]\n");
|
||||
sb.append(" : reusing_files [").append(recoveryStatus.index().numberOfExistingFiles()).append("] with total_size [").append(new ByteSizeValue(recoveryStatus.index().existingTotalSize())).append("]\n");
|
||||
sb.append(" index : total_size [").append(new ByteSizeValue(recoveryStatus.index().totalSize())).append("], took[").append(TimeValue.timeValueMillis(recoveryStatus.index().time())).append("], took [").append(TimeValue.timeValueMillis(recoveryStatus.index().time())).append("]\n");
|
||||
sb.append(" : recovered_files [").append(recoveryStatus.index().numberOfFiles()).append("] with total_size [").append(new ByteSizeValue(recoveryStatus.index().reusedTotalSize())).append("], took [").append(TimeValue.timeValueMillis(recoveryStatus.index().time())).append("]\n");
|
||||
sb.append(" : reusing_files [").append(recoveryStatus.index().numberOfReusedFiles()).append("] with total_size [").append(new ByteSizeValue(recoveryStatus.index().reusedTotalSize())).append("]\n");
|
||||
sb.append(" translog : number_of_operations [").append(recoveryStatus.translog().currentTranslogOperations()).append("], took [").append(TimeValue.timeValueMillis(recoveryStatus.translog().time())).append("]");
|
||||
logger.debug(sb.toString());
|
||||
}
|
||||
|
|
|
@ -113,8 +113,8 @@ public class RecoveryStatus {
|
|||
private long version = -1;
|
||||
private int numberOfFiles = 0;
|
||||
private long totalSize = 0;
|
||||
private int numberOfExistingFiles = 0;
|
||||
private long existingTotalSize = 0;
|
||||
private int numberOfReusedFiles = 0;
|
||||
private long reusedTotalSize = 0;
|
||||
private AtomicLong currentFilesSize = new AtomicLong();
|
||||
|
||||
public long startTime() {
|
||||
|
@ -137,11 +137,11 @@ public class RecoveryStatus {
|
|||
return this.version;
|
||||
}
|
||||
|
||||
public void files(int numberOfFiles, long totalSize, int numberOfExistingFiles, long existingTotalSize) {
|
||||
public void files(int numberOfFiles, long totalSize, int numberOfReusedFiles, long reusedTotalSize) {
|
||||
this.numberOfFiles = numberOfFiles;
|
||||
this.totalSize = totalSize;
|
||||
this.numberOfExistingFiles = numberOfExistingFiles;
|
||||
this.existingTotalSize = existingTotalSize;
|
||||
this.numberOfReusedFiles = numberOfReusedFiles;
|
||||
this.reusedTotalSize = reusedTotalSize;
|
||||
}
|
||||
|
||||
public int numberOfFiles() {
|
||||
|
@ -152,12 +152,16 @@ public class RecoveryStatus {
|
|||
return this.totalSize;
|
||||
}
|
||||
|
||||
public int numberOfExistingFiles() {
|
||||
return numberOfExistingFiles;
|
||||
public int numberOfReusedFiles() {
|
||||
return numberOfReusedFiles;
|
||||
}
|
||||
|
||||
public long existingTotalSize() {
|
||||
return this.existingTotalSize;
|
||||
public long reusedTotalSize() {
|
||||
return this.reusedTotalSize;
|
||||
}
|
||||
|
||||
public long recoveredTotalSize() {
|
||||
return totalSize - reusedTotalSize;
|
||||
}
|
||||
|
||||
public void updateVersion(long version) {
|
||||
|
|
|
@ -477,15 +477,15 @@ public abstract class BlobStoreIndexShardGateway extends AbstractIndexShardCompo
|
|||
private void recoverIndex(CommitPoint commitPoint, ImmutableMap<String, BlobMetaData> blobs) throws Exception {
|
||||
int numberOfFiles = 0;
|
||||
long totalSize = 0;
|
||||
int numberOfExistingFiles = 0;
|
||||
long existingTotalSize = 0;
|
||||
int numberOfReusedFiles = 0;
|
||||
long reusedTotalSize = 0;
|
||||
|
||||
List<CommitPoint.FileInfo> filesToRecover = Lists.newArrayList();
|
||||
for (CommitPoint.FileInfo fileInfo : commitPoint.indexFiles()) {
|
||||
StoreFileMetaData storeFile = store.metaData(fileInfo.physicalName());
|
||||
if (storeFile != null && !storeFile.name().contains("segment") && storeFile.length() == fileInfo.length()) {
|
||||
numberOfExistingFiles++;
|
||||
existingTotalSize += storeFile.length();
|
||||
numberOfReusedFiles++;
|
||||
reusedTotalSize += storeFile.length();
|
||||
totalSize += storeFile.length();
|
||||
if (logger.isTraceEnabled()) {
|
||||
logger.trace("not_recovering [{}], exists in local store and has same length [{}]", fileInfo.physicalName(), fileInfo.length());
|
||||
|
@ -504,13 +504,13 @@ public abstract class BlobStoreIndexShardGateway extends AbstractIndexShardCompo
|
|||
}
|
||||
}
|
||||
|
||||
recoveryStatus.index().files(numberOfFiles, totalSize, numberOfExistingFiles, existingTotalSize);
|
||||
recoveryStatus.index().files(numberOfFiles, totalSize, numberOfReusedFiles, reusedTotalSize);
|
||||
if (filesToRecover.isEmpty()) {
|
||||
logger.trace("no files to recover, all exists within the local store");
|
||||
}
|
||||
|
||||
if (logger.isTraceEnabled()) {
|
||||
logger.trace("recovering_files [{}] with total_size [{}], reusing_files [{}] with reused_size [{}]", numberOfFiles, new ByteSizeValue(totalSize), numberOfExistingFiles, new ByteSizeValue(existingTotalSize));
|
||||
logger.trace("recovering_files [{}] with total_size [{}], reusing_files [{}] with reused_size [{}]", numberOfFiles, new ByteSizeValue(totalSize), numberOfReusedFiles, new ByteSizeValue(reusedTotalSize));
|
||||
}
|
||||
|
||||
final CountDownLatch latch = new CountDownLatch(filesToRecover.size());
|
||||
|
|
Loading…
Reference in New Issue