Skip to content

Commit

Permalink
add snapshot status (either current one or the latest one executed) t…
Browse files Browse the repository at this point in the history
…o indices status api
  • Loading branch information
kimchy committed Aug 18, 2010
1 parent 91aada2 commit a3107bc
Show file tree
Hide file tree
Showing 7 changed files with 243 additions and 37 deletions.
@@ -0,0 +1,124 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.action.admin.indices.status;

import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;

/**
* @author kimchy (shay.banon)
*/
public class GatewaySnapshotStatus {

public static enum Stage {
NONE((byte) 0),
INDEX((byte) 1),
TRANSLOG((byte) 2),
FINALIZE((byte) 3),
DONE((byte) 4),
FAILURE((byte) 5);

private final byte value;

Stage(byte value) {
this.value = value;
}

public byte value() {
return this.value;
}

public static Stage fromValue(byte value) {
if (value == 0) {
return Stage.NONE;
} else if (value == 1) {
return Stage.INDEX;
} else if (value == 2) {
return Stage.TRANSLOG;
} else if (value == 3) {
return Stage.FINALIZE;
} else if (value == 4) {
return Stage.DONE;
} else if (value == 5) {
return Stage.FAILURE;
}
throw new ElasticSearchIllegalArgumentException("No stage found for [" + value + "]");
}
}

final Stage stage;

final long startTime;

final long time;

final long indexSize;

final long translogOperations;

public GatewaySnapshotStatus(Stage stage, long startTime, long time, long indexSize, long translogOperations) {
this.stage = stage;
this.startTime = startTime;
this.time = time;
this.indexSize = indexSize;
this.translogOperations = translogOperations;
}

public Stage stage() {
return this.stage;
}

public Stage getStage() {
return stage();
}

public long startTime() {
return this.startTime;
}

public long getStartTime() {
return startTime();
}

public TimeValue time() {
return TimeValue.timeValueMillis(time);
}

public TimeValue getTime() {
return time();
}

public ByteSizeValue indexSize() {
return new ByteSizeValue(indexSize);
}

public ByteSizeValue getIndexSize() {
return indexSize();
}

public long translogOperations() {
return this.translogOperations;
}

public long getTranslogOperations() {
return translogOperations();
}
}
Expand Up @@ -84,6 +84,8 @@ public int getDeletedDocs() {

GatewayRecoveryStatus gatewayRecoveryStatus;

GatewaySnapshotStatus gatewaySnapshotStatus;

ShardStatus() {
}

Expand Down Expand Up @@ -156,6 +158,14 @@ public GatewayRecoveryStatus getGatewayRecoveryStatus() {
return gatewayRecoveryStatus();
}

public GatewaySnapshotStatus gatewaySnapshotStatus() {
return gatewaySnapshotStatus;
}

public GatewaySnapshotStatus getGatewaySnapshotStatus() {
return gatewaySnapshotStatus();
}

public static ShardStatus readIndexShardStatus(StreamInput in) throws IOException {
ShardStatus shardStatus = new ShardStatus();
shardStatus.readFrom(in);
Expand Down Expand Up @@ -210,6 +220,17 @@ public static ShardStatus readIndexShardStatus(StreamInput in) throws IOExceptio
out.writeVLong(gatewayRecoveryStatus.recoveredIndexSize);
out.writeVLong(gatewayRecoveryStatus.recoveredTranslogOperations);
}

if (gatewaySnapshotStatus == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeByte(gatewaySnapshotStatus.stage.value());
out.writeVLong(gatewaySnapshotStatus.startTime);
out.writeVLong(gatewaySnapshotStatus.time);
out.writeVLong(gatewaySnapshotStatus.indexSize);
out.writeVLong(gatewaySnapshotStatus.translogOperations);
}
}

@Override public void readFrom(StreamInput in) throws IOException {
Expand All @@ -236,5 +257,10 @@ public static ShardStatus readIndexShardStatus(StreamInput in) throws IOExceptio
gatewayRecoveryStatus = new GatewayRecoveryStatus(GatewayRecoveryStatus.Stage.fromValue(in.readByte()),
in.readVLong(), in.readVLong(), in.readVLong(), in.readVLong(), in.readVLong(), in.readVLong(), in.readVLong(), in.readVLong());
}

if (in.readBoolean()) {
gatewaySnapshotStatus = new GatewaySnapshotStatus(GatewaySnapshotStatus.Stage.fromValue(in.readByte()),
in.readVLong(), in.readVLong(), in.readVLong(), in.readVLong());
}
}
}
Expand Up @@ -35,6 +35,7 @@
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.gateway.IndexShardGatewayService;
import org.elasticsearch.index.gateway.SnapshotStatus;
import org.elasticsearch.index.service.InternalIndexService;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.recovery.RecoveryStatus;
Expand Down Expand Up @@ -220,6 +221,33 @@ public class TransportIndicesStatusAction extends TransportBroadcastOperationAct
gatewayRecoveryStatus.index().retryTime(), gatewayRecoveryStatus.index().totalSize(), gatewayRecoveryStatus.index().existingTotalSize(), gatewayRecoveryStatus.index().currentFilesSize(), gatewayRecoveryStatus.translog().currentTranslogOperations());
}

SnapshotStatus snapshotStatus = gatewayService.snapshotStatus();
if (snapshotStatus != null) {
GatewaySnapshotStatus.Stage stage;
switch (snapshotStatus.stage()) {
case DONE:
stage = GatewaySnapshotStatus.Stage.DONE;
break;
case FAILURE:
stage = GatewaySnapshotStatus.Stage.FAILURE;
break;
case TRANSLOG:
stage = GatewaySnapshotStatus.Stage.TRANSLOG;
break;
case FINALIZE:
stage = GatewaySnapshotStatus.Stage.FINALIZE;
break;
case INDEX:
stage = GatewaySnapshotStatus.Stage.INDEX;
break;
default:
stage = GatewaySnapshotStatus.Stage.NONE;
break;
}
shardStatus.gatewaySnapshotStatus = new GatewaySnapshotStatus(stage, snapshotStatus.startTime(), snapshotStatus.time(),
snapshotStatus.index().totalSize(), snapshotStatus.translog().currentTranslogOperations());
}

return shardStatus;
}

Expand Down
Expand Up @@ -115,6 +115,14 @@ public RecoveryStatus recoveryStatus() {
return recoveryStatus;
}

public SnapshotStatus snapshotStatus() {
SnapshotStatus snapshotStatus = shardGateway.currentSnapshotStatus();
if (snapshotStatus != null) {
return snapshotStatus;
}
return shardGateway.lastSnapshotStatus();
}

/**
* Recovers the state of the shard from the gateway.
*/
Expand Down Expand Up @@ -241,9 +249,9 @@ public synchronized void snapshot(final String reason) throws IndexShardGatewayS
if (snapshotStatus != null) {
if (logger.isDebugEnabled()) {
StringBuilder sb = new StringBuilder();
sb.append("snapshot (").append(reason).append(") completed to ").append(shardGateway).append(", took [").append(snapshotStatus.took()).append("]\n");
sb.append(" index : version [").append(lastIndexVersion).append("], number_of_files [").append(snapshotStatus.index().numberOfFiles()).append("] with total_size [").append(snapshotStatus.index().totalSize()).append("], took [").append(snapshotStatus.index().took()).append("]\n");
sb.append(" translog : id [").append(lastTranslogId).append("], number_of_operations [").append(snapshotStatus.translog().currentTranslogOperations()).append("], took [").append(snapshotStatus.translog().took()).append("]");
sb.append("snapshot (").append(reason).append(") completed to ").append(shardGateway).append(", took [").append(TimeValue.timeValueMillis(snapshotStatus.time())).append("]\n");
sb.append(" index : version [").append(lastIndexVersion).append("], number_of_files [").append(snapshotStatus.index().numberOfFiles()).append("] with total_size [").append(new ByteSizeValue(snapshotStatus.index().totalSize())).append("], took [").append(TimeValue.timeValueMillis(snapshotStatus.index().time())).append("]\n");
sb.append(" translog : id [").append(lastTranslogId).append("], number_of_operations [").append(snapshotStatus.translog().currentTranslogOperations()).append("], took [").append(TimeValue.timeValueMillis(snapshotStatus.translog().time())).append("]");
logger.debug(sb.toString());
}
}
Expand Down
Expand Up @@ -19,9 +19,6 @@

package org.elasticsearch.index.gateway;

import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;

/**
* @author kimchy (shay.banon)
*/
Expand All @@ -40,7 +37,7 @@ public static enum Stage {

private long startTime;

private long took;
private long time;

private Index index = new Index();

Expand All @@ -65,12 +62,12 @@ public void startTime(long startTime) {
this.startTime = startTime;
}

public TimeValue took() {
return new TimeValue(this.took);
public long time() {
return this.time;
}

public void took(long took) {
this.took = took;
public void time(long time) {
this.time = time;
}

public void failed(Throwable failure) {
Expand All @@ -87,10 +84,10 @@ public Translog translog() {

public static class Index {
private long startTime;
private long took;
private long time;

private int numberOfFiles;
private long totalSize = -1;
private long totalSize;

public long startTime() {
return this.startTime;
Expand All @@ -100,12 +97,12 @@ public void startTime(long startTime) {
this.startTime = startTime;
}

public TimeValue took() {
return new TimeValue(this.took);
public long time() {
return this.time;
}

public void took(long took) {
this.took = took;
public void time(long time) {
this.time = time;
}

public void files(int numberOfFiles, long totalSize) {
Expand All @@ -117,16 +114,16 @@ public int numberOfFiles() {
return numberOfFiles;
}

public ByteSizeValue totalSize() {
return new ByteSizeValue(totalSize);
public long totalSize() {
return totalSize;
}
}

public static class Translog {
private volatile int currentTranslogOperations;

private long startTime = -1;
private long took;
private long startTime;
private long time;

public long startTime() {
return this.startTime;
Expand All @@ -136,12 +133,12 @@ public void startTime(long startTime) {
this.startTime = startTime;
}

public TimeValue took() {
return new TimeValue(this.took);
public long time() {
return this.time;
}

public void took(long took) {
this.took = took;
public void time(long time) {
this.time = time;
}

public void addTranslogOperations(long count) {
Expand Down

0 comments on commit a3107bc

Please sign in to comment.