You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2015/02/12 23:49:42 UTC

[1/9] hbase git commit: HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Repository: hbase
Updated Branches:
  refs/heads/0.98 0ffb197a6 -> c391dfbd7
  refs/heads/branch-1 7841bf73b -> f5b40200d
  refs/heads/branch-1.0 bdb938ba2 -> c0e26c3a0


http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-protocol/src/main/protobuf/ClusterStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/ClusterStatus.proto b/hbase-protocol/src/main/protobuf/ClusterStatus.proto
index 7e78395..17f79fe 100644
--- a/hbase-protocol/src/main/protobuf/ClusterStatus.proto
+++ b/hbase-protocol/src/main/protobuf/ClusterStatus.proto
@@ -117,6 +117,19 @@ message RegionLoad {
 
 /* Server-level protobufs */
 
+message ReplicationLoadSink {
+  required uint64 ageOfLastAppliedOp = 1;
+  required uint64 timeStampsOfLastAppliedOp = 2;
+}
+
+message ReplicationLoadSource {
+  required string peerID = 1;
+  required uint64 ageOfLastShippedOp = 2;
+  required uint32 sizeOfLogQueue = 3;
+  required uint64 timeStampOfLastShippedOp = 4;
+  required uint64 replicationLag = 5;
+}
+
 message ServerLoad {
   /** Number of requests since last report. */
   optional uint32 number_of_requests = 1;
@@ -158,6 +171,16 @@ message ServerLoad {
    * The port number that this region server is hosing an info server on.
    */
   optional uint32 info_server_port = 9;
+
+  /**
+   * The replicationLoadSource for the replication Source status of this region server.
+   */
+  repeated ReplicationLoadSource replLoadSource = 10;
+
+  /**
+   * The replicationLoadSink for the replication Sink status of this region server.
+   */
+  optional ReplicationLoadSink replLoadSink = 11;
 }
 
 message LiveServerInfo {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index d03e809..4e6465d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -133,6 +133,7 @@ import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.util.Addressing;
@@ -1120,6 +1121,22 @@ public class HRegionServer extends HasThread implements
     } else {
       serverLoad.setInfoServerPort(-1);
     }
+
+    // for the replicationLoad purpose. Only need to get from one service
+    // either source or sink will get the same info
+    ReplicationSourceService rsources = getReplicationSourceService();
+
+    if (rsources != null) {
+      // always refresh first to get the latest value
+      ReplicationLoad rLoad = rsources.refreshAndGetReplicationLoad();
+      if (rLoad != null) {
+        serverLoad.setReplLoadSink(rLoad.getReplicationLoadSink());
+        for (ClusterStatusProtos.ReplicationLoadSource rLS : rLoad.getReplicationLoadSourceList()) {
+          serverLoad.addReplLoadSource(rLS);
+        }
+      }
+    }
+
     return serverLoad.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
index 92ac823..25a27a9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
@@ -22,11 +22,12 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 /**
- * Gateway to Cluster Replication.  
+ * Gateway to Cluster Replication.
  * Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
  * One such application is a cross-datacenter
  * replication service that can keep two hbase clusters in sync.
@@ -52,4 +53,9 @@ public interface ReplicationService {
    * Stops replication service.
    */
   void stopReplicationService();
+
+  /**
+   * Refresh and Get ReplicationLoad
+   */
+  public ReplicationLoad refreshAndGetReplicationLoad();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
index 0c9d016..37dc1dd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
@@ -71,4 +71,21 @@ public class MetricsSink {
     mss.incrAppliedOps(batchSize);
   }
 
+  /**
+   * Get the Age of Last Applied Op
+   * @return ageOfLastAppliedOp
+   */
+  public long getAgeOfLastAppliedOp() {
+    return mss.getLastAppliedOpAge();
+  }
+
+  /**
+   * Get the TimeStampOfLastAppliedOp. If no replication Op applied yet, the value is the timestamp
+   * at which hbase instance starts
+   * @return timeStampsOfLastAppliedOp;
+   */
+  public long getTimeStampOfLastAppliedOp() {
+    return this.lastTimestampForAge;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
index a734b9c..21296a0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
@@ -36,6 +36,7 @@ public class MetricsSource {
 
   private long lastTimestamp = 0;
   private int lastQueueSize = 0;
+  private String id;
 
   private final MetricsReplicationSourceSource singleSourceSource;
   private final MetricsReplicationSourceSource globalSourceSource;
@@ -46,6 +47,7 @@ public class MetricsSource {
    * @param id Name of the source this class is monitoring
    */
   public MetricsSource(String id) {
+    this.id = id;
     singleSourceSource =
         CompatibilitySingletonFactory.getInstance(MetricsReplicationSourceFactory.class)
             .getSource(id);
@@ -143,4 +145,36 @@ public class MetricsSource {
     globalSourceSource.decrSizeOfLogQueue(lastQueueSize);
     lastQueueSize = 0;
   }
+
+  /**
+   * Get AgeOfLastShippedOp
+   * @return AgeOfLastShippedOp
+   */
+  public Long getAgeOfLastShippedOp() {
+    return singleSourceSource.getLastShippedAge();
+  }
+
+  /**
+   * Get the sizeOfLogQueue
+   * @return sizeOfLogQueue
+   */
+  public int getSizeOfLogQueue() {
+    return this.lastQueueSize;
+  }
+
+  /**
+   * Get the timeStampsOfLastShippedOp
+   * @return lastTimestampForAge
+   */
+  public long getTimeStampOfLastShippedOp() {
+    return lastTimestamp;
+  }
+
+  /**
+   * Get the slave peer ID
+   * @return peerID
+   */
+  public String getPeerID() {
+    return id;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 4729644..78bb92e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.hbase.HConstants.REPLICATION_ENABLE_KEY;
 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
@@ -66,7 +67,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * Gateway to Replication.  Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
  */
 @InterfaceAudience.Private
-public class Replication extends WALActionsListener.Base implements 
+public class Replication extends WALActionsListener.Base implements
   ReplicationSourceService, ReplicationSinkService {
   private static final Log LOG =
       LogFactory.getLog(Replication.class);
@@ -82,6 +83,8 @@ public class Replication extends WALActionsListener.Base implements
   /** Statistics thread schedule pool */
   private ScheduledExecutorService scheduleThreadPool;
   private int statsThreadPeriod;
+  // ReplicationLoad to access replication metrics
+  private ReplicationLoad replicationLoad;
 
   /**
    * Instantiate the replication management (if rep is enabled).
@@ -138,11 +141,13 @@ public class Replication extends WALActionsListener.Base implements
       this.statsThreadPeriod =
           this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
       LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
+      this.replicationLoad = new ReplicationLoad();
     } else {
       this.replicationManager = null;
       this.replicationQueues = null;
       this.replicationPeers = null;
       this.replicationTracker = null;
+      this.replicationLoad = null;
     }
   }
 
@@ -310,4 +315,29 @@ public class Replication extends WALActionsListener.Base implements
       }
     }
   }
+
+  @Override
+  public ReplicationLoad refreshAndGetReplicationLoad() {
+    if (this.replicationLoad == null) {
+      return null;
+    }
+    // always build for latest data
+    buildReplicationLoad();
+    return this.replicationLoad;
+  }
+
+  private void buildReplicationLoad() {
+    // get source
+    List<ReplicationSourceInterface> sources = this.replicationManager.getSources();
+    List<MetricsSource> sourceMetricsList = new ArrayList<MetricsSource>();
+
+    for (ReplicationSourceInterface source : sources) {
+      if (source instanceof ReplicationSource) {
+        sourceMetricsList.add(((ReplicationSource) source).getSourceMetrics());
+      }
+    }
+    // get sink
+    MetricsSink sinkMetrics = this.replicationSink.getSinkMetrics();
+    this.replicationLoad.buildReplicationLoad(sourceMetricsList, sinkMetrics);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
new file mode 100644
index 0000000..b3f3ecb
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
@@ -0,0 +1,151 @@
+/**
+ * Copyright 2014 The Apache Software Foundation Licensed to the Apache Software Foundation (ASF)
+ * under one or more contributor license agreements. See the NOTICE file distributed with this work
+ * for additional information regarding copyright ownership. The ASF 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.apache.hadoop.hbase.replication.regionserver;
+
+import java.util.Date;
+import java.util.List;
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Strings;
+
+/**
+ * This class is used for exporting some of the info from replication metrics
+ */
+@InterfaceAudience.Private
+public class ReplicationLoad {
+
+  // Empty load instance.
+  public static final ReplicationLoad EMPTY_REPLICATIONLOAD = new ReplicationLoad();
+
+  private List<MetricsSource> sourceMetricsList;
+  private MetricsSink sinkMetrics;
+
+  private List<ClusterStatusProtos.ReplicationLoadSource> replicationLoadSourceList;
+  private ClusterStatusProtos.ReplicationLoadSink replicationLoadSink;
+
+  /** default constructor */
+  public ReplicationLoad() {
+    super();
+  }
+
+  /**
+   * buildReplicationLoad
+   * @param srMetricsList
+   * @param skMetrics
+   */
+
+  public void buildReplicationLoad(final List<MetricsSource> srMetricsList,
+      final MetricsSink skMetrics) {
+    this.sourceMetricsList = srMetricsList;
+    this.sinkMetrics = skMetrics;
+
+    // build the SinkLoad
+    ClusterStatusProtos.ReplicationLoadSink.Builder rLoadSinkBuild =
+        ClusterStatusProtos.ReplicationLoadSink.newBuilder();
+    rLoadSinkBuild.setAgeOfLastAppliedOp(sinkMetrics.getAgeOfLastAppliedOp());
+    rLoadSinkBuild.setTimeStampsOfLastAppliedOp(sinkMetrics.getTimeStampOfLastAppliedOp());
+    this.replicationLoadSink = rLoadSinkBuild.build();
+
+    // build the SourceLoad List
+    this.replicationLoadSourceList = new ArrayList<ClusterStatusProtos.ReplicationLoadSource>();
+    for (MetricsSource sm : this.sourceMetricsList) {
+      long ageOfLastShippedOp = sm.getAgeOfLastShippedOp();
+      int sizeOfLogQueue = sm.getSizeOfLogQueue();
+      long timeStampOfLastShippedOp = sm.getTimeStampOfLastShippedOp();
+      long replicationLag;
+      long timePassedAfterLastShippedOp =
+          EnvironmentEdgeManager.currentTime() - timeStampOfLastShippedOp;
+      if (sizeOfLogQueue != 0) {
+        // err on the large side
+        replicationLag = Math.max(ageOfLastShippedOp, timePassedAfterLastShippedOp);
+      } else if (timePassedAfterLastShippedOp < 2 * ageOfLastShippedOp) {
+        replicationLag = ageOfLastShippedOp; // last shipped happen recently
+      } else {
+        // last shipped may happen last night,
+        // so NO real lag although ageOfLastShippedOp is non-zero
+        replicationLag = 0;
+      }
+
+      ClusterStatusProtos.ReplicationLoadSource.Builder rLoadSourceBuild =
+          ClusterStatusProtos.ReplicationLoadSource.newBuilder();
+      rLoadSourceBuild.setPeerID(sm.getPeerID());
+      rLoadSourceBuild.setAgeOfLastShippedOp(ageOfLastShippedOp);
+      rLoadSourceBuild.setSizeOfLogQueue(sizeOfLogQueue);
+      rLoadSourceBuild.setTimeStampOfLastShippedOp(timeStampOfLastShippedOp);
+      rLoadSourceBuild.setReplicationLag(replicationLag);
+
+      this.replicationLoadSourceList.add(rLoadSourceBuild.build());
+    }
+
+  }
+
+  /**
+   * sourceToString
+   * @return a string contains sourceReplicationLoad information
+   */
+  public String sourceToString() {
+    if (this.sourceMetricsList == null) return null;
+
+    StringBuilder sb = new StringBuilder();
+
+    for (ClusterStatusProtos.ReplicationLoadSource rls : this.replicationLoadSourceList) {
+
+      sb = Strings.appendKeyValue(sb, "\n           PeerID", rls.getPeerID());
+      sb = Strings.appendKeyValue(sb, "AgeOfLastShippedOp", rls.getAgeOfLastShippedOp());
+      sb = Strings.appendKeyValue(sb, "SizeOfLogQueue", rls.getSizeOfLogQueue());
+      sb =
+          Strings.appendKeyValue(sb, "TimeStampsOfLastShippedOp",
+            (new Date(rls.getTimeStampOfLastShippedOp()).toString()));
+      sb = Strings.appendKeyValue(sb, "Replication Lag", rls.getReplicationLag());
+    }
+
+    return sb.toString();
+  }
+
+  /**
+   * sinkToString
+   * @return a string contains sinkReplicationLoad information
+   */
+  public String sinkToString() {
+    if (this.replicationLoadSink == null) return null;
+
+    StringBuilder sb = new StringBuilder();
+    sb =
+        Strings.appendKeyValue(sb, "AgeOfLastAppliedOp",
+          this.replicationLoadSink.getAgeOfLastAppliedOp());
+    sb =
+        Strings.appendKeyValue(sb, "TimeStampsOfLastAppliedOp",
+          (new Date(this.replicationLoadSink.getTimeStampsOfLastAppliedOp()).toString()));
+
+    return sb.toString();
+  }
+
+  public ClusterStatusProtos.ReplicationLoadSink getReplicationLoadSink() {
+    return this.replicationLoadSink;
+  }
+
+  public List<ClusterStatusProtos.ReplicationLoadSource> getReplicationLoadSourceList() {
+    return this.replicationLoadSourceList;
+  }
+
+  /**
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    return this.sourceToString() + System.getProperty("line.separator") + this.sinkToString();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
index 9a60131..3276418 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
@@ -254,4 +254,12 @@ public class ReplicationSink {
       "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() +
       ", total replicated edits: " + this.totalReplicatedEdits;
   }
+
+  /**
+   * Get replication Sink Metrics
+   * @return MetricsSink
+   */
+  public MetricsSink getSinkMetrics() {
+    return this.metrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 6e2ef2d..ccde5c8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -868,4 +868,12 @@ public class ReplicationSource extends Thread
       ", currently replicating from: " + this.currentPath +
       " at position: " + position;
   }
+
+  /**
+   * Get Replication Source Metrics
+   * @return sourceMetrics
+   */
+  public MetricsSource getSourceMetrics() {
+    return this.metrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index 4163b66..d8d735f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -31,13 +31,16 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -48,6 +51,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
@@ -556,4 +560,45 @@ public class TestReplicationSmallTests extends TestReplicationBase {
     hadmin.close();
   }
 
+  /**
+   * Test for HBASE-9531
+   * put a few rows into htable1, which should be replicated to htable2
+   * create a ClusterStatus instance 'status' from HBaseAdmin
+   * test : status.getLoad(server).getReplicationLoadSourceList()
+   * test : status.getLoad(server).getReplicationLoadSink()
+   * * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testReplicationStatus() throws Exception {
+    LOG.info("testReplicationStatus");
+
+    try (Admin admin = utility1.getConnection().getAdmin()) {
+
+      final byte[] qualName = Bytes.toBytes("q");
+      Put p;
+
+      for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+        p = new Put(Bytes.toBytes("row" + i));
+        p.add(famName, qualName, Bytes.toBytes("val" + i));
+        htable1.put(p);
+      }
+
+      ClusterStatus status = admin.getClusterStatus();
+
+      for (ServerName server : status.getServers()) {
+        ServerLoad sl = status.getLoad(server);
+        List<ReplicationLoadSource> rLoadSourceList = sl.getReplicationLoadSourceList();
+        ReplicationLoadSink rLoadSink = sl.getReplicationLoadSink();
+
+        // check SourceList has at least one entry
+        assertTrue("failed to get ReplicationLoadSourceList", (rLoadSourceList.size() > 0));
+
+        // check Sink exist only as it is difficult to verify the value on the fly
+        assertTrue("failed to get ReplicationLoadSink.AgeOfLastShippedOp ",
+          (rLoadSink.getAgeOfLastAppliedOp() >= 0));
+        assertTrue("failed to get ReplicationLoadSink.TimeStampsOfLastAppliedOp ",
+          (rLoadSink.getTimeStampsOfLastAppliedOp() >= 0));
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index c0ea862..35ee36c 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -608,7 +608,7 @@ module Hbase
       end
     end
 
-    def status(format)
+    def status(format, type)
       status = @admin.getClusterStatus()
       if format == "detailed"
         puts("version %s" % [ status.getHBaseVersion() ])
@@ -635,6 +635,46 @@ module Hbase
         for server in status.getDeadServerNames()
           puts("    %s" % [ server ])
         end
+      elsif format == "replication"
+        #check whether replication is enabled or not
+        if (!@admin.getConfiguration().getBoolean(org.apache.hadoop.hbase.HConstants::REPLICATION_ENABLE_KEY, 
+          org.apache.hadoop.hbase.HConstants::REPLICATION_ENABLE_DEFAULT))
+          puts("Please enable replication first.")
+        else
+          puts("version %s" % [ status.getHBaseVersion() ])
+          puts("%d live servers" % [ status.getServersSize() ])
+          for server in status.getServers()
+            sl = status.getLoad(server)
+            rSinkString   = "       SINK  :"
+            rSourceString = "       SOURCE:"
+            rLoadSink = sl.getReplicationLoadSink()
+            rSinkString << " AgeOfLastAppliedOp=" + rLoadSink.getAgeOfLastAppliedOp().to_s
+            rSinkString << ", TimeStampsOfLastAppliedOp=" + 
+			    (java.util.Date.new(rLoadSink.getTimeStampsOfLastAppliedOp())).toString()
+            rLoadSourceList = sl.getReplicationLoadSourceList()
+            index = 0
+            while index < rLoadSourceList.size()
+              rLoadSource = rLoadSourceList.get(index)
+              rSourceString << " PeerID=" + rLoadSource.getPeerID()
+              rSourceString << ", AgeOfLastShippedOp=" + rLoadSource.getAgeOfLastShippedOp().to_s
+              rSourceString << ", SizeOfLogQueue=" + rLoadSource.getSizeOfLogQueue().to_s
+              rSourceString << ", TimeStampsOfLastShippedOp=" + 
+			      (java.util.Date.new(rLoadSource.getTimeStampOfLastShippedOp())).toString()
+              rSourceString << ", Replication Lag=" + rLoadSource.getReplicationLag().to_s
+              index = index + 1
+            end
+            puts("    %s:" %
+            [ server.getHostname() ])
+            if type.casecmp("SOURCE") == 0
+              puts("%s" % rSourceString)
+            elsif type.casecmp("SINK") == 0
+              puts("%s" % rSinkString)
+            else
+              puts("%s" % rSourceString)
+              puts("%s" % rSinkString)
+            end
+          end
+        end
       elsif format == "simple"
         load = 0
         regions = 0

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-shell/src/main/ruby/shell/commands/status.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/status.rb b/hbase-shell/src/main/ruby/shell/commands/status.rb
index f72c13c..b22b272 100644
--- a/hbase-shell/src/main/ruby/shell/commands/status.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/status.rb
@@ -22,18 +22,21 @@ module Shell
     class Status < Command
       def help
         return <<-EOF
-Show cluster status. Can be 'summary', 'simple', or 'detailed'. The
+Show cluster status. Can be 'summary', 'simple', 'detailed', or 'replication'. The
 default is 'summary'. Examples:
 
   hbase> status
   hbase> status 'simple'
   hbase> status 'summary'
   hbase> status 'detailed'
+  hbase> status 'replication'
+  hbase> status 'replication', 'source'
+  hbase> status 'replication', 'sink'
 EOF
       end
 
-      def command(format = 'summary')
-        admin.status(format)
+      def command(format = 'summary',type = 'both')
+        admin.status(format, type)
       end
     end
   end

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-shell/src/test/ruby/hbase/admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/admin_test.rb b/hbase-shell/src/test/ruby/hbase/admin_test.rb
index caede3a..1925864 100644
--- a/hbase-shell/src/test/ruby/hbase/admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/admin_test.rb
@@ -356,5 +356,17 @@ module Hbase
       assert_not_equal(nil, table)
       table.close
     end
+
+    define_test "Get replication status" do
+      replication_status("replication", "both")
+    end
+
+    define_test "Get replication source metrics information" do
+      replication_status("replication", "source")
+    end
+
+    define_test "Get replication sink metrics information" do
+      replication_status("replication", "sink")
+    end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-shell/src/test/ruby/test_helper.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/test_helper.rb b/hbase-shell/src/test/ruby/test_helper.rb
index 5579761..5dfafc5 100644
--- a/hbase-shell/src/test/ruby/test_helper.rb
+++ b/hbase-shell/src/test/ruby/test_helper.rb
@@ -94,6 +94,10 @@ module Hbase
         puts "IGNORING DROP TABLE ERROR: #{e}"
       end
     end
+
+    def replication_status(format,type)
+      return admin.status(format,type)
+    end
   end
 end
 


[7/9] hbase git commit: HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-protocol/src/main/protobuf/ClusterStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/ClusterStatus.proto b/hbase-protocol/src/main/protobuf/ClusterStatus.proto
index 7e78395..17f79fe 100644
--- a/hbase-protocol/src/main/protobuf/ClusterStatus.proto
+++ b/hbase-protocol/src/main/protobuf/ClusterStatus.proto
@@ -117,6 +117,19 @@ message RegionLoad {
 
 /* Server-level protobufs */
 
+message ReplicationLoadSink {
+  required uint64 ageOfLastAppliedOp = 1;
+  required uint64 timeStampsOfLastAppliedOp = 2;
+}
+
+message ReplicationLoadSource {
+  required string peerID = 1;
+  required uint64 ageOfLastShippedOp = 2;
+  required uint32 sizeOfLogQueue = 3;
+  required uint64 timeStampOfLastShippedOp = 4;
+  required uint64 replicationLag = 5;
+}
+
 message ServerLoad {
   /** Number of requests since last report. */
   optional uint32 number_of_requests = 1;
@@ -158,6 +171,16 @@ message ServerLoad {
    * The port number that this region server is hosing an info server on.
    */
   optional uint32 info_server_port = 9;
+
+  /**
+   * The replicationLoadSource for the replication Source status of this region server.
+   */
+  repeated ReplicationLoadSource replLoadSource = 10;
+
+  /**
+   * The replicationLoadSink for the replication Sink status of this region server.
+   */
+  optional ReplicationLoadSink replLoadSink = 11;
 }
 
 message LiveServerInfo {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index e8e8c1b..edb799c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -218,6 +218,7 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -1170,6 +1171,22 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
     } else {
       serverLoad.setInfoServerPort(-1);
     }
+
+    // for the replicationLoad purpose. Only need to get from one service
+    // either source or sink will get the same info
+    ReplicationSourceService rsources = getReplicationSourceService();
+
+    if (rsources != null) {
+      // always refresh first to get the latest value
+      ReplicationLoad rLoad = rsources.refreshAndGetReplicationLoad();
+      if (rLoad != null) {
+        serverLoad.setReplLoadSink(rLoad.getReplicationLoadSink());
+        for (ClusterStatusProtos.ReplicationLoadSource rLS : rLoad.getReplicationLoadSourceList()) {
+          serverLoad.addReplLoadSource(rLS);
+        }
+      }
+    }
+
     return serverLoad.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
index 92ac823..25a27a9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
@@ -22,11 +22,12 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 /**
- * Gateway to Cluster Replication.  
+ * Gateway to Cluster Replication.
  * Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
  * One such application is a cross-datacenter
  * replication service that can keep two hbase clusters in sync.
@@ -52,4 +53,9 @@ public interface ReplicationService {
    * Stops replication service.
    */
   void stopReplicationService();
+
+  /**
+   * Refresh and Get ReplicationLoad
+   */
+  public ReplicationLoad refreshAndGetReplicationLoad();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
index 9946e37..b1087d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
@@ -78,4 +78,21 @@ public class MetricsSink {
     mss.incrAppliedOps(batchSize);
   }
 
+  /**
+   * Get the Age of Last Applied Op
+   * @return ageOfLastAppliedOp
+   */
+  public long getAgeOfLastAppliedOp() {
+    return mss.getLastAppliedOpAge();
+  }
+
+  /**
+   * Get the TimeStampOfLastAppliedOp. If no replication Op applied yet, the value is the timestamp
+   * at which hbase instance starts
+   * @return timeStampsOfLastAppliedOp;
+   */
+  public long getTimeStampOfLastAppliedOp() {
+    return this.lastTimestampForAge;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
index a2560e6..b5dee6f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
@@ -53,6 +53,7 @@ public class MetricsSource {
   
   private long lastTimestamp = 0;
   private int lastQueueSize = 0;
+  private String id;
 
   private final MetricsReplicationSourceSource singleSourceSource;
   private final MetricsReplicationSourceSource globalSourceSource;
@@ -63,6 +64,7 @@ public class MetricsSource {
    * @param id Name of the source this class is monitoring
    */
   public MetricsSource(String id) {
+    this.id = id;
     singleSourceSource =
       CompatibilitySingletonFactory.getInstance(MetricsReplicationSourceFactory.class)
         .getSource(id);
@@ -162,4 +164,36 @@ public class MetricsSource {
     globalSourceSource.decrSizeOfLogQueue(lastQueueSize);
     lastQueueSize = 0;
   }
+
+  /**
+   * Get AgeOfLastShippedOp
+   * @return AgeOfLastShippedOp
+   */
+  public Long getAgeOfLastShippedOp() {
+    return singleSourceSource.getLastShippedAge();
+  }
+
+  /**
+   * Get the sizeOfLogQueue
+   * @return sizeOfLogQueue
+   */
+  public int getSizeOfLogQueue() {
+    return this.lastQueueSize;
+  }
+
+  /**
+   * Get the timeStampsOfLastShippedOp
+   * @return lastTimestampForAge
+   */
+  public long getTimeStampOfLastShippedOp() {
+    return lastTimestamp;
+  }
+
+  /**
+   * Get the slave peer ID
+   * @return peerID
+   */
+  public String getPeerID() {
+    return id;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index b665d70..04c6f24 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
@@ -80,6 +81,8 @@ public class Replication implements WALActionsListener,
   /** Statistics thread schedule pool */
   private ScheduledExecutorService scheduleThreadPool;
   private int statsThreadPeriod;
+  // ReplicationLoad to access replication metrics
+  private ReplicationLoad replicationLoad;
 
   /**
    * Instantiate the replication management (if rep is enabled).
@@ -136,11 +139,13 @@ public class Replication implements WALActionsListener,
       this.statsThreadPeriod =
           this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
       LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
+      this.replicationLoad = new ReplicationLoad();
     } else {
       this.replicationManager = null;
       this.replicationQueues = null;
       this.replicationPeers = null;
       this.replicationTracker = null;
+      this.replicationLoad = null;
     }
   }
 
@@ -334,4 +339,29 @@ public class Replication implements WALActionsListener,
       }
     }
   }
+
+  @Override
+  public ReplicationLoad refreshAndGetReplicationLoad() {
+    if (this.replicationLoad == null) {
+      return null;
+    }
+    // always build for latest data
+    buildReplicationLoad();
+    return this.replicationLoad;
+  }
+
+  private void buildReplicationLoad() {
+    // get source
+    List<ReplicationSourceInterface> sources = this.replicationManager.getSources();
+    List<MetricsSource> sourceMetricsList = new ArrayList<MetricsSource>();
+
+    for (ReplicationSourceInterface source : sources) {
+      if (source instanceof ReplicationSource) {
+        sourceMetricsList.add(((ReplicationSource) source).getSourceMetrics());
+      }
+    }
+    // get sink
+    MetricsSink sinkMetrics = this.replicationSink.getSinkMetrics();
+    this.replicationLoad.buildReplicationLoad(sourceMetricsList, sinkMetrics);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
new file mode 100644
index 0000000..af034a3
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
@@ -0,0 +1,151 @@
+/**
+ * Copyright 2014 The Apache Software Foundation Licensed to the Apache Software Foundation (ASF)
+ * under one or more contributor license agreements. See the NOTICE file distributed with this work
+ * for additional information regarding copyright ownership. The ASF 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.apache.hadoop.hbase.replication.regionserver;
+
+import java.util.Date;
+import java.util.List;
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Strings;
+
+/**
+ * This class is used for exporting some of the info from replication metrics
+ */
+@InterfaceAudience.Private
+public class ReplicationLoad {
+
+  // Empty load instance.
+  public static final ReplicationLoad EMPTY_REPLICATIONLOAD = new ReplicationLoad();
+
+  private List<MetricsSource> sourceMetricsList;
+  private MetricsSink sinkMetrics;
+
+  private List<ClusterStatusProtos.ReplicationLoadSource> replicationLoadSourceList;
+  private ClusterStatusProtos.ReplicationLoadSink replicationLoadSink;
+
+  /** default constructor */
+  public ReplicationLoad() {
+    super();
+  }
+
+  /**
+   * buildReplicationLoad
+   * @param srMetricsList
+   * @param skMetrics
+   */
+
+  public void buildReplicationLoad(final List<MetricsSource> srMetricsList,
+      final MetricsSink skMetrics) {
+    this.sourceMetricsList = srMetricsList;
+    this.sinkMetrics = skMetrics;
+
+    // build the SinkLoad
+    ClusterStatusProtos.ReplicationLoadSink.Builder rLoadSinkBuild =
+        ClusterStatusProtos.ReplicationLoadSink.newBuilder();
+    rLoadSinkBuild.setAgeOfLastAppliedOp(sinkMetrics.getAgeOfLastAppliedOp());
+    rLoadSinkBuild.setTimeStampsOfLastAppliedOp(sinkMetrics.getTimeStampOfLastAppliedOp());
+    this.replicationLoadSink = rLoadSinkBuild.build();
+
+    // build the SourceLoad List
+    this.replicationLoadSourceList = new ArrayList<ClusterStatusProtos.ReplicationLoadSource>();
+    for (MetricsSource sm : this.sourceMetricsList) {
+      long ageOfLastShippedOp = sm.getAgeOfLastShippedOp();
+      int sizeOfLogQueue = sm.getSizeOfLogQueue();
+      long timeStampOfLastShippedOp = sm.getTimeStampOfLastShippedOp();
+      long replicationLag;
+      long timePassedAfterLastShippedOp =
+          EnvironmentEdgeManager.currentTimeMillis() - timeStampOfLastShippedOp;
+      if (sizeOfLogQueue != 0) {
+        // err on the large side
+        replicationLag = Math.max(ageOfLastShippedOp, timePassedAfterLastShippedOp);
+      } else if (timePassedAfterLastShippedOp < 2 * ageOfLastShippedOp) {
+        replicationLag = ageOfLastShippedOp; // last shipped happen recently
+      } else {
+        // last shipped may happen last night,
+        // so NO real lag although ageOfLastShippedOp is non-zero
+        replicationLag = 0;
+      }
+
+      ClusterStatusProtos.ReplicationLoadSource.Builder rLoadSourceBuild =
+          ClusterStatusProtos.ReplicationLoadSource.newBuilder();
+      rLoadSourceBuild.setPeerID(sm.getPeerID());
+      rLoadSourceBuild.setAgeOfLastShippedOp(ageOfLastShippedOp);
+      rLoadSourceBuild.setSizeOfLogQueue(sizeOfLogQueue);
+      rLoadSourceBuild.setTimeStampOfLastShippedOp(timeStampOfLastShippedOp);
+      rLoadSourceBuild.setReplicationLag(replicationLag);
+
+      this.replicationLoadSourceList.add(rLoadSourceBuild.build());
+    }
+
+  }
+
+  /**
+   * sourceToString
+   * @return a string contains sourceReplicationLoad information
+   */
+  public String sourceToString() {
+    if (this.sourceMetricsList == null) return null;
+
+    StringBuilder sb = new StringBuilder();
+
+    for (ClusterStatusProtos.ReplicationLoadSource rls : this.replicationLoadSourceList) {
+
+      sb = Strings.appendKeyValue(sb, "\n           PeerID", rls.getPeerID());
+      sb = Strings.appendKeyValue(sb, "AgeOfLastShippedOp", rls.getAgeOfLastShippedOp());
+      sb = Strings.appendKeyValue(sb, "SizeOfLogQueue", rls.getSizeOfLogQueue());
+      sb =
+          Strings.appendKeyValue(sb, "TimeStampsOfLastShippedOp",
+            (new Date(rls.getTimeStampOfLastShippedOp()).toString()));
+      sb = Strings.appendKeyValue(sb, "Replication Lag", rls.getReplicationLag());
+    }
+
+    return sb.toString();
+  }
+
+  /**
+   * sinkToString
+   * @return a string contains sinkReplicationLoad information
+   */
+  public String sinkToString() {
+    if (this.replicationLoadSink == null) return null;
+
+    StringBuilder sb = new StringBuilder();
+    sb =
+        Strings.appendKeyValue(sb, "AgeOfLastAppliedOp",
+          this.replicationLoadSink.getAgeOfLastAppliedOp());
+    sb =
+        Strings.appendKeyValue(sb, "TimeStampsOfLastAppliedOp",
+          (new Date(this.replicationLoadSink.getTimeStampsOfLastAppliedOp()).toString()));
+
+    return sb.toString();
+  }
+
+  public ClusterStatusProtos.ReplicationLoadSink getReplicationLoadSink() {
+    return this.replicationLoadSink;
+  }
+
+  public List<ClusterStatusProtos.ReplicationLoadSource> getReplicationLoadSourceList() {
+    return this.replicationLoadSourceList;
+  }
+
+  /**
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    return this.sourceToString() + System.getProperty("line.separator") + this.sinkToString();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
index 746f1fd..cd45461 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
@@ -255,4 +255,12 @@ public class ReplicationSink {
       "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() +
       ", total replicated edits: " + this.totalReplicatedEdits;
   }
+
+  /**
+   * Get replication Sink Metrics
+   * @return MetricsSink
+   */
+  public MetricsSink getSinkMetrics() {
+    return this.metrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 1e7f5c9..17b8ef0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -856,4 +856,12 @@ public class ReplicationSource extends Thread
       ", currently replicating from: " + this.currentPath +
       " at position: " + position;
   }
+
+  /**
+   * Get Replication Source Metrics
+   * @return sourceMetrics
+   */
+  public MetricsSource getSourceMetrics() {
+    return this.metrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index a5ab2cd..5e02198 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -30,11 +30,13 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -50,6 +52,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
@@ -555,4 +558,48 @@ public class TestReplicationSmallTests extends TestReplicationBase {
     hadmin.close();
   }
 
+  /**
+   * Test for HBASE-9531
+   * put a few rows into htable1, which should be replicated to htable2
+   * create a ClusterStatus instance 'status' from HBaseAdmin
+   * test : status.getLoad(server).getReplicationLoadSourceList()
+   * test : status.getLoad(server).getReplicationLoadSink()
+   * * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testReplicationStatus() throws Exception {
+    LOG.info("testReplicationStatus");
+
+    HBaseAdmin admin = utility1.getHBaseAdmin();
+    try {
+
+      final byte[] qualName = Bytes.toBytes("q");
+      Put p;
+
+      for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+        p = new Put(Bytes.toBytes("row" + i));
+        p.add(famName, qualName, Bytes.toBytes("val" + i));
+        htable1.put(p);
+      }
+
+      ClusterStatus status = admin.getClusterStatus();
+
+      for (ServerName server : status.getServers()) {
+        ServerLoad sl = status.getLoad(server);
+        List<ReplicationLoadSource> rLoadSourceList = sl.getReplicationLoadSourceList();
+        ReplicationLoadSink rLoadSink = sl.getReplicationLoadSink();
+
+        // check SourceList has at least one entry
+        assertTrue("failed to get ReplicationLoadSourceList", (rLoadSourceList.size() > 0));
+
+        // check Sink exist only as it is difficult to verify the value on the fly
+        assertTrue("failed to get ReplicationLoadSink.AgeOfLastShippedOp ",
+          (rLoadSink.getAgeOfLastAppliedOp() >= 0));
+        assertTrue("failed to get ReplicationLoadSink.TimeStampsOfLastAppliedOp ",
+          (rLoadSink.getTimeStampsOfLastAppliedOp() >= 0));
+      }
+    } finally {
+      admin.close();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index f267857..4964836 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -585,7 +585,7 @@ module Hbase
       end
     end
 
-    def status(format)
+    def status(format, type)
       status = @admin.getClusterStatus()
       if format == "detailed"
         puts("version %s" % [ status.getHBaseVersion() ])
@@ -612,6 +612,46 @@ module Hbase
         for server in status.getDeadServerNames()
           puts("    %s" % [ server ])
         end
+      elsif format == "replication"
+        #check whether replication is enabled or not
+        if (!@admin.getConfiguration().getBoolean(org.apache.hadoop.hbase.HConstants::REPLICATION_ENABLE_KEY, 
+          org.apache.hadoop.hbase.HConstants::REPLICATION_ENABLE_DEFAULT))
+          puts("Please enable replication first.")
+        else
+          puts("version %s" % [ status.getHBaseVersion() ])
+          puts("%d live servers" % [ status.getServersSize() ])
+          for server in status.getServers()
+            sl = status.getLoad(server)
+            rSinkString   = "       SINK  :"
+            rSourceString = "       SOURCE:"
+            rLoadSink = sl.getReplicationLoadSink()
+            rSinkString << " AgeOfLastAppliedOp=" + rLoadSink.getAgeOfLastAppliedOp().to_s
+            rSinkString << ", TimeStampsOfLastAppliedOp=" + 
+			    (java.util.Date.new(rLoadSink.getTimeStampsOfLastAppliedOp())).toString()
+            rLoadSourceList = sl.getReplicationLoadSourceList()
+            index = 0
+            while index < rLoadSourceList.size()
+              rLoadSource = rLoadSourceList.get(index)
+              rSourceString << " PeerID=" + rLoadSource.getPeerID()
+              rSourceString << ", AgeOfLastShippedOp=" + rLoadSource.getAgeOfLastShippedOp().to_s
+              rSourceString << ", SizeOfLogQueue=" + rLoadSource.getSizeOfLogQueue().to_s
+              rSourceString << ", TimeStampsOfLastShippedOp=" + 
+			      (java.util.Date.new(rLoadSource.getTimeStampOfLastShippedOp())).toString()
+              rSourceString << ", Replication Lag=" + rLoadSource.getReplicationLag().to_s
+              index = index + 1
+            end
+            puts("    %s:" %
+            [ server.getHostname() ])
+            if type.casecmp("SOURCE") == 0
+              puts("%s" % rSourceString)
+            elsif type.casecmp("SINK") == 0
+              puts("%s" % rSinkString)
+            else
+              puts("%s" % rSourceString)
+              puts("%s" % rSinkString)
+            end
+          end
+        end
       elsif format == "simple"
         load = 0
         regions = 0

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-shell/src/main/ruby/shell/commands/status.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/status.rb b/hbase-shell/src/main/ruby/shell/commands/status.rb
index f72c13c..b22b272 100644
--- a/hbase-shell/src/main/ruby/shell/commands/status.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/status.rb
@@ -22,18 +22,21 @@ module Shell
     class Status < Command
       def help
         return <<-EOF
-Show cluster status. Can be 'summary', 'simple', or 'detailed'. The
+Show cluster status. Can be 'summary', 'simple', 'detailed', or 'replication'. The
 default is 'summary'. Examples:
 
   hbase> status
   hbase> status 'simple'
   hbase> status 'summary'
   hbase> status 'detailed'
+  hbase> status 'replication'
+  hbase> status 'replication', 'source'
+  hbase> status 'replication', 'sink'
 EOF
       end
 
-      def command(format = 'summary')
-        admin.status(format)
+      def command(format = 'summary',type = 'both')
+        admin.status(format, type)
       end
     end
   end

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-shell/src/test/ruby/hbase/admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/admin_test.rb b/hbase-shell/src/test/ruby/hbase/admin_test.rb
index 0b12df9..50c7235 100644
--- a/hbase-shell/src/test/ruby/hbase/admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/admin_test.rb
@@ -343,5 +343,17 @@ module Hbase
       table = table(@test_name)
       assert_not_equal(nil, table)
     end
+
+    define_test "Get replication status" do
+      replication_status("replication", "both")
+    end
+
+    define_test "Get replication source metrics information" do
+      replication_status("replication", "source")
+    end
+
+    define_test "Get replication sink metrics information" do
+      replication_status("replication", "sink")
+    end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-shell/src/test/ruby/test_helper.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/test_helper.rb b/hbase-shell/src/test/ruby/test_helper.rb
index 0540a57..a776c23 100644
--- a/hbase-shell/src/test/ruby/test_helper.rb
+++ b/hbase-shell/src/test/ruby/test_helper.rb
@@ -90,6 +90,10 @@ module Hbase
         puts "IGNORING DROP TABLE ERROR: #{e}"
       end
     end
+
+    def replication_status(format,type)
+      return admin.status(format,type)
+    end
   end
 end
 


[8/9] hbase git commit: HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
index 5bc44ff..95a6e42 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
@@ -4348,273 +4348,48 @@ public final class ClusterStatusProtos {
     // @@protoc_insertion_point(class_scope:RegionLoad)
   }
 
-  public interface ServerLoadOrBuilder
+  public interface ReplicationLoadSinkOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // optional uint32 number_of_requests = 1;
-    /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
-     */
-    boolean hasNumberOfRequests();
-    /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
-     */
-    int getNumberOfRequests();
-
-    // optional uint32 total_number_of_requests = 2;
-    /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
-     */
-    boolean hasTotalNumberOfRequests();
-    /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
-     */
-    int getTotalNumberOfRequests();
-
-    // optional uint32 used_heap_MB = 3;
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    boolean hasUsedHeapMB();
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    int getUsedHeapMB();
-
-    // optional uint32 max_heap_MB = 4;
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    boolean hasMaxHeapMB();
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    int getMaxHeapMB();
-
-    // repeated .RegionLoad region_loads = 5;
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad> 
-        getRegionLoadsList();
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad getRegionLoads(int index);
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    int getRegionLoadsCount();
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder> 
-        getRegionLoadsOrBuilderList();
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder getRegionLoadsOrBuilder(
-        int index);
-
-    // repeated .Coprocessor coprocessors = 6;
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor> 
-        getCoprocessorsList();
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor getCoprocessors(int index);
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    int getCoprocessorsCount();
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder> 
-        getCoprocessorsOrBuilderList();
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder getCoprocessorsOrBuilder(
-        int index);
-
-    // optional uint64 report_start_time = 7;
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    boolean hasReportStartTime();
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    long getReportStartTime();
-
-    // optional uint64 report_end_time = 8;
+    // required uint64 ageOfLastAppliedOp = 1;
     /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    boolean hasReportEndTime();
+    boolean hasAgeOfLastAppliedOp();
     /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    long getReportEndTime();
+    long getAgeOfLastAppliedOp();
 
-    // optional uint32 info_server_port = 9;
+    // required uint64 timeStampsOfLastAppliedOp = 2;
     /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    boolean hasInfoServerPort();
+    boolean hasTimeStampsOfLastAppliedOp();
     /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    int getInfoServerPort();
+    long getTimeStampsOfLastAppliedOp();
   }
   /**
-   * Protobuf type {@code ServerLoad}
+   * Protobuf type {@code ReplicationLoadSink}
    */
-  public static final class ServerLoad extends
+  public static final class ReplicationLoadSink extends
       com.google.protobuf.GeneratedMessage
-      implements ServerLoadOrBuilder {
-    // Use ServerLoad.newBuilder() to construct.
-    private ServerLoad(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements ReplicationLoadSinkOrBuilder {
+    // Use ReplicationLoadSink.newBuilder() to construct.
+    private ReplicationLoadSink(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private ServerLoad(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private ReplicationLoadSink(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final ServerLoad defaultInstance;
-    public static ServerLoad getDefaultInstance() {
+    private static final ReplicationLoadSink defaultInstance;
+    public static ReplicationLoadSink getDefaultInstance() {
       return defaultInstance;
     }
 
-    public ServerLoad getDefaultInstanceForType() {
+    public ReplicationLoadSink getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -4624,7 +4399,7 @@ public final class ClusterStatusProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private ServerLoad(
+    private ReplicationLoadSink(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -4649,53 +4424,12 @@ public final class ClusterStatusProtos {
             }
             case 8: {
               bitField0_ |= 0x00000001;
-              numberOfRequests_ = input.readUInt32();
+              ageOfLastAppliedOp_ = input.readUInt64();
               break;
             }
             case 16: {
               bitField0_ |= 0x00000002;
-              totalNumberOfRequests_ = input.readUInt32();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              usedHeapMB_ = input.readUInt32();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              maxHeapMB_ = input.readUInt32();
-              break;
-            }
-            case 42: {
-              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-                regionLoads_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad>();
-                mutable_bitField0_ |= 0x00000010;
-              }
-              regionLoads_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad.PARSER, extensionRegistry));
-              break;
-            }
-            case 50: {
-              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-                coprocessors_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor>();
-                mutable_bitField0_ |= 0x00000020;
-              }
-              coprocessors_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor.PARSER, extensionRegistry));
-              break;
-            }
-            case 56: {
-              bitField0_ |= 0x00000010;
-              reportStartTime_ = input.readUInt64();
-              break;
-            }
-            case 64: {
-              bitField0_ |= 0x00000020;
-              reportEndTime_ = input.readUInt64();
-              break;
-            }
-            case 72: {
-              bitField0_ |= 0x00000040;
-              infoServerPort_ = input.readUInt32();
+              timeStampsOfLastAppliedOp_ = input.readUInt64();
               break;
             }
           }
@@ -4706,1957 +4440,4480 @@ public final class ClusterStatusProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-          regionLoads_ = java.util.Collections.unmodifiableList(regionLoads_);
-        }
-        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-          coprocessors_ = java.util.Collections.unmodifiableList(coprocessors_);
-        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<ServerLoad> PARSER =
-        new com.google.protobuf.AbstractParser<ServerLoad>() {
-      public ServerLoad parsePartialFrom(
+    public static com.google.protobuf.Parser<ReplicationLoadSink> PARSER =
+        new com.google.protobuf.AbstractParser<ReplicationLoadSink>() {
+      public ReplicationLoadSink parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new ServerLoad(input, extensionRegistry);
+        return new ReplicationLoadSink(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<ServerLoad> getParserForType() {
+    public com.google.protobuf.Parser<ReplicationLoadSink> getParserForType() {
       return PARSER;
     }
 
     private int bitField0_;
-    // optional uint32 number_of_requests = 1;
-    public static final int NUMBER_OF_REQUESTS_FIELD_NUMBER = 1;
-    private int numberOfRequests_;
+    // required uint64 ageOfLastAppliedOp = 1;
+    public static final int AGEOFLASTAPPLIEDOP_FIELD_NUMBER = 1;
+    private long ageOfLastAppliedOp_;
     /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    public boolean hasNumberOfRequests() {
+    public boolean hasAgeOfLastAppliedOp() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    public int getNumberOfRequests() {
-      return numberOfRequests_;
+    public long getAgeOfLastAppliedOp() {
+      return ageOfLastAppliedOp_;
     }
 
-    // optional uint32 total_number_of_requests = 2;
-    public static final int TOTAL_NUMBER_OF_REQUESTS_FIELD_NUMBER = 2;
-    private int totalNumberOfRequests_;
+    // required uint64 timeStampsOfLastAppliedOp = 2;
+    public static final int TIMESTAMPSOFLASTAPPLIEDOP_FIELD_NUMBER = 2;
+    private long timeStampsOfLastAppliedOp_;
     /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    public boolean hasTotalNumberOfRequests() {
+    public boolean hasTimeStampsOfLastAppliedOp() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    public int getTotalNumberOfRequests() {
-      return totalNumberOfRequests_;
+    public long getTimeStampsOfLastAppliedOp() {
+      return timeStampsOfLastAppliedOp_;
     }
 
-    // optional uint32 used_heap_MB = 3;
-    public static final int USED_HEAP_MB_FIELD_NUMBER = 3;
-    private int usedHeapMB_;
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    public boolean hasUsedHeapMB() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
+    private void initFields() {
+      ageOfLastAppliedOp_ = 0L;
+      timeStampsOfLastAppliedOp_ = 0L;
     }
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    public int getUsedHeapMB() {
-      return usedHeapMB_;
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasAgeOfLastAppliedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTimeStampsOfLastAppliedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
     }
 
-    // optional uint32 max_heap_MB = 4;
-    public static final int MAX_HEAP_MB_FIELD_NUMBER = 4;
-    private int maxHeapMB_;
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    public boolean hasMaxHeapMB() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt64(1, ageOfLastAppliedOp_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, timeStampsOfLastAppliedOp_);
+      }
+      getUnknownFields().writeTo(output);
     }
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    public int getMaxHeapMB() {
-      return maxHeapMB_;
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(1, ageOfLastAppliedOp_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, timeStampsOfLastAppliedOp_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
     }
 
-    // repeated .RegionLoad region_loads = 5;
-    public static final int REGION_LOADS_FIELD_NUMBER = 5;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad> regionLoads_;
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad> getRegionLoadsList() {
-      return regionLoads_;
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder> 
-        getRegionLoadsOrBuilderList() {
-      return regionLoads_;
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink other = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink) obj;
+
+      boolean result = true;
+      result = result && (hasAgeOfLastAppliedOp() == other.hasAgeOfLastAppliedOp());
+      if (hasAgeOfLastAppliedOp()) {
+        result = result && (getAgeOfLastAppliedOp()
+            == other.getAgeOfLastAppliedOp());
+      }
+      result = result && (hasTimeStampsOfLastAppliedOp() == other.hasTimeStampsOfLastAppliedOp());
+      if (hasTimeStampsOfLastAppliedOp()) {
+        result = result && (getTimeStampsOfLastAppliedOp()
+            == other.getTimeStampsOfLastAppliedOp());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public int getRegionLoadsCount() {
-      return regionLoads_.size();
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasAgeOfLastAppliedOp()) {
+        hash = (37 * hash) + AGEOFLASTAPPLIEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getAgeOfLastAppliedOp());
+      }
+      if (hasTimeStampsOfLastAppliedOp()) {
+        hash = (37 * hash) + TIMESTAMPSOFLASTAPPLIEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTimeStampsOfLastAppliedOp());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad getRegionLoads(int index) {
-      return regionLoads_.get(index);
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder getRegionLoadsOrBuilder(
-        int index) {
-      return regionLoads_.get(index);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
     }
-
-    // repeated .Coprocessor coprocessors = 6;
-    public static final int COPROCESSORS_FIELD_NUMBER = 6;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor> coprocessors_;
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor> getCoprocessorsList() {
-      return coprocessors_;
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder> 
-        getCoprocessorsOrBuilderList() {
-      return coprocessors_;
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public int getCoprocessorsCount() {
-      return coprocessors_.size();
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor getCoprocessors(int index) {
-      return coprocessors_.get(index);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder getCoprocessorsOrBuilder(
-        int index) {
-      return coprocessors_.get(index);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
     }
-
-    // optional uint64 report_start_time = 7;
-    public static final int REPORT_START_TIME_FIELD_NUMBER = 7;
-    private long reportStartTime_;
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public boolean hasReportStartTime() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public long getReportStartTime() {
-      return reportStartTime_;
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
     }
-
-    // optional uint64 report_end_time = 8;
-    public static final int REPORT_END_TIME_FIELD_NUMBER = 8;
-    private long reportEndTime_;
-    /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public boolean hasReportEndTime() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
     }
-    /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public long getReportEndTime() {
-      return reportEndTime_;
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink prototype) {
+      return newBuilder().mergeFrom(prototype);
     }
+    public Builder toBuilder() { return newBuilder(this); }
 
-    // optional uint32 info_server_port = 9;
-    public static final int INFO_SERVER_PORT_FIELD_NUMBER = 9;
-    private int infoServerPort_;
-    /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
-     */
-    public boolean hasInfoServerPort() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
     }
     /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
+     * Protobuf type {@code ReplicationLoadSink}
      */
-    public int getInfoServerPort() {
-      return infoServerPort_;
-    }
-
-    private void initFields() {
-      numberOfRequests_ = 0;
-      totalNumberOfRequests_ = 0;
-      usedHeapMB_ = 0;
-      maxHeapMB_ = 0;
-      regionLoads_ = java.util.Collections.emptyList();
-      coprocessors_ = java.util.Collections.emptyList();
-      reportStartTime_ = 0L;
-      reportEndTime_ = 0L;
-      infoServerPort_ = 0;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      for (int i = 0; i < getRegionLoadsCount(); i++) {
-        if (!getRegionLoads(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSinkOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_descriptor;
       }
-      for (int i = 0; i < getCoprocessorsCount(); i++) {
-        if (!getCoprocessors(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.Builder.class);
       }
-      memoizedIsInitialized = 1;
-      return true;
-    }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeUInt32(1, numberOfRequests_);
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeUInt32(2, totalNumberOfRequests_);
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeUInt32(3, usedHeapMB_);
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
       }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeUInt32(4, maxHeapMB_);
+      private static Builder create() {
+        return new Builder();
       }
-      for (int i = 0; i < regionLoads_.size(); i++) {
-        output.writeMessage(5, regionLoads_.get(i));
+
+      public Builder clear() {
+        super.clear();
+        ageOfLastAppliedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        timeStampsOfLastAppliedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
       }
-      for (int i = 0; i < coprocessors_.size(); i++) {
-        output.writeMessage(6, coprocessors_.get(i));
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
       }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeUInt64(7, reportStartTime_);
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_descriptor;
       }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeUInt64(8, reportEndTime_);
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.getDefaultInstance();
       }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeUInt32(9, infoServerPort_);
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
 
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(1, numberOfRequests_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(2, totalNumberOfRequests_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(3, usedHeapMB_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(4, maxHeapMB_);
-      }
-      for (int i = 0; i < regionLoads_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(5, regionLoads_.get(i));
-      }
-      for (int i = 0; i < coprocessors_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(6, coprocessors_.get(i));
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(7, reportStartTime_);
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(8, reportEndTime_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(9, infoServerPort_);
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink build() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
       }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
 
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad)) {
-        return super.equals(obj);
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink result = new org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.ageOfLastAppliedOp_ = ageOfLastAppliedOp_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.timeStampsOfLastAppliedOp_ = timeStampsOfLastAppliedOp_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
       }
-      org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad other = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad) obj;
 
-      boolean result = true;
-      result = result && (hasNumberOfRequests() == other.hasNumberOfRequests());
-      if (hasNumberOfRequests()) {
-        result = result && (getNumberOfRequests()
-            == other.getNumberOfRequests());
-      }
-      result = result && (hasTotalNumberOfRequests() == other.hasTotalNumberOfRequests());
-      if (hasTotalNumberOfRequests()) {
-        result = result && (getTotalNumberOfRequests()
-            == other.getTotalNumberOfRequests());
-      }
-      result = result && (hasUsedHeapMB() == other.hasUsedHeapMB());
-      if (hasUsedHeapMB()) {
-        result = result && (getUsedHeapMB()
-            == other.getUsedHeapMB());
-      }
-      result = result && (hasMaxHeapMB() == other.hasMaxHeapMB());
-      if (hasMaxHeapMB()) {
-        result = result && (getMaxHeapMB()
-            == other.getMaxHeapMB());
-      }
-      result = result && getRegionLoadsList()
-          .equals(other.getRegionLoadsList());
-      result = result && getCoprocessorsList()
-          .equals(other.getCoprocessorsList());
-      result = result && (hasReportStartTime() == other.hasReportStartTime());
-      if (hasReportStartTime()) {
-        result = result && (getReportStartTime()
-            == other.getReportStartTime());
-      }
-      result = result && (hasReportEndTime() == other.hasReportEndTime());
-      if (hasReportEndTime()) {
-        result = result && (getReportEndTime()
-            == other.getReportEndTime());
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
       }
-      result = result && (hasInfoServerPort() == other.hasInfoServerPort());
-      if (hasInfoServerPort()) {
-        result = result && (getInfoServerPort()
-            == other.getInfoServerPort());
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.getDefaultInstance()) return this;
+        if (other.hasAgeOfLastAppliedOp()) {
+          setAgeOfLastAppliedOp(other.getAgeOfLastAppliedOp());
+        }
+        if (other.hasTimeStampsOfLastAppliedOp()) {
+          setTimeStampsOfLastAppliedOp(other.getTimeStampsOfLastAppliedOp());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
       }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
 
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
+      public final boolean isInitialized() {
+        if (!hasAgeOfLastAppliedOp()) {
+          
+          return false;
+        }
+        if (!hasTimeStampsOfLastAppliedOp()) {
+          
+          return false;
+        }
+        return true;
       }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasNumberOfRequests()) {
-        hash = (37 * hash) + NUMBER_OF_REQUESTS_FIELD_NUMBER;
-        hash = (53 * hash) + getNumberOfRequests();
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
       }
-      if (hasTotalNumberOfRequests()) {
-        hash = (37 * hash) + TOTAL_NUMBER_OF_REQUESTS_FIELD_NUMBER;
-        hash = (53 * hash) + getTotalNumberOfRequests();
+      private int bitField0_;
+
+      // required uint64 ageOfLastAppliedOp = 1;
+      private long ageOfLastAppliedOp_ ;
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public boolean hasAgeOfLastAppliedOp() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
-      if (hasUsedHeapMB()) {
-        hash = (37 * hash) + USED_HEAP_MB_FIELD_NUMBER;
-        hash = (53 * hash) + getUsedHeapMB();
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public long getAgeOfLastAppliedOp() {
+        return ageOfLastAppliedOp_;
       }
-      if (hasMaxHeapMB()) {
-        hash = (37 * hash) + MAX_HEAP_MB_FIELD_NUMBER;
-        hash = (53 * hash) + getMaxHeapMB();
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public Builder setAgeOfLastAppliedOp(long value) {
+        bitField0_ |= 0x00000001;
+        ageOfLastAppliedOp_ = value;
+        onChanged();
+        return this;
       }
-      if (getRegionLoadsCount() > 0) {
-        hash = (37 * hash) + REGION_LOADS_FIELD_NUMBER;
-        hash = (53 * hash) + getRegionLoadsList().hashCode();
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public Builder clearAgeOfLastAppliedOp() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        ageOfLastAppliedOp_ = 0L;
+        onChanged();
+        return this;
       }
-      if (getCoprocessorsCount() > 0) {
-        hash = (37 * hash) + COPROCESSORS_FIELD_NUMBER;
-        hash = (53 * hash) + getCoprocessorsList().hashCode();
+
+      // required uint64 timeStampsOfLastAppliedOp = 2;
+      private long timeStampsOfLastAppliedOp_ ;
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public boolean hasTimeStampsOfLastAppliedOp() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
       }
-      if (hasReportStartTime()) {
-        hash = (37 * hash) + REPORT_START_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getReportStartTime());
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public long getTimeStampsOfLastAppliedOp() {
+        return timeStampsOfLastAppliedOp_;
       }
-      if (hasReportEndTime()) {
-        hash = (37 * hash) + REPORT_END_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getReportEndTime());
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public Builder setTimeStampsOfLastAppliedOp(long value) {
+        bitField0_ |= 0x00000002;
+        timeStampsOfLastAppliedOp_ = value;
+        onChanged();
+        return this;
       }
-      if (hasInfoServerPort()) {
-        hash = (37 * hash) + INFO_SERVER_PORT_FIELD_NUMBER;
-        hash = (53 * hash) + getInfoServerPort();
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public Builder clearTimeStampsOfLastAppliedOp() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        timeStampsOfLastAppliedOp_ = 0L;
+        onChanged();
+        return this;
       }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
+      // @@protoc_insertion_point(builder_scope:ReplicationLoadSink)
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
+
+    static {
+      defaultInstance = new ReplicationLoadSink(true);
+      defaultInstance.initFields();
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
+
+    // @@protoc_insertion_point(class_scope:ReplicationLoadSink)
+  }
+
+  public interface ReplicationLoadSourceOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string peerID = 1;
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    boolean hasPeerID();
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    java.lang.String getPeerID();
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getPeerIDBytes();
+
+    // required uint64 ageOfLastShippedOp = 2;
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    boolean hasAgeOfLastShippedOp();
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    long getAgeOfLastShippedOp();
+
+    // required uint32 sizeOfLogQueue = 3;
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    boolean hasSizeOfLogQueue();
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    int getSizeOfLogQueue();
+
+    // required uint64 timeStampOfLastShippedOp = 4;
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    boolean hasTimeStampOfLastShippedOp();
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    long getTimeStampOfLastShippedOp();
+
+    // required uint64 replicationLag = 5;
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    boolean hasReplicationLag();
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    long getReplicationLag();
+  }
+  /**
+   * Protobuf type {@code ReplicationLoadSource}
+   */
+  public static final class ReplicationLoadSource extends
+      com.google.protobuf.GeneratedMessage
+      implements ReplicationLoadSourceOrBuilder {
+    // Use ReplicationLoadSource.newBuilder() to construct.
+    private ReplicationLoadSource(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
+    private ReplicationLoadSource(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ReplicationLoadSource defaultInstance;
+    public static ReplicationLoadSource getDefaultInstance() {
+      return defaultInstance;
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+
+    public ReplicationLoadSource getDefaultInstanceForType() {
+      return defaultInstance;
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
+    private ReplicationLoadSource(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              peerID_ = input.readBytes();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              ageOfLastShippedOp_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              sizeOfLogQueue_ = input.readUInt32();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              timeStampOfLastShippedOp_ = input.readUInt64();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              replicationLag_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_descriptor;
     }
 
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad prototype) {
-      return newBuilder().mergeFrom(prototype);
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.Builder.class);
     }
-    public Builder toBuilder() { return newBuilder(this); }
+
+    public static com.google.protobuf.Parser<ReplicationLoadSource> PARSER =
+        new com.google.protobuf.AbstractParser<ReplicationLoadSource>() {
+      public ReplicationLoadSource parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ReplicationLoadSource(input, extensionRegistry);
+      }
+    };
 
     @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
+    public com.google.protobuf.Parser<ReplicationLoadSource> getParserForType() {
+      return PARSER;
     }
+
+    private int bitField0_;
+    // required string peerID = 1;
+    public static final int PEERID_FIELD_NUMBER = 1;
+    private java.lang.Object peerID_;
     /**
-     * Protobuf type {@code ServerLoad}
+     * <code>required string peerID = 1;</code>
      */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoadOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_descriptor;
+    public boolean hasPeerID() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    public java.lang.String getPeerID() {
+      java.lang.Object ref = peerID_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          peerID_ = s;
+        }
+        return s;
       }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder.class);
+    }
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getPeerIDBytes() {
+      java.lang.Object ref = peerID_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        peerID_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
       }
+    }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
+    // required uint64 ageOfLastShippedOp = 2;
+    public static final int AGEOFLASTSHIPPEDOP_FIELD_NUMBER = 2;
+    private long ageOfLastShippedOp_;
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    public boolean hasAgeOfLastShippedOp() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    public long getAgeOfLastShippedOp() {
+      return ageOfLastShippedOp_;
+    }
 
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getRegionLoadsFieldBuilder();
-          getCoprocessorsFieldBuilder();
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
+    // required uint32 sizeOfLogQueue = 3;
+    public static final int SIZEOFLOGQUEUE_FIELD_NUMBER = 3;
+    private int sizeOfLogQueue_;
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    public boolean hasSizeOfLogQueue() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    public int getSizeOfLogQueue() {
+      return sizeOfLogQueue_;
+    }
 
-      public Builder clear() {
-        super.clear();
-        numberOfRequests_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        totalNumberOfRequests_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000002);
-        usedHeapMB_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        maxHeapMB_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        if (regionLoadsBuilder_ == null) {
-          regionLoads_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000010);
-        } else {
-          regionLoadsBuilder_.clear();
-        }
-        if (coprocessorsBuilder_ == null) {
-          coprocessors_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000020);
-        } else {
-          coprocessorsBuilder_.clear();
-        }
-        reportStartTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000040);
-        reportEndTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000080);
-        infoServerPort_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000100);
-        return this;
-      }
+    // required uint64 timeStampOfLastShippedOp = 4;
+    public static final int TIMESTAMPOFLASTSHIPPEDOP_FIELD_NUMBER = 4;
+    private long timeStampOfLastShippedOp_;
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    public boolean hasTimeStampOfLastShippedOp() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    public long getTimeStampOfLastShippedOp() {
+      return timeStampOfLastShippedOp_;
+    }
 
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
+    // required uint64 replicationLag = 5;
+    public static final int REPLICATIONLAG_FIELD_NUMBER = 5;
+    private long replicationLag_;
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    public boolean hasReplicationLag() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    public long getReplicationLag() {
+      return replicationLag_;
+    }
 
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_descriptor;
-      }
+    private void initFields() {
+      peerID_ = "";
+      ageOfLastShippedOp_ = 0L;
+      sizeOfLogQueue_ = 0;
+      timeStampOfLastShippedOp_ = 0L;
+      replicationLag_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
 
-      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.getDefaultInstance();
+      if (!hasPeerID()) {
+        memoizedIsInitialized = 0;
+        return false;
       }
-
-      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad build() {
-        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad result = buildPartial();
+      if (!hasAgeOfLastShippedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasSizeOfLogQueue()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTimeStampOfLastShippedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasReplicationLag()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getPeerIDBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, ageOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt32(3, sizeOfLogQueue_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeUInt64(4, timeStampOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeUInt64(5, replicationLag_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getPeerIDBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, ageOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(3, sizeOfLogQueue_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(4, timeStampOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(5, replicationLag_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource other = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource) obj;
+
+      boolean result = true;
+      result = result && (hasPeerID() == other.hasPeerID());
+      if (hasPeerID()) {
+        result = result && getPeerID()
+            .equals(other.getPeerID());
+      }
+      result = result && (hasAgeOfLastShippedOp() == other.hasAgeOfLastShippedOp());
+      if (hasAgeOfLastShippedOp()) {
+        result = result && (getAgeOfLastShippedOp()
+            == other.getAgeOfLastShippedOp());
+      }
+      result = result && (hasSizeOfLogQueue() == other.hasSizeOfLogQueue());
+      if (hasSizeOfLogQueue()) {
+        result = result && (getSizeOfLogQueue()
+            == other.getSizeOfLogQueue());
+      }
+      result = result && (hasTimeStampOfLastShippedOp() == other.hasTimeStampOfLastShippedOp());
+      if (hasTimeStampOfLastShippedOp()) {
+        result = result && (getTimeStampOfLastShippedOp()
+            == other.getTimeStampOfLastShippedOp());
+      }
+      result = result && (hasReplicationLag() == other.hasReplicationLag());
+      if (hasReplicationLag()) {
+        result = result && (getReplicationLag()
+            == other.getReplicationLag());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasPeerID()) {
+        hash = (37 * hash) + PEERID_FIELD_NUMBER;
+        hash = (53 * hash) + getPeerID().hashCode();
+      }
+      if (hasAgeOfLastShippedOp()) {
+        hash = (37 * hash) + AGEOFLASTSHIPPEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getAgeOfLastShippedOp());
+      }
+      if (hasSizeOfLogQueue()) {
+        hash = (37 * hash) + SIZEOFLOGQUEUE_FIELD_NUMBER;
+        hash = (53 * hash) + getSizeOfLogQueue();
+      }
+      if (hasTimeStampOfLastShippedOp()) {
+        hash = (37 * hash) + TIMESTAMPOFLASTSHIPPEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTimeStampOfLastShippedOp());
+      }
+      if (hasReplicationLag()) {
+        hash = (37 * hash) + REPLICATIONLAG_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getReplicationLag());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code ReplicationLoadSource}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSourceOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        peerID_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        ageOfLastShippedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        sizeOfLogQueue_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        timeStampOfLastShippedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        replicationLag_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource build() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad result = new org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad(this);
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource result = new org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.numberOfRequests_ = numberOfRequests_;
+        result.peerID_ = peerID_;
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.totalNumberOfRequests_ = totalNumberOfRequests_;
+        result.ageOfLastShippedOp_ = ageOfLastShippedOp_;
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000004;
         }
-        result.usedHeapMB_ = usedHeapMB_;
+        result.sizeOfLogQueue_ = sizeOfLogQueue_;
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000008;
         }
-        result.maxHeapMB_ = maxHeapMB_;
+        result.timeStampOfLastShippedOp_ = timeStampOfLastShippedOp_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.replicationLag_ = replicationLag_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.getDefaultInstance()) return this;
+        if (other.hasPeerID()) {
+          bitField0_ |= 0x00000001;
+          peerID_ = other.peerID_;
+          onChanged();
+        }
+        if (other.hasAgeOfLastShippedOp()) {
+          setAgeOfLastShippedOp(other.getAgeOfLastShippedOp());
+        }
+        if (other.hasSizeOfLogQueue()) {
+          setSizeOfLogQueue(other.getSizeOfLogQueue());
+        }
+        if (other.hasTimeStampOfLastShippedOp()) {
+          setTimeStampOfLastShippedOp(other.getTimeStampOfLastShippedOp());
+        }
+        if (other.hasReplicationLag()) {
+          setReplicationLag(other.getReplicationLag());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasPeerID()) {
+          
+          return false;
+        }
+        if (!hasAgeOfLastShippedOp()) {
+          
+          return false;
+        }
+        if (!hasSizeOfLogQueue()) {
+          
+          return false;
+        }
+        if (!hasTimeStampOfLastShippedOp()) {
+          
+          return false;
+        }
+        if (!hasReplicationLag()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string peerID = 1;
+      private java.lang.Object peerID_ = "";
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public boolean hasPeerID() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public java.lang.String getPeerID() {
+        java.lang.Object ref = peerID_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          peerID_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getPeerIDBytes() {
+        java.lang.Object ref = peerID_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          peerID_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public Builder setPeerID(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        peerID_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public Builder clearPeerID() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        peerID_ = getDefaultInstance().getPeerID();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public Builder setPeerIDBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        peerID_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 ageOfLastShippedOp = 2;
+      private long ageOfLastShippedOp_ ;
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public boolean hasAgeOfLastShippedOp() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public long getAgeOfLastShippedOp() {
+        return ageOfLastShippedOp_;
+      }
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public Builder setAgeOfLastShippedOp(long value) {
+        bitField0_ |= 0x00000002;
+        ageOfLastShippedOp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public Builder clearAgeOfLastShippedOp() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        ageOfLastShippedOp_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required uint32 sizeOfLogQueue = 3;
+      private int sizeOfLogQueue_ ;
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public boolean hasSizeOfLogQueue() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public int getSizeOfLogQueue() {
+        return sizeOfLogQueue_;
+      }
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public Builder setSizeOfLogQueue(int value) {
+        bitField0_ |= 0x00000004;
+        sizeOfLogQueue_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public Builder clearSizeOfLogQueue() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        sizeOfLogQueue_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 timeStampOfLastShippedOp = 4;
+      private long timeStampOfLastShippedOp_ ;
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public boolean hasTimeStampOfLastShippedOp() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public long getTimeStampOfLastShippedOp() {
+        return timeStampOfLastShippedOp_;
+      }
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public Builder setTimeStampOfLastShippedOp(long value) {
+        bitField0_ |= 0x00000008;
+        timeStampOfLastShippedOp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public Builder clearTimeStampOfLastShippedOp() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        timeStampOfLastShippedOp_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 replicationLag = 5;
+      private long replicationLag_ ;
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public boolean hasReplicationLag() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public long getReplicationLag() {
+        return replicationLag_;
+      }
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public Builder setReplicationLag(long value) {
+        bitField0_ |= 0x00000010;
+        replicationLag_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public Builder clearReplicationLag() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        replicationLag_ = 0

<TRUNCATED>

[3/9] hbase git commit: HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Posted by ap...@apache.org.
HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Signed-off-by: Andrew Purtell <ap...@apache.org>

Conflicts:
	hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
	hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c0e26c3a
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c0e26c3a
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c0e26c3a

Branch: refs/heads/branch-1.0
Commit: c0e26c3a0704deb69473bf3daec851afd17688a3
Parents: bdb938b
Author: Ashish Singhi <as...@huawei.com>
Authored: Thu Feb 12 14:00:19 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Feb 12 14:16:22 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ServerLoad.java     |   27 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   24 +
 .../hbase/replication/ReplicationLoadSink.java  |   36 +
 .../replication/ReplicationLoadSource.java      |   53 +
 .../MetricsReplicationSinkSource.java           |    1 +
 .../MetricsReplicationSourceSource.java         |    1 +
 .../MetricsReplicationGlobalSourceSource.java   |    5 +
 .../MetricsReplicationSinkSourceImpl.java       |    5 +
 .../MetricsReplicationSourceSourceImpl.java     |    5 +
 .../protobuf/generated/ClusterStatusProtos.java | 5535 +++++++++++++-----
 .../src/main/protobuf/ClusterStatus.proto       |   23 +
 .../hbase/regionserver/HRegionServer.java       |   17 +
 .../hbase/regionserver/ReplicationService.java  |    8 +-
 .../replication/regionserver/MetricsSink.java   |   17 +
 .../replication/regionserver/MetricsSource.java |   34 +
 .../replication/regionserver/Replication.java   |   32 +-
 .../regionserver/ReplicationLoad.java           |  151 +
 .../regionserver/ReplicationSink.java           |    8 +
 .../regionserver/ReplicationSource.java         |    8 +
 .../replication/TestReplicationSmallTests.java  |   49 +-
 hbase-shell/src/main/ruby/hbase/admin.rb        |   42 +-
 .../src/main/ruby/shell/commands/status.rb      |    9 +-
 hbase-shell/src/test/ruby/hbase/admin_test.rb   |   12 +
 hbase-shell/src/test/ruby/test_helper.rb        |    4 +
 24 files changed, 4472 insertions(+), 1634 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
index 06a61c0..370862a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
@@ -22,8 +22,11 @@ package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Strings;
 
@@ -52,7 +55,7 @@ public class ServerLoad {
   private int totalStaticBloomSizeKB = 0;
   private long totalCompactingKVs = 0;
   private long currentCompactedKVs = 0;
-  
+
   public ServerLoad(ClusterStatusProtos.ServerLoad serverLoad) {
     this.serverLoad = serverLoad;
     for (ClusterStatusProtos.RegionLoad rl: serverLoad.getRegionLoadsList()) {
@@ -70,7 +73,7 @@ public class ServerLoad {
       totalCompactingKVs += rl.getTotalCompactingKVs();
       currentCompactedKVs += rl.getCurrentCompactedKVs();
     }
-    
+
   }
 
   // NOTE: Function name cannot start with "get" because then an OpenDataException is thrown because
@@ -178,6 +181,26 @@ public class ServerLoad {
   }
 
   /**
+   * Call directly from client such as hbase shell
+   * @return the list of ReplicationLoadSource
+   */
+  public List<ReplicationLoadSource> getReplicationLoadSourceList() {
+    return ProtobufUtil.toReplicationLoadSourceList(serverLoad.getReplLoadSourceList());
+  }
+
+  /**
+   * Call directly from client such as hbase shell
+   * @return ReplicationLoadSink
+   */
+  public ReplicationLoadSink getReplicationLoadSink() {
+    if (serverLoad.hasReplLoadSink()) {
+      return ProtobufUtil.toReplicationLoadSink(serverLoad.getReplLoadSink());
+    } else {
+      return null;
+    }
+  }
+
+  /**
    * Originally, this method factored in the effect of requests going to the
    * server as well. However, this does not interact very well with the current
    * region rebalancing code, which only factors number of regions. For the

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 9893825..56d9a64 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -102,6 +102,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Col
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
+import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
 import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -123,6 +124,8 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.TablePermission;
 import org.apache.hadoop.hbase.security.access.UserPermission;
@@ -2828,4 +2831,25 @@ public final class ProtobufUtil {
     }
     return result;
   }
+
+  public static ReplicationLoadSink toReplicationLoadSink(
+      ClusterStatusProtos.ReplicationLoadSink cls) {
+    return new ReplicationLoadSink(cls.getAgeOfLastAppliedOp(), cls.getTimeStampsOfLastAppliedOp());
+  }
+
+  public static ReplicationLoadSource toReplicationLoadSource(
+      ClusterStatusProtos.ReplicationLoadSource cls) {
+    return new ReplicationLoadSource(cls.getPeerID(), cls.getAgeOfLastShippedOp(),
+        cls.getSizeOfLogQueue(), cls.getTimeStampOfLastShippedOp(), cls.getReplicationLag());
+  }
+
+  public static List<ReplicationLoadSource> toReplicationLoadSourceList(
+      List<ClusterStatusProtos.ReplicationLoadSource> clsList) {
+    ArrayList<ReplicationLoadSource> rlsList = new ArrayList<ReplicationLoadSource>();
+    for (ClusterStatusProtos.ReplicationLoadSource cls : clsList) {
+      rlsList.add(toReplicationLoadSource(cls));
+    }
+    return rlsList;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
new file mode 100644
index 0000000..63fe334
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF 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.apache.hadoop.hbase.replication;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A HBase ReplicationLoad to present MetricsSink information
+ */
+@InterfaceAudience.Private
+public class ReplicationLoadSink {
+  private long ageOfLastAppliedOp;
+  private long timeStampsOfLastAppliedOp;
+
+  public ReplicationLoadSink(long age, long timeStamp) {
+    this.ageOfLastAppliedOp = age;
+    this.timeStampsOfLastAppliedOp = timeStamp;
+  }
+
+  public long getAgeOfLastAppliedOp() {
+    return this.ageOfLastAppliedOp;
+  }
+
+  public long getTimeStampsOfLastAppliedOp() {
+    return this.timeStampsOfLastAppliedOp;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
new file mode 100644
index 0000000..bfd1599
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF 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.apache.hadoop.hbase.replication;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A HBase ReplicationLoad to present MetricsSource information
+ */
+@InterfaceAudience.Private
+public class ReplicationLoadSource {
+  private String peerID;
+  private long ageOfLastShippedOp;
+  private int sizeOfLogQueue;
+  private long timeStampOfLastShippedOp;
+  private long replicationLag;
+
+  public ReplicationLoadSource(String id, long age, int size, long timeStamp, long lag) {
+    this.peerID = id;
+    this.ageOfLastShippedOp = age;
+    this.sizeOfLogQueue = size;
+    this.timeStampOfLastShippedOp = timeStamp;
+    this.replicationLag = lag;
+  }
+
+  public String getPeerID() {
+    return this.peerID;
+  }
+
+  public long getAgeOfLastShippedOp() {
+    return this.ageOfLastShippedOp;
+  }
+
+  public long getSizeOfLogQueue() {
+    return this.sizeOfLogQueue;
+  }
+
+  public long getTimeStampOfLastShippedOp() {
+    return this.timeStampOfLastShippedOp;
+  }
+
+  public long getReplicationLag() {
+    return this.replicationLag;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
index 805dfca..698a59a 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
@@ -26,4 +26,5 @@ public interface MetricsReplicationSinkSource {
   void setLastAppliedOpAge(long age);
   void incrAppliedBatches(long batches);
   void incrAppliedOps(long batchsize);
+  long getLastAppliedOpAge();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
index 66d265a..fecf191 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
@@ -43,4 +43,5 @@ public interface MetricsReplicationSourceSource {
   void incrLogReadInBytes(long size);
   void incrLogReadInEdits(long size);
   void clear();
+  long getLastShippedAge();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
index a210171..6dace10 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
@@ -95,4 +95,9 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
 
   @Override public void clear() {
   }
+
+  @Override
+  public long getLastShippedAge() {
+    return ageOfLastShippedOpGauge.value();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
index 3025e3e..14212ba 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
@@ -44,4 +44,9 @@ public class MetricsReplicationSinkSourceImpl implements MetricsReplicationSinkS
   @Override public void incrAppliedOps(long batchsize) {
     opsCounter.incr(batchsize);
   }
+
+  @Override
+  public long getLastAppliedOpAge() {
+    return ageGauge.value();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
index 89ef4de..1422e7e 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
@@ -125,4 +125,9 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
 
     rms.removeMetric(logEditsFilteredKey);
   }
+
+  @Override
+  public long getLastShippedAge() {
+    return ageOfLastShippedOpGauge.value();
+  }
 }


[4/9] hbase git commit: HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-protocol/src/main/protobuf/ClusterStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/ClusterStatus.proto b/hbase-protocol/src/main/protobuf/ClusterStatus.proto
index 2b2d9eb..bb531cc 100644
--- a/hbase-protocol/src/main/protobuf/ClusterStatus.proto
+++ b/hbase-protocol/src/main/protobuf/ClusterStatus.proto
@@ -119,6 +119,19 @@ message RegionLoad {
 
 /* Server-level protobufs */
 
+message ReplicationLoadSink {
+  required uint64 ageOfLastAppliedOp = 1;
+  required uint64 timeStampsOfLastAppliedOp = 2;
+}
+
+message ReplicationLoadSource {
+  required string peerID = 1;
+  required uint64 ageOfLastShippedOp = 2;
+  required uint32 sizeOfLogQueue = 3;
+  required uint64 timeStampOfLastShippedOp = 4;
+  required uint64 replicationLag = 5;
+}
+
 message ServerLoad {
   /** Number of requests since last report. */
   optional uint32 number_of_requests = 1;
@@ -160,6 +173,16 @@ message ServerLoad {
    * The port number that this region server is hosing an info server on.
    */
   optional uint32 info_server_port = 9;
+
+  /**
+   * The replicationLoadSource for the replication Source status of this region server.
+   */
+  repeated ReplicationLoadSource replLoadSource = 10;
+
+  /**
+   * The replicationLoadSink for the replication Sink status of this region server.
+   */
+  optional ReplicationLoadSink replLoadSink = 11;
 }
 
 message LiveServerInfo {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index f215d7e..ed5ff41 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -131,6 +131,7 @@ import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.util.Addressing;
@@ -1130,6 +1131,22 @@ public class HRegionServer extends HasThread implements
     } else {
       serverLoad.setInfoServerPort(-1);
     }
+
+    // for the replicationLoad purpose. Only need to get from one service
+    // either source or sink will get the same info
+    ReplicationSourceService rsources = getReplicationSourceService();
+
+    if (rsources != null) {
+      // always refresh first to get the latest value
+      ReplicationLoad rLoad = rsources.refreshAndGetReplicationLoad();
+      if (rLoad != null) {
+        serverLoad.setReplLoadSink(rLoad.getReplicationLoadSink());
+        for (ClusterStatusProtos.ReplicationLoadSource rLS : rLoad.getReplicationLoadSourceList()) {
+          serverLoad.addReplLoadSource(rLS);
+        }
+      }
+    }
+
     return serverLoad.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
index 92ac823..25a27a9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
@@ -22,11 +22,12 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 /**
- * Gateway to Cluster Replication.  
+ * Gateway to Cluster Replication.
  * Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
  * One such application is a cross-datacenter
  * replication service that can keep two hbase clusters in sync.
@@ -52,4 +53,9 @@ public interface ReplicationService {
    * Stops replication service.
    */
   void stopReplicationService();
+
+  /**
+   * Refresh and Get ReplicationLoad
+   */
+  public ReplicationLoad refreshAndGetReplicationLoad();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
index 0c9d016..37dc1dd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
@@ -71,4 +71,21 @@ public class MetricsSink {
     mss.incrAppliedOps(batchSize);
   }
 
+  /**
+   * Get the Age of Last Applied Op
+   * @return ageOfLastAppliedOp
+   */
+  public long getAgeOfLastAppliedOp() {
+    return mss.getLastAppliedOpAge();
+  }
+
+  /**
+   * Get the TimeStampOfLastAppliedOp. If no replication Op applied yet, the value is the timestamp
+   * at which hbase instance starts
+   * @return timeStampsOfLastAppliedOp;
+   */
+  public long getTimeStampOfLastAppliedOp() {
+    return this.lastTimestampForAge;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
index a734b9c..21296a0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
@@ -36,6 +36,7 @@ public class MetricsSource {
 
   private long lastTimestamp = 0;
   private int lastQueueSize = 0;
+  private String id;
 
   private final MetricsReplicationSourceSource singleSourceSource;
   private final MetricsReplicationSourceSource globalSourceSource;
@@ -46,6 +47,7 @@ public class MetricsSource {
    * @param id Name of the source this class is monitoring
    */
   public MetricsSource(String id) {
+    this.id = id;
     singleSourceSource =
         CompatibilitySingletonFactory.getInstance(MetricsReplicationSourceFactory.class)
             .getSource(id);
@@ -143,4 +145,36 @@ public class MetricsSource {
     globalSourceSource.decrSizeOfLogQueue(lastQueueSize);
     lastQueueSize = 0;
   }
+
+  /**
+   * Get AgeOfLastShippedOp
+   * @return AgeOfLastShippedOp
+   */
+  public Long getAgeOfLastShippedOp() {
+    return singleSourceSource.getLastShippedAge();
+  }
+
+  /**
+   * Get the sizeOfLogQueue
+   * @return sizeOfLogQueue
+   */
+  public int getSizeOfLogQueue() {
+    return this.lastQueueSize;
+  }
+
+  /**
+   * Get the timeStampsOfLastShippedOp
+   * @return lastTimestampForAge
+   */
+  public long getTimeStampOfLastShippedOp() {
+    return lastTimestamp;
+  }
+
+  /**
+   * Get the slave peer ID
+   * @return peerID
+   */
+  public String getPeerID() {
+    return id;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 4729644..78bb92e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.hbase.HConstants.REPLICATION_ENABLE_KEY;
 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
@@ -66,7 +67,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * Gateway to Replication.  Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
  */
 @InterfaceAudience.Private
-public class Replication extends WALActionsListener.Base implements 
+public class Replication extends WALActionsListener.Base implements
   ReplicationSourceService, ReplicationSinkService {
   private static final Log LOG =
       LogFactory.getLog(Replication.class);
@@ -82,6 +83,8 @@ public class Replication extends WALActionsListener.Base implements
   /** Statistics thread schedule pool */
   private ScheduledExecutorService scheduleThreadPool;
   private int statsThreadPeriod;
+  // ReplicationLoad to access replication metrics
+  private ReplicationLoad replicationLoad;
 
   /**
    * Instantiate the replication management (if rep is enabled).
@@ -138,11 +141,13 @@ public class Replication extends WALActionsListener.Base implements
       this.statsThreadPeriod =
           this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
       LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
+      this.replicationLoad = new ReplicationLoad();
     } else {
       this.replicationManager = null;
       this.replicationQueues = null;
       this.replicationPeers = null;
       this.replicationTracker = null;
+      this.replicationLoad = null;
     }
   }
 
@@ -310,4 +315,29 @@ public class Replication extends WALActionsListener.Base implements
       }
     }
   }
+
+  @Override
+  public ReplicationLoad refreshAndGetReplicationLoad() {
+    if (this.replicationLoad == null) {
+      return null;
+    }
+    // always build for latest data
+    buildReplicationLoad();
+    return this.replicationLoad;
+  }
+
+  private void buildReplicationLoad() {
+    // get source
+    List<ReplicationSourceInterface> sources = this.replicationManager.getSources();
+    List<MetricsSource> sourceMetricsList = new ArrayList<MetricsSource>();
+
+    for (ReplicationSourceInterface source : sources) {
+      if (source instanceof ReplicationSource) {
+        sourceMetricsList.add(((ReplicationSource) source).getSourceMetrics());
+      }
+    }
+    // get sink
+    MetricsSink sinkMetrics = this.replicationSink.getSinkMetrics();
+    this.replicationLoad.buildReplicationLoad(sourceMetricsList, sinkMetrics);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
new file mode 100644
index 0000000..b3f3ecb
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationLoad.java
@@ -0,0 +1,151 @@
+/**
+ * Copyright 2014 The Apache Software Foundation Licensed to the Apache Software Foundation (ASF)
+ * under one or more contributor license agreements. See the NOTICE file distributed with this work
+ * for additional information regarding copyright ownership. The ASF 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.apache.hadoop.hbase.replication.regionserver;
+
+import java.util.Date;
+import java.util.List;
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Strings;
+
+/**
+ * This class is used for exporting some of the info from replication metrics
+ */
+@InterfaceAudience.Private
+public class ReplicationLoad {
+
+  // Empty load instance.
+  public static final ReplicationLoad EMPTY_REPLICATIONLOAD = new ReplicationLoad();
+
+  private List<MetricsSource> sourceMetricsList;
+  private MetricsSink sinkMetrics;
+
+  private List<ClusterStatusProtos.ReplicationLoadSource> replicationLoadSourceList;
+  private ClusterStatusProtos.ReplicationLoadSink replicationLoadSink;
+
+  /** default constructor */
+  public ReplicationLoad() {
+    super();
+  }
+
+  /**
+   * buildReplicationLoad
+   * @param srMetricsList
+   * @param skMetrics
+   */
+
+  public void buildReplicationLoad(final List<MetricsSource> srMetricsList,
+      final MetricsSink skMetrics) {
+    this.sourceMetricsList = srMetricsList;
+    this.sinkMetrics = skMetrics;
+
+    // build the SinkLoad
+    ClusterStatusProtos.ReplicationLoadSink.Builder rLoadSinkBuild =
+        ClusterStatusProtos.ReplicationLoadSink.newBuilder();
+    rLoadSinkBuild.setAgeOfLastAppliedOp(sinkMetrics.getAgeOfLastAppliedOp());
+    rLoadSinkBuild.setTimeStampsOfLastAppliedOp(sinkMetrics.getTimeStampOfLastAppliedOp());
+    this.replicationLoadSink = rLoadSinkBuild.build();
+
+    // build the SourceLoad List
+    this.replicationLoadSourceList = new ArrayList<ClusterStatusProtos.ReplicationLoadSource>();
+    for (MetricsSource sm : this.sourceMetricsList) {
+      long ageOfLastShippedOp = sm.getAgeOfLastShippedOp();
+      int sizeOfLogQueue = sm.getSizeOfLogQueue();
+      long timeStampOfLastShippedOp = sm.getTimeStampOfLastShippedOp();
+      long replicationLag;
+      long timePassedAfterLastShippedOp =
+          EnvironmentEdgeManager.currentTime() - timeStampOfLastShippedOp;
+      if (sizeOfLogQueue != 0) {
+        // err on the large side
+        replicationLag = Math.max(ageOfLastShippedOp, timePassedAfterLastShippedOp);
+      } else if (timePassedAfterLastShippedOp < 2 * ageOfLastShippedOp) {
+        replicationLag = ageOfLastShippedOp; // last shipped happen recently
+      } else {
+        // last shipped may happen last night,
+        // so NO real lag although ageOfLastShippedOp is non-zero
+        replicationLag = 0;
+      }
+
+      ClusterStatusProtos.ReplicationLoadSource.Builder rLoadSourceBuild =
+          ClusterStatusProtos.ReplicationLoadSource.newBuilder();
+      rLoadSourceBuild.setPeerID(sm.getPeerID());
+      rLoadSourceBuild.setAgeOfLastShippedOp(ageOfLastShippedOp);
+      rLoadSourceBuild.setSizeOfLogQueue(sizeOfLogQueue);
+      rLoadSourceBuild.setTimeStampOfLastShippedOp(timeStampOfLastShippedOp);
+      rLoadSourceBuild.setReplicationLag(replicationLag);
+
+      this.replicationLoadSourceList.add(rLoadSourceBuild.build());
+    }
+
+  }
+
+  /**
+   * sourceToString
+   * @return a string contains sourceReplicationLoad information
+   */
+  public String sourceToString() {
+    if (this.sourceMetricsList == null) return null;
+
+    StringBuilder sb = new StringBuilder();
+
+    for (ClusterStatusProtos.ReplicationLoadSource rls : this.replicationLoadSourceList) {
+
+      sb = Strings.appendKeyValue(sb, "\n           PeerID", rls.getPeerID());
+      sb = Strings.appendKeyValue(sb, "AgeOfLastShippedOp", rls.getAgeOfLastShippedOp());
+      sb = Strings.appendKeyValue(sb, "SizeOfLogQueue", rls.getSizeOfLogQueue());
+      sb =
+          Strings.appendKeyValue(sb, "TimeStampsOfLastShippedOp",
+            (new Date(rls.getTimeStampOfLastShippedOp()).toString()));
+      sb = Strings.appendKeyValue(sb, "Replication Lag", rls.getReplicationLag());
+    }
+
+    return sb.toString();
+  }
+
+  /**
+   * sinkToString
+   * @return a string contains sinkReplicationLoad information
+   */
+  public String sinkToString() {
+    if (this.replicationLoadSink == null) return null;
+
+    StringBuilder sb = new StringBuilder();
+    sb =
+        Strings.appendKeyValue(sb, "AgeOfLastAppliedOp",
+          this.replicationLoadSink.getAgeOfLastAppliedOp());
+    sb =
+        Strings.appendKeyValue(sb, "TimeStampsOfLastAppliedOp",
+          (new Date(this.replicationLoadSink.getTimeStampsOfLastAppliedOp()).toString()));
+
+    return sb.toString();
+  }
+
+  public ClusterStatusProtos.ReplicationLoadSink getReplicationLoadSink() {
+    return this.replicationLoadSink;
+  }
+
+  public List<ClusterStatusProtos.ReplicationLoadSource> getReplicationLoadSourceList() {
+    return this.replicationLoadSourceList;
+  }
+
+  /**
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    return this.sourceToString() + System.getProperty("line.separator") + this.sinkToString();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
index 9a60131..3276418 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
@@ -254,4 +254,12 @@ public class ReplicationSink {
       "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() +
       ", total replicated edits: " + this.totalReplicatedEdits;
   }
+
+  /**
+   * Get replication Sink Metrics
+   * @return MetricsSink
+   */
+  public MetricsSink getSinkMetrics() {
+    return this.metrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index ee43956..714080f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -869,4 +869,12 @@ public class ReplicationSource extends Thread
       ", currently replicating from: " + this.currentPath +
       " at position: " + position;
   }
+
+  /**
+   * Get Replication Source Metrics
+   * @return sourceMetrics
+   */
+  public MetricsSource getSourceMetrics() {
+    return this.metrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index 4163b66..d8d735f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -31,13 +31,16 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -48,6 +51,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
@@ -556,4 +560,45 @@ public class TestReplicationSmallTests extends TestReplicationBase {
     hadmin.close();
   }
 
+  /**
+   * Test for HBASE-9531
+   * put a few rows into htable1, which should be replicated to htable2
+   * create a ClusterStatus instance 'status' from HBaseAdmin
+   * test : status.getLoad(server).getReplicationLoadSourceList()
+   * test : status.getLoad(server).getReplicationLoadSink()
+   * * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testReplicationStatus() throws Exception {
+    LOG.info("testReplicationStatus");
+
+    try (Admin admin = utility1.getConnection().getAdmin()) {
+
+      final byte[] qualName = Bytes.toBytes("q");
+      Put p;
+
+      for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+        p = new Put(Bytes.toBytes("row" + i));
+        p.add(famName, qualName, Bytes.toBytes("val" + i));
+        htable1.put(p);
+      }
+
+      ClusterStatus status = admin.getClusterStatus();
+
+      for (ServerName server : status.getServers()) {
+        ServerLoad sl = status.getLoad(server);
+        List<ReplicationLoadSource> rLoadSourceList = sl.getReplicationLoadSourceList();
+        ReplicationLoadSink rLoadSink = sl.getReplicationLoadSink();
+
+        // check SourceList has at least one entry
+        assertTrue("failed to get ReplicationLoadSourceList", (rLoadSourceList.size() > 0));
+
+        // check Sink exist only as it is difficult to verify the value on the fly
+        assertTrue("failed to get ReplicationLoadSink.AgeOfLastShippedOp ",
+          (rLoadSink.getAgeOfLastAppliedOp() >= 0));
+        assertTrue("failed to get ReplicationLoadSink.TimeStampsOfLastAppliedOp ",
+          (rLoadSink.getTimeStampsOfLastAppliedOp() >= 0));
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index c0ea862..35ee36c 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -608,7 +608,7 @@ module Hbase
       end
     end
 
-    def status(format)
+    def status(format, type)
       status = @admin.getClusterStatus()
       if format == "detailed"
         puts("version %s" % [ status.getHBaseVersion() ])
@@ -635,6 +635,46 @@ module Hbase
         for server in status.getDeadServerNames()
           puts("    %s" % [ server ])
         end
+      elsif format == "replication"
+        #check whether replication is enabled or not
+        if (!@admin.getConfiguration().getBoolean(org.apache.hadoop.hbase.HConstants::REPLICATION_ENABLE_KEY, 
+          org.apache.hadoop.hbase.HConstants::REPLICATION_ENABLE_DEFAULT))
+          puts("Please enable replication first.")
+        else
+          puts("version %s" % [ status.getHBaseVersion() ])
+          puts("%d live servers" % [ status.getServersSize() ])
+          for server in status.getServers()
+            sl = status.getLoad(server)
+            rSinkString   = "       SINK  :"
+            rSourceString = "       SOURCE:"
+            rLoadSink = sl.getReplicationLoadSink()
+            rSinkString << " AgeOfLastAppliedOp=" + rLoadSink.getAgeOfLastAppliedOp().to_s
+            rSinkString << ", TimeStampsOfLastAppliedOp=" + 
+			    (java.util.Date.new(rLoadSink.getTimeStampsOfLastAppliedOp())).toString()
+            rLoadSourceList = sl.getReplicationLoadSourceList()
+            index = 0
+            while index < rLoadSourceList.size()
+              rLoadSource = rLoadSourceList.get(index)
+              rSourceString << " PeerID=" + rLoadSource.getPeerID()
+              rSourceString << ", AgeOfLastShippedOp=" + rLoadSource.getAgeOfLastShippedOp().to_s
+              rSourceString << ", SizeOfLogQueue=" + rLoadSource.getSizeOfLogQueue().to_s
+              rSourceString << ", TimeStampsOfLastShippedOp=" + 
+			      (java.util.Date.new(rLoadSource.getTimeStampOfLastShippedOp())).toString()
+              rSourceString << ", Replication Lag=" + rLoadSource.getReplicationLag().to_s
+              index = index + 1
+            end
+            puts("    %s:" %
+            [ server.getHostname() ])
+            if type.casecmp("SOURCE") == 0
+              puts("%s" % rSourceString)
+            elsif type.casecmp("SINK") == 0
+              puts("%s" % rSinkString)
+            else
+              puts("%s" % rSourceString)
+              puts("%s" % rSinkString)
+            end
+          end
+        end
       elsif format == "simple"
         load = 0
         regions = 0

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-shell/src/main/ruby/shell/commands/status.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/status.rb b/hbase-shell/src/main/ruby/shell/commands/status.rb
index f72c13c..b22b272 100644
--- a/hbase-shell/src/main/ruby/shell/commands/status.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/status.rb
@@ -22,18 +22,21 @@ module Shell
     class Status < Command
       def help
         return <<-EOF
-Show cluster status. Can be 'summary', 'simple', or 'detailed'. The
+Show cluster status. Can be 'summary', 'simple', 'detailed', or 'replication'. The
 default is 'summary'. Examples:
 
   hbase> status
   hbase> status 'simple'
   hbase> status 'summary'
   hbase> status 'detailed'
+  hbase> status 'replication'
+  hbase> status 'replication', 'source'
+  hbase> status 'replication', 'sink'
 EOF
       end
 
-      def command(format = 'summary')
-        admin.status(format)
+      def command(format = 'summary',type = 'both')
+        admin.status(format, type)
       end
     end
   end

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-shell/src/test/ruby/hbase/admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/admin_test.rb b/hbase-shell/src/test/ruby/hbase/admin_test.rb
index caede3a..1925864 100644
--- a/hbase-shell/src/test/ruby/hbase/admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/admin_test.rb
@@ -356,5 +356,17 @@ module Hbase
       assert_not_equal(nil, table)
       table.close
     end
+
+    define_test "Get replication status" do
+      replication_status("replication", "both")
+    end
+
+    define_test "Get replication source metrics information" do
+      replication_status("replication", "source")
+    end
+
+    define_test "Get replication sink metrics information" do
+      replication_status("replication", "sink")
+    end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-shell/src/test/ruby/test_helper.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/test_helper.rb b/hbase-shell/src/test/ruby/test_helper.rb
index 5579761..5dfafc5 100644
--- a/hbase-shell/src/test/ruby/test_helper.rb
+++ b/hbase-shell/src/test/ruby/test_helper.rb
@@ -94,6 +94,10 @@ module Hbase
         puts "IGNORING DROP TABLE ERROR: #{e}"
       end
     end
+
+    def replication_status(format,type)
+      return admin.status(format,type)
+    end
   end
 end
 


[9/9] hbase git commit: HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Posted by ap...@apache.org.
HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Signed-off-by: Andrew Purtell <ap...@apache.org>

Conflicts:
	hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
	hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c391dfbd
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c391dfbd
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c391dfbd

Branch: refs/heads/0.98
Commit: c391dfbd75de3f4d53d052b88048d833289901b6
Parents: 0ffb197
Author: Ashish Singhi <as...@huawei.com>
Authored: Thu Feb 12 14:00:19 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Feb 12 14:28:07 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ServerLoad.java     |   27 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   24 +
 .../hbase/replication/ReplicationLoadSink.java  |   36 +
 .../replication/ReplicationLoadSource.java      |   53 +
 .../MetricsReplicationSinkSource.java           |    1 +
 .../MetricsReplicationSourceSource.java         |    1 +
 .../MetricsReplicationGlobalSourceSource.java   |    5 +
 .../MetricsReplicationSinkSourceImpl.java       |    5 +
 .../MetricsReplicationSourceSourceImpl.java     |    5 +
 .../protobuf/generated/ClusterStatusProtos.java | 5535 +++++++++++++-----
 .../src/main/protobuf/ClusterStatus.proto       |   23 +
 .../hbase/regionserver/HRegionServer.java       |   17 +
 .../hbase/regionserver/ReplicationService.java  |    8 +-
 .../replication/regionserver/MetricsSink.java   |   17 +
 .../replication/regionserver/MetricsSource.java |   34 +
 .../replication/regionserver/Replication.java   |   30 +
 .../regionserver/ReplicationLoad.java           |  151 +
 .../regionserver/ReplicationSink.java           |    8 +
 .../regionserver/ReplicationSource.java         |    8 +
 .../replication/TestReplicationSmallTests.java  |   49 +-
 hbase-shell/src/main/ruby/hbase/admin.rb        |   42 +-
 .../src/main/ruby/shell/commands/status.rb      |    9 +-
 hbase-shell/src/test/ruby/hbase/admin_test.rb   |   12 +
 hbase-shell/src/test/ruby/test_helper.rb        |    4 +
 24 files changed, 4472 insertions(+), 1632 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
index 4c69e6d..bea9d5c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
@@ -22,9 +22,12 @@ package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Strings;
 
@@ -53,7 +56,7 @@ public class ServerLoad {
   private int totalStaticBloomSizeKB = 0;
   private long totalCompactingKVs = 0;
   private long currentCompactedKVs = 0;
-  
+
   public ServerLoad(ClusterStatusProtos.ServerLoad serverLoad) {
     this.serverLoad = serverLoad;
     for (ClusterStatusProtos.RegionLoad rl: serverLoad.getRegionLoadsList()) {
@@ -71,7 +74,7 @@ public class ServerLoad {
       totalCompactingKVs += rl.getTotalCompactingKVs();
       currentCompactedKVs += rl.getCurrentCompactedKVs();
     }
-    
+
   }
 
   // NOTE: Function name cannot start with "get" because then an OpenDataException is thrown because
@@ -179,6 +182,26 @@ public class ServerLoad {
   }
 
   /**
+   * Call directly from client such as hbase shell
+   * @return the list of ReplicationLoadSource
+   */
+  public List<ReplicationLoadSource> getReplicationLoadSourceList() {
+    return ProtobufUtil.toReplicationLoadSourceList(serverLoad.getReplLoadSourceList());
+  }
+
+  /**
+   * Call directly from client such as hbase shell
+   * @return ReplicationLoadSink
+   */
+  public ReplicationLoadSink getReplicationLoadSink() {
+    if (serverLoad.hasReplLoadSink()) {
+      return ProtobufUtil.toReplicationLoadSink(serverLoad.getReplLoadSink());
+    } else {
+      return null;
+    }
+  }
+
+  /**
    * Originally, this method factored in the effect of requests going to the
    * server as well. However, this does not interact very well with the current
    * region rebalancing code, which only factors number of regions. For the

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 5f76bb1..e19c5b3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -95,6 +95,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Col
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
+import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
 import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -110,6 +111,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.TablePermission;
 import org.apache.hadoop.hbase.security.access.UserPermission;
@@ -2787,4 +2790,25 @@ public final class ProtobufUtil {
     }
     return result;
   }
+
+  public static ReplicationLoadSink toReplicationLoadSink(
+      ClusterStatusProtos.ReplicationLoadSink cls) {
+    return new ReplicationLoadSink(cls.getAgeOfLastAppliedOp(), cls.getTimeStampsOfLastAppliedOp());
+  }
+
+  public static ReplicationLoadSource toReplicationLoadSource(
+      ClusterStatusProtos.ReplicationLoadSource cls) {
+    return new ReplicationLoadSource(cls.getPeerID(), cls.getAgeOfLastShippedOp(),
+        cls.getSizeOfLogQueue(), cls.getTimeStampOfLastShippedOp(), cls.getReplicationLag());
+  }
+
+  public static List<ReplicationLoadSource> toReplicationLoadSourceList(
+      List<ClusterStatusProtos.ReplicationLoadSource> clsList) {
+    ArrayList<ReplicationLoadSource> rlsList = new ArrayList<ReplicationLoadSource>();
+    for (ClusterStatusProtos.ReplicationLoadSource cls : clsList) {
+      rlsList.add(toReplicationLoadSource(cls));
+    }
+    return rlsList;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
new file mode 100644
index 0000000..63fe334
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF 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.apache.hadoop.hbase.replication;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A HBase ReplicationLoad to present MetricsSink information
+ */
+@InterfaceAudience.Private
+public class ReplicationLoadSink {
+  private long ageOfLastAppliedOp;
+  private long timeStampsOfLastAppliedOp;
+
+  public ReplicationLoadSink(long age, long timeStamp) {
+    this.ageOfLastAppliedOp = age;
+    this.timeStampsOfLastAppliedOp = timeStamp;
+  }
+
+  public long getAgeOfLastAppliedOp() {
+    return this.ageOfLastAppliedOp;
+  }
+
+  public long getTimeStampsOfLastAppliedOp() {
+    return this.timeStampsOfLastAppliedOp;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
new file mode 100644
index 0000000..bfd1599
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF 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.apache.hadoop.hbase.replication;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A HBase ReplicationLoad to present MetricsSource information
+ */
+@InterfaceAudience.Private
+public class ReplicationLoadSource {
+  private String peerID;
+  private long ageOfLastShippedOp;
+  private int sizeOfLogQueue;
+  private long timeStampOfLastShippedOp;
+  private long replicationLag;
+
+  public ReplicationLoadSource(String id, long age, int size, long timeStamp, long lag) {
+    this.peerID = id;
+    this.ageOfLastShippedOp = age;
+    this.sizeOfLogQueue = size;
+    this.timeStampOfLastShippedOp = timeStamp;
+    this.replicationLag = lag;
+  }
+
+  public String getPeerID() {
+    return this.peerID;
+  }
+
+  public long getAgeOfLastShippedOp() {
+    return this.ageOfLastShippedOp;
+  }
+
+  public long getSizeOfLogQueue() {
+    return this.sizeOfLogQueue;
+  }
+
+  public long getTimeStampOfLastShippedOp() {
+    return this.timeStampOfLastShippedOp;
+  }
+
+  public long getReplicationLag() {
+    return this.replicationLag;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
index 50af94e..0aeb2af 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
@@ -25,4 +25,5 @@ public interface MetricsReplicationSinkSource {
   void setLastAppliedOpAge(long age);
   void incrAppliedBatches(long batches);
   void incrAppliedOps(long batchsize);
+  long getLastAppliedOpAge();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
index 66d265a..fecf191 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
@@ -43,4 +43,5 @@ public interface MetricsReplicationSourceSource {
   void incrLogReadInBytes(long size);
   void incrLogReadInEdits(long size);
   void clear();
+  long getLastShippedAge();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
index 5c9261e..08baf23 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
@@ -93,4 +93,9 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
 
   @Override public void clear() {
   }
+
+  @Override
+  public long getLastShippedAge() {
+    return ageOfLastShippedOpGauge.value();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
index 3025e3e..14212ba 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
@@ -44,4 +44,9 @@ public class MetricsReplicationSinkSourceImpl implements MetricsReplicationSinkS
   @Override public void incrAppliedOps(long batchsize) {
     opsCounter.incr(batchsize);
   }
+
+  @Override
+  public long getLastAppliedOpAge() {
+    return ageGauge.value();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c391dfbd/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
index b4c631c..d5e52b8 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
@@ -126,4 +126,9 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
 
     rms.removeMetric(logEditsFilteredKey);
   }
+
+  @Override
+  public long getLastShippedAge() {
+    return ageOfLastShippedOpGauge.value();
+  }
 }


[6/9] hbase git commit: HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Posted by ap...@apache.org.
HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Signed-off-by: Andrew Purtell <ap...@apache.org>

Conflicts:
	hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/f5b40200
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f5b40200
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f5b40200

Branch: refs/heads/branch-1
Commit: f5b40200db644c7c9f0cf51128337faad65b32a1
Parents: 7841bf7
Author: Ashish Singhi <as...@huawei.com>
Authored: Thu Feb 12 14:00:19 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Feb 12 14:16:28 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ServerLoad.java     |   27 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   24 +
 .../hbase/replication/ReplicationLoadSink.java  |   36 +
 .../replication/ReplicationLoadSource.java      |   53 +
 .../MetricsReplicationSinkSource.java           |    1 +
 .../MetricsReplicationSourceSource.java         |    1 +
 .../MetricsReplicationGlobalSourceSource.java   |    5 +
 .../MetricsReplicationSinkSourceImpl.java       |    5 +
 .../MetricsReplicationSourceSourceImpl.java     |    5 +
 .../protobuf/generated/ClusterStatusProtos.java | 5535 +++++++++++++-----
 .../src/main/protobuf/ClusterStatus.proto       |   23 +
 .../hbase/regionserver/HRegionServer.java       |   17 +
 .../hbase/regionserver/ReplicationService.java  |    8 +-
 .../replication/regionserver/MetricsSink.java   |   17 +
 .../replication/regionserver/MetricsSource.java |   34 +
 .../replication/regionserver/Replication.java   |   32 +-
 .../regionserver/ReplicationLoad.java           |  151 +
 .../regionserver/ReplicationSink.java           |    8 +
 .../regionserver/ReplicationSource.java         |    8 +
 .../replication/TestReplicationSmallTests.java  |   49 +-
 hbase-shell/src/main/ruby/hbase/admin.rb        |   42 +-
 .../src/main/ruby/shell/commands/status.rb      |    9 +-
 hbase-shell/src/test/ruby/hbase/admin_test.rb   |   12 +
 hbase-shell/src/test/ruby/test_helper.rb        |    4 +
 24 files changed, 4472 insertions(+), 1634 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
index c535cec..4f4d301 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
@@ -22,8 +22,11 @@ package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Strings;
 
@@ -52,7 +55,7 @@ public class ServerLoad {
   private int totalStaticBloomSizeKB = 0;
   private long totalCompactingKVs = 0;
   private long currentCompactedKVs = 0;
-  
+
   public ServerLoad(ClusterStatusProtos.ServerLoad serverLoad) {
     this.serverLoad = serverLoad;
     for (ClusterStatusProtos.RegionLoad rl: serverLoad.getRegionLoadsList()) {
@@ -70,7 +73,7 @@ public class ServerLoad {
       totalCompactingKVs += rl.getTotalCompactingKVs();
       currentCompactedKVs += rl.getCurrentCompactedKVs();
     }
-    
+
   }
 
   // NOTE: Function name cannot start with "get" because then an OpenDataException is thrown because
@@ -178,6 +181,26 @@ public class ServerLoad {
   }
 
   /**
+   * Call directly from client such as hbase shell
+   * @return the list of ReplicationLoadSource
+   */
+  public List<ReplicationLoadSource> getReplicationLoadSourceList() {
+    return ProtobufUtil.toReplicationLoadSourceList(serverLoad.getReplLoadSourceList());
+  }
+
+  /**
+   * Call directly from client such as hbase shell
+   * @return ReplicationLoadSink
+   */
+  public ReplicationLoadSink getReplicationLoadSink() {
+    if (serverLoad.hasReplLoadSink()) {
+      return ProtobufUtil.toReplicationLoadSink(serverLoad.getReplLoadSink());
+    } else {
+      return null;
+    }
+  }
+
+  /**
    * Originally, this method factored in the effect of requests going to the
    * server as well. However, this does not interact very well with the current
    * region rebalancing code, which only factors number of regions. For the

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index b8d8ce6..851406a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -102,6 +102,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Col
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
+import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
 import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -124,6 +125,8 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescripto
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.TablePermission;
 import org.apache.hadoop.hbase.security.access.UserPermission;
@@ -2859,4 +2862,25 @@ public final class ProtobufUtil {
 
     return desc.build();
   }
+
+  public static ReplicationLoadSink toReplicationLoadSink(
+      ClusterStatusProtos.ReplicationLoadSink cls) {
+    return new ReplicationLoadSink(cls.getAgeOfLastAppliedOp(), cls.getTimeStampsOfLastAppliedOp());
+  }
+
+  public static ReplicationLoadSource toReplicationLoadSource(
+      ClusterStatusProtos.ReplicationLoadSource cls) {
+    return new ReplicationLoadSource(cls.getPeerID(), cls.getAgeOfLastShippedOp(),
+        cls.getSizeOfLogQueue(), cls.getTimeStampOfLastShippedOp(), cls.getReplicationLag());
+  }
+
+  public static List<ReplicationLoadSource> toReplicationLoadSourceList(
+      List<ClusterStatusProtos.ReplicationLoadSource> clsList) {
+    ArrayList<ReplicationLoadSource> rlsList = new ArrayList<ReplicationLoadSource>();
+    for (ClusterStatusProtos.ReplicationLoadSource cls : clsList) {
+      rlsList.add(toReplicationLoadSource(cls));
+    }
+    return rlsList;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
new file mode 100644
index 0000000..63fe334
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF 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.apache.hadoop.hbase.replication;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A HBase ReplicationLoad to present MetricsSink information
+ */
+@InterfaceAudience.Private
+public class ReplicationLoadSink {
+  private long ageOfLastAppliedOp;
+  private long timeStampsOfLastAppliedOp;
+
+  public ReplicationLoadSink(long age, long timeStamp) {
+    this.ageOfLastAppliedOp = age;
+    this.timeStampsOfLastAppliedOp = timeStamp;
+  }
+
+  public long getAgeOfLastAppliedOp() {
+    return this.ageOfLastAppliedOp;
+  }
+
+  public long getTimeStampsOfLastAppliedOp() {
+    return this.timeStampsOfLastAppliedOp;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
new file mode 100644
index 0000000..bfd1599
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF 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.apache.hadoop.hbase.replication;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A HBase ReplicationLoad to present MetricsSource information
+ */
+@InterfaceAudience.Private
+public class ReplicationLoadSource {
+  private String peerID;
+  private long ageOfLastShippedOp;
+  private int sizeOfLogQueue;
+  private long timeStampOfLastShippedOp;
+  private long replicationLag;
+
+  public ReplicationLoadSource(String id, long age, int size, long timeStamp, long lag) {
+    this.peerID = id;
+    this.ageOfLastShippedOp = age;
+    this.sizeOfLogQueue = size;
+    this.timeStampOfLastShippedOp = timeStamp;
+    this.replicationLag = lag;
+  }
+
+  public String getPeerID() {
+    return this.peerID;
+  }
+
+  public long getAgeOfLastShippedOp() {
+    return this.ageOfLastShippedOp;
+  }
+
+  public long getSizeOfLogQueue() {
+    return this.sizeOfLogQueue;
+  }
+
+  public long getTimeStampOfLastShippedOp() {
+    return this.timeStampOfLastShippedOp;
+  }
+
+  public long getReplicationLag() {
+    return this.replicationLag;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
index 805dfca..698a59a 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
@@ -26,4 +26,5 @@ public interface MetricsReplicationSinkSource {
   void setLastAppliedOpAge(long age);
   void incrAppliedBatches(long batches);
   void incrAppliedOps(long batchsize);
+  long getLastAppliedOpAge();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
index 66d265a..fecf191 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
@@ -43,4 +43,5 @@ public interface MetricsReplicationSourceSource {
   void incrLogReadInBytes(long size);
   void incrLogReadInEdits(long size);
   void clear();
+  long getLastShippedAge();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
index a210171..6dace10 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
@@ -95,4 +95,9 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
 
   @Override public void clear() {
   }
+
+  @Override
+  public long getLastShippedAge() {
+    return ageOfLastShippedOpGauge.value();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
index 3025e3e..14212ba 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
@@ -44,4 +44,9 @@ public class MetricsReplicationSinkSourceImpl implements MetricsReplicationSinkS
   @Override public void incrAppliedOps(long batchsize) {
     opsCounter.incr(batchsize);
   }
+
+  @Override
+  public long getLastAppliedOpAge() {
+    return ageGauge.value();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
index 89ef4de..1422e7e 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
@@ -125,4 +125,9 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
 
     rms.removeMetric(logEditsFilteredKey);
   }
+
+  @Override
+  public long getLastShippedAge() {
+    return ageOfLastShippedOpGauge.value();
+  }
 }


[5/9] hbase git commit: HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/f5b40200/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
index 6dc48fa..0d69d7a 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
@@ -4438,273 +4438,48 @@ public final class ClusterStatusProtos {
     // @@protoc_insertion_point(class_scope:RegionLoad)
   }
 
-  public interface ServerLoadOrBuilder
+  public interface ReplicationLoadSinkOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // optional uint32 number_of_requests = 1;
-    /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
-     */
-    boolean hasNumberOfRequests();
-    /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
-     */
-    int getNumberOfRequests();
-
-    // optional uint32 total_number_of_requests = 2;
-    /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
-     */
-    boolean hasTotalNumberOfRequests();
-    /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
-     */
-    int getTotalNumberOfRequests();
-
-    // optional uint32 used_heap_MB = 3;
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    boolean hasUsedHeapMB();
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    int getUsedHeapMB();
-
-    // optional uint32 max_heap_MB = 4;
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    boolean hasMaxHeapMB();
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    int getMaxHeapMB();
-
-    // repeated .RegionLoad region_loads = 5;
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad> 
-        getRegionLoadsList();
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad getRegionLoads(int index);
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    int getRegionLoadsCount();
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder> 
-        getRegionLoadsOrBuilderList();
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder getRegionLoadsOrBuilder(
-        int index);
-
-    // repeated .Coprocessor coprocessors = 6;
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor> 
-        getCoprocessorsList();
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor getCoprocessors(int index);
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    int getCoprocessorsCount();
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder> 
-        getCoprocessorsOrBuilderList();
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder getCoprocessorsOrBuilder(
-        int index);
-
-    // optional uint64 report_start_time = 7;
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    boolean hasReportStartTime();
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    long getReportStartTime();
-
-    // optional uint64 report_end_time = 8;
+    // required uint64 ageOfLastAppliedOp = 1;
     /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    boolean hasReportEndTime();
+    boolean hasAgeOfLastAppliedOp();
     /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    long getReportEndTime();
+    long getAgeOfLastAppliedOp();
 
-    // optional uint32 info_server_port = 9;
+    // required uint64 timeStampsOfLastAppliedOp = 2;
     /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    boolean hasInfoServerPort();
+    boolean hasTimeStampsOfLastAppliedOp();
     /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    int getInfoServerPort();
+    long getTimeStampsOfLastAppliedOp();
   }
   /**
-   * Protobuf type {@code ServerLoad}
+   * Protobuf type {@code ReplicationLoadSink}
    */
-  public static final class ServerLoad extends
+  public static final class ReplicationLoadSink extends
       com.google.protobuf.GeneratedMessage
-      implements ServerLoadOrBuilder {
-    // Use ServerLoad.newBuilder() to construct.
-    private ServerLoad(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements ReplicationLoadSinkOrBuilder {
+    // Use ReplicationLoadSink.newBuilder() to construct.
+    private ReplicationLoadSink(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private ServerLoad(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private ReplicationLoadSink(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final ServerLoad defaultInstance;
-    public static ServerLoad getDefaultInstance() {
+    private static final ReplicationLoadSink defaultInstance;
+    public static ReplicationLoadSink getDefaultInstance() {
       return defaultInstance;
     }
 
-    public ServerLoad getDefaultInstanceForType() {
+    public ReplicationLoadSink getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -4714,7 +4489,7 @@ public final class ClusterStatusProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private ServerLoad(
+    private ReplicationLoadSink(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -4739,53 +4514,12 @@ public final class ClusterStatusProtos {
             }
             case 8: {
               bitField0_ |= 0x00000001;
-              numberOfRequests_ = input.readUInt32();
+              ageOfLastAppliedOp_ = input.readUInt64();
               break;
             }
             case 16: {
               bitField0_ |= 0x00000002;
-              totalNumberOfRequests_ = input.readUInt32();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              usedHeapMB_ = input.readUInt32();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              maxHeapMB_ = input.readUInt32();
-              break;
-            }
-            case 42: {
-              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-                regionLoads_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad>();
-                mutable_bitField0_ |= 0x00000010;
-              }
-              regionLoads_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad.PARSER, extensionRegistry));
-              break;
-            }
-            case 50: {
-              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-                coprocessors_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor>();
-                mutable_bitField0_ |= 0x00000020;
-              }
-              coprocessors_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor.PARSER, extensionRegistry));
-              break;
-            }
-            case 56: {
-              bitField0_ |= 0x00000010;
-              reportStartTime_ = input.readUInt64();
-              break;
-            }
-            case 64: {
-              bitField0_ |= 0x00000020;
-              reportEndTime_ = input.readUInt64();
-              break;
-            }
-            case 72: {
-              bitField0_ |= 0x00000040;
-              infoServerPort_ = input.readUInt32();
+              timeStampsOfLastAppliedOp_ = input.readUInt64();
               break;
             }
           }
@@ -4796,1957 +4530,4480 @@ public final class ClusterStatusProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-          regionLoads_ = java.util.Collections.unmodifiableList(regionLoads_);
-        }
-        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-          coprocessors_ = java.util.Collections.unmodifiableList(coprocessors_);
-        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<ServerLoad> PARSER =
-        new com.google.protobuf.AbstractParser<ServerLoad>() {
-      public ServerLoad parsePartialFrom(
+    public static com.google.protobuf.Parser<ReplicationLoadSink> PARSER =
+        new com.google.protobuf.AbstractParser<ReplicationLoadSink>() {
+      public ReplicationLoadSink parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new ServerLoad(input, extensionRegistry);
+        return new ReplicationLoadSink(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<ServerLoad> getParserForType() {
+    public com.google.protobuf.Parser<ReplicationLoadSink> getParserForType() {
       return PARSER;
     }
 
     private int bitField0_;
-    // optional uint32 number_of_requests = 1;
-    public static final int NUMBER_OF_REQUESTS_FIELD_NUMBER = 1;
-    private int numberOfRequests_;
+    // required uint64 ageOfLastAppliedOp = 1;
+    public static final int AGEOFLASTAPPLIEDOP_FIELD_NUMBER = 1;
+    private long ageOfLastAppliedOp_;
     /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    public boolean hasNumberOfRequests() {
+    public boolean hasAgeOfLastAppliedOp() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    public int getNumberOfRequests() {
-      return numberOfRequests_;
+    public long getAgeOfLastAppliedOp() {
+      return ageOfLastAppliedOp_;
     }
 
-    // optional uint32 total_number_of_requests = 2;
-    public static final int TOTAL_NUMBER_OF_REQUESTS_FIELD_NUMBER = 2;
-    private int totalNumberOfRequests_;
+    // required uint64 timeStampsOfLastAppliedOp = 2;
+    public static final int TIMESTAMPSOFLASTAPPLIEDOP_FIELD_NUMBER = 2;
+    private long timeStampsOfLastAppliedOp_;
     /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    public boolean hasTotalNumberOfRequests() {
+    public boolean hasTimeStampsOfLastAppliedOp() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    public int getTotalNumberOfRequests() {
-      return totalNumberOfRequests_;
+    public long getTimeStampsOfLastAppliedOp() {
+      return timeStampsOfLastAppliedOp_;
     }
 
-    // optional uint32 used_heap_MB = 3;
-    public static final int USED_HEAP_MB_FIELD_NUMBER = 3;
-    private int usedHeapMB_;
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    public boolean hasUsedHeapMB() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
+    private void initFields() {
+      ageOfLastAppliedOp_ = 0L;
+      timeStampsOfLastAppliedOp_ = 0L;
     }
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    public int getUsedHeapMB() {
-      return usedHeapMB_;
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasAgeOfLastAppliedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTimeStampsOfLastAppliedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
     }
 
-    // optional uint32 max_heap_MB = 4;
-    public static final int MAX_HEAP_MB_FIELD_NUMBER = 4;
-    private int maxHeapMB_;
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    public boolean hasMaxHeapMB() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt64(1, ageOfLastAppliedOp_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, timeStampsOfLastAppliedOp_);
+      }
+      getUnknownFields().writeTo(output);
     }
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    public int getMaxHeapMB() {
-      return maxHeapMB_;
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(1, ageOfLastAppliedOp_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, timeStampsOfLastAppliedOp_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
     }
 
-    // repeated .RegionLoad region_loads = 5;
-    public static final int REGION_LOADS_FIELD_NUMBER = 5;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad> regionLoads_;
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad> getRegionLoadsList() {
-      return regionLoads_;
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder> 
-        getRegionLoadsOrBuilderList() {
-      return regionLoads_;
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink other = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink) obj;
+
+      boolean result = true;
+      result = result && (hasAgeOfLastAppliedOp() == other.hasAgeOfLastAppliedOp());
+      if (hasAgeOfLastAppliedOp()) {
+        result = result && (getAgeOfLastAppliedOp()
+            == other.getAgeOfLastAppliedOp());
+      }
+      result = result && (hasTimeStampsOfLastAppliedOp() == other.hasTimeStampsOfLastAppliedOp());
+      if (hasTimeStampsOfLastAppliedOp()) {
+        result = result && (getTimeStampsOfLastAppliedOp()
+            == other.getTimeStampsOfLastAppliedOp());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public int getRegionLoadsCount() {
-      return regionLoads_.size();
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasAgeOfLastAppliedOp()) {
+        hash = (37 * hash) + AGEOFLASTAPPLIEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getAgeOfLastAppliedOp());
+      }
+      if (hasTimeStampsOfLastAppliedOp()) {
+        hash = (37 * hash) + TIMESTAMPSOFLASTAPPLIEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTimeStampsOfLastAppliedOp());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad getRegionLoads(int index) {
-      return regionLoads_.get(index);
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder getRegionLoadsOrBuilder(
-        int index) {
-      return regionLoads_.get(index);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
     }
-
-    // repeated .Coprocessor coprocessors = 6;
-    public static final int COPROCESSORS_FIELD_NUMBER = 6;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor> coprocessors_;
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor> getCoprocessorsList() {
-      return coprocessors_;
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder> 
-        getCoprocessorsOrBuilderList() {
-      return coprocessors_;
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public int getCoprocessorsCount() {
-      return coprocessors_.size();
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor getCoprocessors(int index) {
-      return coprocessors_.get(index);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder getCoprocessorsOrBuilder(
-        int index) {
-      return coprocessors_.get(index);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
     }
-
-    // optional uint64 report_start_time = 7;
-    public static final int REPORT_START_TIME_FIELD_NUMBER = 7;
-    private long reportStartTime_;
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public boolean hasReportStartTime() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public long getReportStartTime() {
-      return reportStartTime_;
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
     }
-
-    // optional uint64 report_end_time = 8;
-    public static final int REPORT_END_TIME_FIELD_NUMBER = 8;
-    private long reportEndTime_;
-    /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public boolean hasReportEndTime() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
     }
-    /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public long getReportEndTime() {
-      return reportEndTime_;
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink prototype) {
+      return newBuilder().mergeFrom(prototype);
     }
+    public Builder toBuilder() { return newBuilder(this); }
 
-    // optional uint32 info_server_port = 9;
-    public static final int INFO_SERVER_PORT_FIELD_NUMBER = 9;
-    private int infoServerPort_;
-    /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
-     */
-    public boolean hasInfoServerPort() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
     }
     /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
+     * Protobuf type {@code ReplicationLoadSink}
      */
-    public int getInfoServerPort() {
-      return infoServerPort_;
-    }
-
-    private void initFields() {
-      numberOfRequests_ = 0;
-      totalNumberOfRequests_ = 0;
-      usedHeapMB_ = 0;
-      maxHeapMB_ = 0;
-      regionLoads_ = java.util.Collections.emptyList();
-      coprocessors_ = java.util.Collections.emptyList();
-      reportStartTime_ = 0L;
-      reportEndTime_ = 0L;
-      infoServerPort_ = 0;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      for (int i = 0; i < getRegionLoadsCount(); i++) {
-        if (!getRegionLoads(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSinkOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_descriptor;
       }
-      for (int i = 0; i < getCoprocessorsCount(); i++) {
-        if (!getCoprocessors(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.Builder.class);
       }
-      memoizedIsInitialized = 1;
-      return true;
-    }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeUInt32(1, numberOfRequests_);
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeUInt32(2, totalNumberOfRequests_);
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeUInt32(3, usedHeapMB_);
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
       }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeUInt32(4, maxHeapMB_);
+      private static Builder create() {
+        return new Builder();
       }
-      for (int i = 0; i < regionLoads_.size(); i++) {
-        output.writeMessage(5, regionLoads_.get(i));
+
+      public Builder clear() {
+        super.clear();
+        ageOfLastAppliedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        timeStampsOfLastAppliedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
       }
-      for (int i = 0; i < coprocessors_.size(); i++) {
-        output.writeMessage(6, coprocessors_.get(i));
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
       }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeUInt64(7, reportStartTime_);
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_descriptor;
       }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeUInt64(8, reportEndTime_);
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.getDefaultInstance();
       }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeUInt32(9, infoServerPort_);
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
 
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(1, numberOfRequests_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(2, totalNumberOfRequests_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(3, usedHeapMB_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(4, maxHeapMB_);
-      }
-      for (int i = 0; i < regionLoads_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(5, regionLoads_.get(i));
-      }
-      for (int i = 0; i < coprocessors_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(6, coprocessors_.get(i));
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(7, reportStartTime_);
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(8, reportEndTime_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(9, infoServerPort_);
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink build() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
       }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
 
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad)) {
-        return super.equals(obj);
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink result = new org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.ageOfLastAppliedOp_ = ageOfLastAppliedOp_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.timeStampsOfLastAppliedOp_ = timeStampsOfLastAppliedOp_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
       }
-      org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad other = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad) obj;
 
-      boolean result = true;
-      result = result && (hasNumberOfRequests() == other.hasNumberOfRequests());
-      if (hasNumberOfRequests()) {
-        result = result && (getNumberOfRequests()
-            == other.getNumberOfRequests());
-      }
-      result = result && (hasTotalNumberOfRequests() == other.hasTotalNumberOfRequests());
-      if (hasTotalNumberOfRequests()) {
-        result = result && (getTotalNumberOfRequests()
-            == other.getTotalNumberOfRequests());
-      }
-      result = result && (hasUsedHeapMB() == other.hasUsedHeapMB());
-      if (hasUsedHeapMB()) {
-        result = result && (getUsedHeapMB()
-            == other.getUsedHeapMB());
-      }
-      result = result && (hasMaxHeapMB() == other.hasMaxHeapMB());
-      if (hasMaxHeapMB()) {
-        result = result && (getMaxHeapMB()
-            == other.getMaxHeapMB());
-      }
-      result = result && getRegionLoadsList()
-          .equals(other.getRegionLoadsList());
-      result = result && getCoprocessorsList()
-          .equals(other.getCoprocessorsList());
-      result = result && (hasReportStartTime() == other.hasReportStartTime());
-      if (hasReportStartTime()) {
-        result = result && (getReportStartTime()
-            == other.getReportStartTime());
-      }
-      result = result && (hasReportEndTime() == other.hasReportEndTime());
-      if (hasReportEndTime()) {
-        result = result && (getReportEndTime()
-            == other.getReportEndTime());
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
       }
-      result = result && (hasInfoServerPort() == other.hasInfoServerPort());
-      if (hasInfoServerPort()) {
-        result = result && (getInfoServerPort()
-            == other.getInfoServerPort());
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.getDefaultInstance()) return this;
+        if (other.hasAgeOfLastAppliedOp()) {
+          setAgeOfLastAppliedOp(other.getAgeOfLastAppliedOp());
+        }
+        if (other.hasTimeStampsOfLastAppliedOp()) {
+          setTimeStampsOfLastAppliedOp(other.getTimeStampsOfLastAppliedOp());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
       }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
 
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
+      public final boolean isInitialized() {
+        if (!hasAgeOfLastAppliedOp()) {
+          
+          return false;
+        }
+        if (!hasTimeStampsOfLastAppliedOp()) {
+          
+          return false;
+        }
+        return true;
       }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasNumberOfRequests()) {
-        hash = (37 * hash) + NUMBER_OF_REQUESTS_FIELD_NUMBER;
-        hash = (53 * hash) + getNumberOfRequests();
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
       }
-      if (hasTotalNumberOfRequests()) {
-        hash = (37 * hash) + TOTAL_NUMBER_OF_REQUESTS_FIELD_NUMBER;
-        hash = (53 * hash) + getTotalNumberOfRequests();
+      private int bitField0_;
+
+      // required uint64 ageOfLastAppliedOp = 1;
+      private long ageOfLastAppliedOp_ ;
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public boolean hasAgeOfLastAppliedOp() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
-      if (hasUsedHeapMB()) {
-        hash = (37 * hash) + USED_HEAP_MB_FIELD_NUMBER;
-        hash = (53 * hash) + getUsedHeapMB();
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public long getAgeOfLastAppliedOp() {
+        return ageOfLastAppliedOp_;
       }
-      if (hasMaxHeapMB()) {
-        hash = (37 * hash) + MAX_HEAP_MB_FIELD_NUMBER;
-        hash = (53 * hash) + getMaxHeapMB();
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public Builder setAgeOfLastAppliedOp(long value) {
+        bitField0_ |= 0x00000001;
+        ageOfLastAppliedOp_ = value;
+        onChanged();
+        return this;
       }
-      if (getRegionLoadsCount() > 0) {
-        hash = (37 * hash) + REGION_LOADS_FIELD_NUMBER;
-        hash = (53 * hash) + getRegionLoadsList().hashCode();
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public Builder clearAgeOfLastAppliedOp() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        ageOfLastAppliedOp_ = 0L;
+        onChanged();
+        return this;
       }
-      if (getCoprocessorsCount() > 0) {
-        hash = (37 * hash) + COPROCESSORS_FIELD_NUMBER;
-        hash = (53 * hash) + getCoprocessorsList().hashCode();
+
+      // required uint64 timeStampsOfLastAppliedOp = 2;
+      private long timeStampsOfLastAppliedOp_ ;
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public boolean hasTimeStampsOfLastAppliedOp() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
       }
-      if (hasReportStartTime()) {
-        hash = (37 * hash) + REPORT_START_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getReportStartTime());
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public long getTimeStampsOfLastAppliedOp() {
+        return timeStampsOfLastAppliedOp_;
       }
-      if (hasReportEndTime()) {
-        hash = (37 * hash) + REPORT_END_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getReportEndTime());
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public Builder setTimeStampsOfLastAppliedOp(long value) {
+        bitField0_ |= 0x00000002;
+        timeStampsOfLastAppliedOp_ = value;
+        onChanged();
+        return this;
       }
-      if (hasInfoServerPort()) {
-        hash = (37 * hash) + INFO_SERVER_PORT_FIELD_NUMBER;
-        hash = (53 * hash) + getInfoServerPort();
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public Builder clearTimeStampsOfLastAppliedOp() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        timeStampsOfLastAppliedOp_ = 0L;
+        onChanged();
+        return this;
       }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
+      // @@protoc_insertion_point(builder_scope:ReplicationLoadSink)
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
+
+    static {
+      defaultInstance = new ReplicationLoadSink(true);
+      defaultInstance.initFields();
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
+
+    // @@protoc_insertion_point(class_scope:ReplicationLoadSink)
+  }
+
+  public interface ReplicationLoadSourceOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string peerID = 1;
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    boolean hasPeerID();
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    java.lang.String getPeerID();
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getPeerIDBytes();
+
+    // required uint64 ageOfLastShippedOp = 2;
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    boolean hasAgeOfLastShippedOp();
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    long getAgeOfLastShippedOp();
+
+    // required uint32 sizeOfLogQueue = 3;
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    boolean hasSizeOfLogQueue();
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    int getSizeOfLogQueue();
+
+    // required uint64 timeStampOfLastShippedOp = 4;
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    boolean hasTimeStampOfLastShippedOp();
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    long getTimeStampOfLastShippedOp();
+
+    // required uint64 replicationLag = 5;
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    boolean hasReplicationLag();
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    long getReplicationLag();
+  }
+  /**
+   * Protobuf type {@code ReplicationLoadSource}
+   */
+  public static final class ReplicationLoadSource extends
+      com.google.protobuf.GeneratedMessage
+      implements ReplicationLoadSourceOrBuilder {
+    // Use ReplicationLoadSource.newBuilder() to construct.
+    private ReplicationLoadSource(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
+    private ReplicationLoadSource(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ReplicationLoadSource defaultInstance;
+    public static ReplicationLoadSource getDefaultInstance() {
+      return defaultInstance;
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+
+    public ReplicationLoadSource getDefaultInstanceForType() {
+      return defaultInstance;
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
+    private ReplicationLoadSource(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              peerID_ = input.readBytes();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              ageOfLastShippedOp_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              sizeOfLogQueue_ = input.readUInt32();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              timeStampOfLastShippedOp_ = input.readUInt64();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              replicationLag_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_descriptor;
     }
 
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad prototype) {
-      return newBuilder().mergeFrom(prototype);
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.Builder.class);
     }
-    public Builder toBuilder() { return newBuilder(this); }
+
+    public static com.google.protobuf.Parser<ReplicationLoadSource> PARSER =
+        new com.google.protobuf.AbstractParser<ReplicationLoadSource>() {
+      public ReplicationLoadSource parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ReplicationLoadSource(input, extensionRegistry);
+      }
+    };
 
     @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
+    public com.google.protobuf.Parser<ReplicationLoadSource> getParserForType() {
+      return PARSER;
     }
+
+    private int bitField0_;
+    // required string peerID = 1;
+    public static final int PEERID_FIELD_NUMBER = 1;
+    private java.lang.Object peerID_;
     /**
-     * Protobuf type {@code ServerLoad}
+     * <code>required string peerID = 1;</code>
      */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoadOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_descriptor;
+    public boolean hasPeerID() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    public java.lang.String getPeerID() {
+      java.lang.Object ref = peerID_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          peerID_ = s;
+        }
+        return s;
       }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder.class);
+    }
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getPeerIDBytes() {
+      java.lang.Object ref = peerID_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        peerID_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
       }
+    }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
+    // required uint64 ageOfLastShippedOp = 2;
+    public static final int AGEOFLASTSHIPPEDOP_FIELD_NUMBER = 2;
+    private long ageOfLastShippedOp_;
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    public boolean hasAgeOfLastShippedOp() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    public long getAgeOfLastShippedOp() {
+      return ageOfLastShippedOp_;
+    }
 
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getRegionLoadsFieldBuilder();
-          getCoprocessorsFieldBuilder();
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
+    // required uint32 sizeOfLogQueue = 3;
+    public static final int SIZEOFLOGQUEUE_FIELD_NUMBER = 3;
+    private int sizeOfLogQueue_;
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    public boolean hasSizeOfLogQueue() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    public int getSizeOfLogQueue() {
+      return sizeOfLogQueue_;
+    }
 
-      public Builder clear() {
-        super.clear();
-        numberOfRequests_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        totalNumberOfRequests_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000002);
-        usedHeapMB_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        maxHeapMB_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        if (regionLoadsBuilder_ == null) {
-          regionLoads_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000010);
-        } else {
-          regionLoadsBuilder_.clear();
-        }
-        if (coprocessorsBuilder_ == null) {
-          coprocessors_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000020);
-        } else {
-          coprocessorsBuilder_.clear();
-        }
-        reportStartTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000040);
-        reportEndTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000080);
-        infoServerPort_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000100);
-        return this;
-      }
+    // required uint64 timeStampOfLastShippedOp = 4;
+    public static final int TIMESTAMPOFLASTSHIPPEDOP_FIELD_NUMBER = 4;
+    private long timeStampOfLastShippedOp_;
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    public boolean hasTimeStampOfLastShippedOp() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    public long getTimeStampOfLastShippedOp() {
+      return timeStampOfLastShippedOp_;
+    }
 
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
+    // required uint64 replicationLag = 5;
+    public static final int REPLICATIONLAG_FIELD_NUMBER = 5;
+    private long replicationLag_;
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    public boolean hasReplicationLag() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    public long getReplicationLag() {
+      return replicationLag_;
+    }
 
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_descriptor;
-      }
+    private void initFields() {
+      peerID_ = "";
+      ageOfLastShippedOp_ = 0L;
+      sizeOfLogQueue_ = 0;
+      timeStampOfLastShippedOp_ = 0L;
+      replicationLag_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
 
-      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.getDefaultInstance();
+      if (!hasPeerID()) {
+        memoizedIsInitialized = 0;
+        return false;
       }
-
-      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad build() {
-        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad result = buildPartial();
+      if (!hasAgeOfLastShippedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasSizeOfLogQueue()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTimeStampOfLastShippedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasReplicationLag()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getPeerIDBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, ageOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt32(3, sizeOfLogQueue_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeUInt64(4, timeStampOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeUInt64(5, replicationLag_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getPeerIDBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, ageOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(3, sizeOfLogQueue_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(4, timeStampOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(5, replicationLag_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource other = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource) obj;
+
+      boolean result = true;
+      result = result && (hasPeerID() == other.hasPeerID());
+      if (hasPeerID()) {
+        result = result && getPeerID()
+            .equals(other.getPeerID());
+      }
+      result = result && (hasAgeOfLastShippedOp() == other.hasAgeOfLastShippedOp());
+      if (hasAgeOfLastShippedOp()) {
+        result = result && (getAgeOfLastShippedOp()
+            == other.getAgeOfLastShippedOp());
+      }
+      result = result && (hasSizeOfLogQueue() == other.hasSizeOfLogQueue());
+      if (hasSizeOfLogQueue()) {
+        result = result && (getSizeOfLogQueue()
+            == other.getSizeOfLogQueue());
+      }
+      result = result && (hasTimeStampOfLastShippedOp() == other.hasTimeStampOfLastShippedOp());
+      if (hasTimeStampOfLastShippedOp()) {
+        result = result && (getTimeStampOfLastShippedOp()
+            == other.getTimeStampOfLastShippedOp());
+      }
+      result = result && (hasReplicationLag() == other.hasReplicationLag());
+      if (hasReplicationLag()) {
+        result = result && (getReplicationLag()
+            == other.getReplicationLag());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasPeerID()) {
+        hash = (37 * hash) + PEERID_FIELD_NUMBER;
+        hash = (53 * hash) + getPeerID().hashCode();
+      }
+      if (hasAgeOfLastShippedOp()) {
+        hash = (37 * hash) + AGEOFLASTSHIPPEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getAgeOfLastShippedOp());
+      }
+      if (hasSizeOfLogQueue()) {
+        hash = (37 * hash) + SIZEOFLOGQUEUE_FIELD_NUMBER;
+        hash = (53 * hash) + getSizeOfLogQueue();
+      }
+      if (hasTimeStampOfLastShippedOp()) {
+        hash = (37 * hash) + TIMESTAMPOFLASTSHIPPEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTimeStampOfLastShippedOp());
+      }
+      if (hasReplicationLag()) {
+        hash = (37 * hash) + REPLICATIONLAG_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getReplicationLag());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code ReplicationLoadSource}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSourceOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        peerID_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        ageOfLastShippedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        sizeOfLogQueue_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        timeStampOfLastShippedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        replicationLag_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource build() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad result = new org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad(this);
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource result = new org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.numberOfRequests_ = numberOfRequests_;
+        result.peerID_ = peerID_;
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.totalNumberOfRequests_ = totalNumberOfRequests_;
+        result.ageOfLastShippedOp_ = ageOfLastShippedOp_;
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000004;
         }
-        result.usedHeapMB_ = usedHeapMB_;
+        result.sizeOfLogQueue_ = sizeOfLogQueue_;
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000008;
         }
-        result.maxHeapMB_ = maxHeapMB_;
+        result.timeStampOfLastShippedOp_ = timeStampOfLastShippedOp_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.replicationLag_ = replicationLag_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.getDefaultInstance()) return this;
+        if (other.hasPeerID()) {
+          bitField0_ |= 0x00000001;
+          peerID_ = other.peerID_;
+          onChanged();
+        }
+        if (other.hasAgeOfLastShippedOp()) {
+          setAgeOfLastShippedOp(other.getAgeOfLastShippedOp());
+        }
+        if (other.hasSizeOfLogQueue()) {
+          setSizeOfLogQueue(other.getSizeOfLogQueue());
+        }
+        if (other.hasTimeStampOfLastShippedOp()) {
+          setTimeStampOfLastShippedOp(other.getTimeStampOfLastShippedOp());
+        }
+        if (other.hasReplicationLag()) {
+          setReplicationLag(other.getReplicationLag());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasPeerID()) {
+          
+          return false;
+        }
+        if (!hasAgeOfLastShippedOp()) {
+          
+          return false;
+        }
+        if (!hasSizeOfLogQueue()) {
+          
+          return false;
+        }
+        if (!hasTimeStampOfLastShippedOp()) {
+          
+          return false;
+        }
+        if (!hasReplicationLag()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string peerID = 1;
+      private java.lang.Object peerID_ = "";
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public boolean hasPeerID() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public java.lang.String getPeerID() {
+        java.lang.Object ref = peerID_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          peerID_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getPeerIDBytes() {
+        java.lang.Object ref = peerID_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          peerID_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public Builder setPeerID(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        peerID_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public Builder clearPeerID() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        peerID_ = getDefaultInstance().getPeerID();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public Builder setPeerIDBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        peerID_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 ageOfLastShippedOp = 2;
+      private long ageOfLastShippedOp_ ;
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public boolean hasAgeOfLastShippedOp() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public long getAgeOfLastShippedOp() {
+        return ageOfLastShippedOp_;
+      }
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public Builder setAgeOfLastShippedOp(long value) {
+        bitField0_ |= 0x00000002;
+        ageOfLastShippedOp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public Builder clearAgeOfLastShippedOp() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        ageOfLastShippedOp_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required uint32 sizeOfLogQueue = 3;
+      private int sizeOfLogQueue_ ;
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public boolean hasSizeOfLogQueue() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public int getSizeOfLogQueue() {
+        return sizeOfLogQueue_;
+      }
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public Builder setSizeOfLogQueue(int value) {
+        bitField0_ |= 0x00000004;
+        sizeOfLogQueue_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public Builder clearSizeOfLogQueue() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        sizeOfLogQueue_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 timeStampOfLastShippedOp = 4;
+      private long timeStampOfLastShippedOp_ ;
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public boolean hasTimeStampOfLastShippedOp() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public long getTimeStampOfLastShippedOp() {
+        return timeStampOfLastShippedOp_;
+      }
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public Builder setTimeStampOfLastShippedOp(long value) {
+        bitField0_ |= 0x00000008;
+        timeStampOfLastShippedOp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public Builder clearTimeStampOfLastShippedOp() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        timeStampOfLastShippedOp_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 replicationLag = 5;
+      private long replicationLag_ ;
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public boolean hasReplicationLag() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public long getReplicationLag() {
+        return replicationLag_;
+      }
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public Builder setReplicationLag(long value) {
+        bitField0_ |= 0x00000010;
+        replicationLag_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public Builder clearReplicationLag() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        replicationLag_ = 0

<TRUNCATED>

[2/9] hbase git commit: HBASE-9531 a command line (hbase shell) interface to retreive the replication metrics and show replication lag

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c0e26c3a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
index 5bc44ff..95a6e42 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
@@ -4348,273 +4348,48 @@ public final class ClusterStatusProtos {
     // @@protoc_insertion_point(class_scope:RegionLoad)
   }
 
-  public interface ServerLoadOrBuilder
+  public interface ReplicationLoadSinkOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // optional uint32 number_of_requests = 1;
-    /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
-     */
-    boolean hasNumberOfRequests();
-    /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
-     */
-    int getNumberOfRequests();
-
-    // optional uint32 total_number_of_requests = 2;
-    /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
-     */
-    boolean hasTotalNumberOfRequests();
-    /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
-     */
-    int getTotalNumberOfRequests();
-
-    // optional uint32 used_heap_MB = 3;
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    boolean hasUsedHeapMB();
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    int getUsedHeapMB();
-
-    // optional uint32 max_heap_MB = 4;
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    boolean hasMaxHeapMB();
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    int getMaxHeapMB();
-
-    // repeated .RegionLoad region_loads = 5;
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad> 
-        getRegionLoadsList();
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad getRegionLoads(int index);
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    int getRegionLoadsCount();
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder> 
-        getRegionLoadsOrBuilderList();
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder getRegionLoadsOrBuilder(
-        int index);
-
-    // repeated .Coprocessor coprocessors = 6;
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor> 
-        getCoprocessorsList();
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor getCoprocessors(int index);
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    int getCoprocessorsCount();
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder> 
-        getCoprocessorsOrBuilderList();
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder getCoprocessorsOrBuilder(
-        int index);
-
-    // optional uint64 report_start_time = 7;
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    boolean hasReportStartTime();
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    long getReportStartTime();
-
-    // optional uint64 report_end_time = 8;
+    // required uint64 ageOfLastAppliedOp = 1;
     /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    boolean hasReportEndTime();
+    boolean hasAgeOfLastAppliedOp();
     /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    long getReportEndTime();
+    long getAgeOfLastAppliedOp();
 
-    // optional uint32 info_server_port = 9;
+    // required uint64 timeStampsOfLastAppliedOp = 2;
     /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    boolean hasInfoServerPort();
+    boolean hasTimeStampsOfLastAppliedOp();
     /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    int getInfoServerPort();
+    long getTimeStampsOfLastAppliedOp();
   }
   /**
-   * Protobuf type {@code ServerLoad}
+   * Protobuf type {@code ReplicationLoadSink}
    */
-  public static final class ServerLoad extends
+  public static final class ReplicationLoadSink extends
       com.google.protobuf.GeneratedMessage
-      implements ServerLoadOrBuilder {
-    // Use ServerLoad.newBuilder() to construct.
-    private ServerLoad(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements ReplicationLoadSinkOrBuilder {
+    // Use ReplicationLoadSink.newBuilder() to construct.
+    private ReplicationLoadSink(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private ServerLoad(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private ReplicationLoadSink(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final ServerLoad defaultInstance;
-    public static ServerLoad getDefaultInstance() {
+    private static final ReplicationLoadSink defaultInstance;
+    public static ReplicationLoadSink getDefaultInstance() {
       return defaultInstance;
     }
 
-    public ServerLoad getDefaultInstanceForType() {
+    public ReplicationLoadSink getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -4624,7 +4399,7 @@ public final class ClusterStatusProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private ServerLoad(
+    private ReplicationLoadSink(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -4649,53 +4424,12 @@ public final class ClusterStatusProtos {
             }
             case 8: {
               bitField0_ |= 0x00000001;
-              numberOfRequests_ = input.readUInt32();
+              ageOfLastAppliedOp_ = input.readUInt64();
               break;
             }
             case 16: {
               bitField0_ |= 0x00000002;
-              totalNumberOfRequests_ = input.readUInt32();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              usedHeapMB_ = input.readUInt32();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              maxHeapMB_ = input.readUInt32();
-              break;
-            }
-            case 42: {
-              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-                regionLoads_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad>();
-                mutable_bitField0_ |= 0x00000010;
-              }
-              regionLoads_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad.PARSER, extensionRegistry));
-              break;
-            }
-            case 50: {
-              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-                coprocessors_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor>();
-                mutable_bitField0_ |= 0x00000020;
-              }
-              coprocessors_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor.PARSER, extensionRegistry));
-              break;
-            }
-            case 56: {
-              bitField0_ |= 0x00000010;
-              reportStartTime_ = input.readUInt64();
-              break;
-            }
-            case 64: {
-              bitField0_ |= 0x00000020;
-              reportEndTime_ = input.readUInt64();
-              break;
-            }
-            case 72: {
-              bitField0_ |= 0x00000040;
-              infoServerPort_ = input.readUInt32();
+              timeStampsOfLastAppliedOp_ = input.readUInt64();
               break;
             }
           }
@@ -4706,1957 +4440,4480 @@ public final class ClusterStatusProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-          regionLoads_ = java.util.Collections.unmodifiableList(regionLoads_);
-        }
-        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-          coprocessors_ = java.util.Collections.unmodifiableList(coprocessors_);
-        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<ServerLoad> PARSER =
-        new com.google.protobuf.AbstractParser<ServerLoad>() {
-      public ServerLoad parsePartialFrom(
+    public static com.google.protobuf.Parser<ReplicationLoadSink> PARSER =
+        new com.google.protobuf.AbstractParser<ReplicationLoadSink>() {
+      public ReplicationLoadSink parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new ServerLoad(input, extensionRegistry);
+        return new ReplicationLoadSink(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<ServerLoad> getParserForType() {
+    public com.google.protobuf.Parser<ReplicationLoadSink> getParserForType() {
       return PARSER;
     }
 
     private int bitField0_;
-    // optional uint32 number_of_requests = 1;
-    public static final int NUMBER_OF_REQUESTS_FIELD_NUMBER = 1;
-    private int numberOfRequests_;
+    // required uint64 ageOfLastAppliedOp = 1;
+    public static final int AGEOFLASTAPPLIEDOP_FIELD_NUMBER = 1;
+    private long ageOfLastAppliedOp_;
     /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    public boolean hasNumberOfRequests() {
+    public boolean hasAgeOfLastAppliedOp() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>optional uint32 number_of_requests = 1;</code>
-     *
-     * <pre>
-     ** Number of requests since last report. 
-     * </pre>
+     * <code>required uint64 ageOfLastAppliedOp = 1;</code>
      */
-    public int getNumberOfRequests() {
-      return numberOfRequests_;
+    public long getAgeOfLastAppliedOp() {
+      return ageOfLastAppliedOp_;
     }
 
-    // optional uint32 total_number_of_requests = 2;
-    public static final int TOTAL_NUMBER_OF_REQUESTS_FIELD_NUMBER = 2;
-    private int totalNumberOfRequests_;
+    // required uint64 timeStampsOfLastAppliedOp = 2;
+    public static final int TIMESTAMPSOFLASTAPPLIEDOP_FIELD_NUMBER = 2;
+    private long timeStampsOfLastAppliedOp_;
     /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    public boolean hasTotalNumberOfRequests() {
+    public boolean hasTimeStampsOfLastAppliedOp() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional uint32 total_number_of_requests = 2;</code>
-     *
-     * <pre>
-     ** Total Number of requests from the start of the region server. 
-     * </pre>
+     * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
      */
-    public int getTotalNumberOfRequests() {
-      return totalNumberOfRequests_;
+    public long getTimeStampsOfLastAppliedOp() {
+      return timeStampsOfLastAppliedOp_;
     }
 
-    // optional uint32 used_heap_MB = 3;
-    public static final int USED_HEAP_MB_FIELD_NUMBER = 3;
-    private int usedHeapMB_;
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    public boolean hasUsedHeapMB() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
+    private void initFields() {
+      ageOfLastAppliedOp_ = 0L;
+      timeStampsOfLastAppliedOp_ = 0L;
     }
-    /**
-     * <code>optional uint32 used_heap_MB = 3;</code>
-     *
-     * <pre>
-     ** the amount of used heap, in MB. 
-     * </pre>
-     */
-    public int getUsedHeapMB() {
-      return usedHeapMB_;
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasAgeOfLastAppliedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTimeStampsOfLastAppliedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
     }
 
-    // optional uint32 max_heap_MB = 4;
-    public static final int MAX_HEAP_MB_FIELD_NUMBER = 4;
-    private int maxHeapMB_;
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    public boolean hasMaxHeapMB() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt64(1, ageOfLastAppliedOp_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, timeStampsOfLastAppliedOp_);
+      }
+      getUnknownFields().writeTo(output);
     }
-    /**
-     * <code>optional uint32 max_heap_MB = 4;</code>
-     *
-     * <pre>
-     ** the maximum allowable size of the heap, in MB. 
-     * </pre>
-     */
-    public int getMaxHeapMB() {
-      return maxHeapMB_;
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(1, ageOfLastAppliedOp_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, timeStampsOfLastAppliedOp_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
     }
 
-    // repeated .RegionLoad region_loads = 5;
-    public static final int REGION_LOADS_FIELD_NUMBER = 5;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad> regionLoads_;
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad> getRegionLoadsList() {
-      return regionLoads_;
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder> 
-        getRegionLoadsOrBuilderList() {
-      return regionLoads_;
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink other = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink) obj;
+
+      boolean result = true;
+      result = result && (hasAgeOfLastAppliedOp() == other.hasAgeOfLastAppliedOp());
+      if (hasAgeOfLastAppliedOp()) {
+        result = result && (getAgeOfLastAppliedOp()
+            == other.getAgeOfLastAppliedOp());
+      }
+      result = result && (hasTimeStampsOfLastAppliedOp() == other.hasTimeStampsOfLastAppliedOp());
+      if (hasTimeStampsOfLastAppliedOp()) {
+        result = result && (getTimeStampsOfLastAppliedOp()
+            == other.getTimeStampsOfLastAppliedOp());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public int getRegionLoadsCount() {
-      return regionLoads_.size();
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasAgeOfLastAppliedOp()) {
+        hash = (37 * hash) + AGEOFLASTAPPLIEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getAgeOfLastAppliedOp());
+      }
+      if (hasTimeStampsOfLastAppliedOp()) {
+        hash = (37 * hash) + TIMESTAMPSOFLASTAPPLIEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTimeStampsOfLastAppliedOp());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad getRegionLoads(int index) {
-      return regionLoads_.get(index);
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
     }
-    /**
-     * <code>repeated .RegionLoad region_loads = 5;</code>
-     *
-     * <pre>
-     ** Information on the load of individual regions. 
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder getRegionLoadsOrBuilder(
-        int index) {
-      return regionLoads_.get(index);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
     }
-
-    // repeated .Coprocessor coprocessors = 6;
-    public static final int COPROCESSORS_FIELD_NUMBER = 6;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor> coprocessors_;
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor> getCoprocessorsList() {
-      return coprocessors_;
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder> 
-        getCoprocessorsOrBuilderList() {
-      return coprocessors_;
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public int getCoprocessorsCount() {
-      return coprocessors_.size();
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor getCoprocessors(int index) {
-      return coprocessors_.get(index);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
     }
-    /**
-     * <code>repeated .Coprocessor coprocessors = 6;</code>
-     *
-     * <pre>
-     **
-     * Regionserver-level coprocessors, e.g., WALObserver implementations.
-     * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-     * objects.
-     * </pre>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CoprocessorOrBuilder getCoprocessorsOrBuilder(
-        int index) {
-      return coprocessors_.get(index);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
     }
-
-    // optional uint64 report_start_time = 7;
-    public static final int REPORT_START_TIME_FIELD_NUMBER = 7;
-    private long reportStartTime_;
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public boolean hasReportStartTime() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    /**
-     * <code>optional uint64 report_start_time = 7;</code>
-     *
-     * <pre>
-     **
-     * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public long getReportStartTime() {
-      return reportStartTime_;
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
     }
-
-    // optional uint64 report_end_time = 8;
-    public static final int REPORT_END_TIME_FIELD_NUMBER = 8;
-    private long reportEndTime_;
-    /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public boolean hasReportEndTime() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
     }
-    /**
-     * <code>optional uint64 report_end_time = 8;</code>
-     *
-     * <pre>
-     **
-     * Time when report was generated.
-     * time is measured as the difference, measured in milliseconds, between the current time
-     * and midnight, January 1, 1970 UTC.
-     * </pre>
-     */
-    public long getReportEndTime() {
-      return reportEndTime_;
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink prototype) {
+      return newBuilder().mergeFrom(prototype);
     }
+    public Builder toBuilder() { return newBuilder(this); }
 
-    // optional uint32 info_server_port = 9;
-    public static final int INFO_SERVER_PORT_FIELD_NUMBER = 9;
-    private int infoServerPort_;
-    /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
-     */
-    public boolean hasInfoServerPort() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
     }
     /**
-     * <code>optional uint32 info_server_port = 9;</code>
-     *
-     * <pre>
-     **
-     * The port number that this region server is hosing an info server on.
-     * </pre>
+     * Protobuf type {@code ReplicationLoadSink}
      */
-    public int getInfoServerPort() {
-      return infoServerPort_;
-    }
-
-    private void initFields() {
-      numberOfRequests_ = 0;
-      totalNumberOfRequests_ = 0;
-      usedHeapMB_ = 0;
-      maxHeapMB_ = 0;
-      regionLoads_ = java.util.Collections.emptyList();
-      coprocessors_ = java.util.Collections.emptyList();
-      reportStartTime_ = 0L;
-      reportEndTime_ = 0L;
-      infoServerPort_ = 0;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      for (int i = 0; i < getRegionLoadsCount(); i++) {
-        if (!getRegionLoads(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSinkOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_descriptor;
       }
-      for (int i = 0; i < getCoprocessorsCount(); i++) {
-        if (!getCoprocessors(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.Builder.class);
       }
-      memoizedIsInitialized = 1;
-      return true;
-    }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeUInt32(1, numberOfRequests_);
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeUInt32(2, totalNumberOfRequests_);
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeUInt32(3, usedHeapMB_);
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
       }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeUInt32(4, maxHeapMB_);
+      private static Builder create() {
+        return new Builder();
       }
-      for (int i = 0; i < regionLoads_.size(); i++) {
-        output.writeMessage(5, regionLoads_.get(i));
+
+      public Builder clear() {
+        super.clear();
+        ageOfLastAppliedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        timeStampsOfLastAppliedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
       }
-      for (int i = 0; i < coprocessors_.size(); i++) {
-        output.writeMessage(6, coprocessors_.get(i));
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
       }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeUInt64(7, reportStartTime_);
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSink_descriptor;
       }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeUInt64(8, reportEndTime_);
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.getDefaultInstance();
       }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeUInt32(9, infoServerPort_);
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
 
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(1, numberOfRequests_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(2, totalNumberOfRequests_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(3, usedHeapMB_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(4, maxHeapMB_);
-      }
-      for (int i = 0; i < regionLoads_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(5, regionLoads_.get(i));
-      }
-      for (int i = 0; i < coprocessors_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(6, coprocessors_.get(i));
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(7, reportStartTime_);
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(8, reportEndTime_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(9, infoServerPort_);
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink build() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
       }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
 
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad)) {
-        return super.equals(obj);
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink result = new org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.ageOfLastAppliedOp_ = ageOfLastAppliedOp_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.timeStampsOfLastAppliedOp_ = timeStampsOfLastAppliedOp_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
       }
-      org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad other = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad) obj;
 
-      boolean result = true;
-      result = result && (hasNumberOfRequests() == other.hasNumberOfRequests());
-      if (hasNumberOfRequests()) {
-        result = result && (getNumberOfRequests()
-            == other.getNumberOfRequests());
-      }
-      result = result && (hasTotalNumberOfRequests() == other.hasTotalNumberOfRequests());
-      if (hasTotalNumberOfRequests()) {
-        result = result && (getTotalNumberOfRequests()
-            == other.getTotalNumberOfRequests());
-      }
-      result = result && (hasUsedHeapMB() == other.hasUsedHeapMB());
-      if (hasUsedHeapMB()) {
-        result = result && (getUsedHeapMB()
-            == other.getUsedHeapMB());
-      }
-      result = result && (hasMaxHeapMB() == other.hasMaxHeapMB());
-      if (hasMaxHeapMB()) {
-        result = result && (getMaxHeapMB()
-            == other.getMaxHeapMB());
-      }
-      result = result && getRegionLoadsList()
-          .equals(other.getRegionLoadsList());
-      result = result && getCoprocessorsList()
-          .equals(other.getCoprocessorsList());
-      result = result && (hasReportStartTime() == other.hasReportStartTime());
-      if (hasReportStartTime()) {
-        result = result && (getReportStartTime()
-            == other.getReportStartTime());
-      }
-      result = result && (hasReportEndTime() == other.hasReportEndTime());
-      if (hasReportEndTime()) {
-        result = result && (getReportEndTime()
-            == other.getReportEndTime());
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
       }
-      result = result && (hasInfoServerPort() == other.hasInfoServerPort());
-      if (hasInfoServerPort()) {
-        result = result && (getInfoServerPort()
-            == other.getInfoServerPort());
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.getDefaultInstance()) return this;
+        if (other.hasAgeOfLastAppliedOp()) {
+          setAgeOfLastAppliedOp(other.getAgeOfLastAppliedOp());
+        }
+        if (other.hasTimeStampsOfLastAppliedOp()) {
+          setTimeStampsOfLastAppliedOp(other.getTimeStampsOfLastAppliedOp());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
       }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
 
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
+      public final boolean isInitialized() {
+        if (!hasAgeOfLastAppliedOp()) {
+          
+          return false;
+        }
+        if (!hasTimeStampsOfLastAppliedOp()) {
+          
+          return false;
+        }
+        return true;
       }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasNumberOfRequests()) {
-        hash = (37 * hash) + NUMBER_OF_REQUESTS_FIELD_NUMBER;
-        hash = (53 * hash) + getNumberOfRequests();
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
       }
-      if (hasTotalNumberOfRequests()) {
-        hash = (37 * hash) + TOTAL_NUMBER_OF_REQUESTS_FIELD_NUMBER;
-        hash = (53 * hash) + getTotalNumberOfRequests();
+      private int bitField0_;
+
+      // required uint64 ageOfLastAppliedOp = 1;
+      private long ageOfLastAppliedOp_ ;
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public boolean hasAgeOfLastAppliedOp() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
-      if (hasUsedHeapMB()) {
-        hash = (37 * hash) + USED_HEAP_MB_FIELD_NUMBER;
-        hash = (53 * hash) + getUsedHeapMB();
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public long getAgeOfLastAppliedOp() {
+        return ageOfLastAppliedOp_;
       }
-      if (hasMaxHeapMB()) {
-        hash = (37 * hash) + MAX_HEAP_MB_FIELD_NUMBER;
-        hash = (53 * hash) + getMaxHeapMB();
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public Builder setAgeOfLastAppliedOp(long value) {
+        bitField0_ |= 0x00000001;
+        ageOfLastAppliedOp_ = value;
+        onChanged();
+        return this;
       }
-      if (getRegionLoadsCount() > 0) {
-        hash = (37 * hash) + REGION_LOADS_FIELD_NUMBER;
-        hash = (53 * hash) + getRegionLoadsList().hashCode();
+      /**
+       * <code>required uint64 ageOfLastAppliedOp = 1;</code>
+       */
+      public Builder clearAgeOfLastAppliedOp() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        ageOfLastAppliedOp_ = 0L;
+        onChanged();
+        return this;
       }
-      if (getCoprocessorsCount() > 0) {
-        hash = (37 * hash) + COPROCESSORS_FIELD_NUMBER;
-        hash = (53 * hash) + getCoprocessorsList().hashCode();
+
+      // required uint64 timeStampsOfLastAppliedOp = 2;
+      private long timeStampsOfLastAppliedOp_ ;
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public boolean hasTimeStampsOfLastAppliedOp() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
       }
-      if (hasReportStartTime()) {
-        hash = (37 * hash) + REPORT_START_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getReportStartTime());
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public long getTimeStampsOfLastAppliedOp() {
+        return timeStampsOfLastAppliedOp_;
       }
-      if (hasReportEndTime()) {
-        hash = (37 * hash) + REPORT_END_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getReportEndTime());
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public Builder setTimeStampsOfLastAppliedOp(long value) {
+        bitField0_ |= 0x00000002;
+        timeStampsOfLastAppliedOp_ = value;
+        onChanged();
+        return this;
       }
-      if (hasInfoServerPort()) {
-        hash = (37 * hash) + INFO_SERVER_PORT_FIELD_NUMBER;
-        hash = (53 * hash) + getInfoServerPort();
+      /**
+       * <code>required uint64 timeStampsOfLastAppliedOp = 2;</code>
+       */
+      public Builder clearTimeStampsOfLastAppliedOp() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        timeStampsOfLastAppliedOp_ = 0L;
+        onChanged();
+        return this;
       }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
+      // @@protoc_insertion_point(builder_scope:ReplicationLoadSink)
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
+
+    static {
+      defaultInstance = new ReplicationLoadSink(true);
+      defaultInstance.initFields();
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
+
+    // @@protoc_insertion_point(class_scope:ReplicationLoadSink)
+  }
+
+  public interface ReplicationLoadSourceOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string peerID = 1;
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    boolean hasPeerID();
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    java.lang.String getPeerID();
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getPeerIDBytes();
+
+    // required uint64 ageOfLastShippedOp = 2;
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    boolean hasAgeOfLastShippedOp();
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    long getAgeOfLastShippedOp();
+
+    // required uint32 sizeOfLogQueue = 3;
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    boolean hasSizeOfLogQueue();
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    int getSizeOfLogQueue();
+
+    // required uint64 timeStampOfLastShippedOp = 4;
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    boolean hasTimeStampOfLastShippedOp();
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    long getTimeStampOfLastShippedOp();
+
+    // required uint64 replicationLag = 5;
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    boolean hasReplicationLag();
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    long getReplicationLag();
+  }
+  /**
+   * Protobuf type {@code ReplicationLoadSource}
+   */
+  public static final class ReplicationLoadSource extends
+      com.google.protobuf.GeneratedMessage
+      implements ReplicationLoadSourceOrBuilder {
+    // Use ReplicationLoadSource.newBuilder() to construct.
+    private ReplicationLoadSource(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
+    private ReplicationLoadSource(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ReplicationLoadSource defaultInstance;
+    public static ReplicationLoadSource getDefaultInstance() {
+      return defaultInstance;
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+
+    public ReplicationLoadSource getDefaultInstanceForType() {
+      return defaultInstance;
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
+    private ReplicationLoadSource(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              peerID_ = input.readBytes();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              ageOfLastShippedOp_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              sizeOfLogQueue_ = input.readUInt32();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              timeStampOfLastShippedOp_ = input.readUInt64();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              replicationLag_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_descriptor;
     }
 
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad prototype) {
-      return newBuilder().mergeFrom(prototype);
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.Builder.class);
     }
-    public Builder toBuilder() { return newBuilder(this); }
+
+    public static com.google.protobuf.Parser<ReplicationLoadSource> PARSER =
+        new com.google.protobuf.AbstractParser<ReplicationLoadSource>() {
+      public ReplicationLoadSource parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ReplicationLoadSource(input, extensionRegistry);
+      }
+    };
 
     @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
+    public com.google.protobuf.Parser<ReplicationLoadSource> getParserForType() {
+      return PARSER;
     }
+
+    private int bitField0_;
+    // required string peerID = 1;
+    public static final int PEERID_FIELD_NUMBER = 1;
+    private java.lang.Object peerID_;
     /**
-     * Protobuf type {@code ServerLoad}
+     * <code>required string peerID = 1;</code>
      */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoadOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_descriptor;
+    public boolean hasPeerID() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    public java.lang.String getPeerID() {
+      java.lang.Object ref = peerID_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          peerID_ = s;
+        }
+        return s;
       }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder.class);
+    }
+    /**
+     * <code>required string peerID = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getPeerIDBytes() {
+      java.lang.Object ref = peerID_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        peerID_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
       }
+    }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
+    // required uint64 ageOfLastShippedOp = 2;
+    public static final int AGEOFLASTSHIPPEDOP_FIELD_NUMBER = 2;
+    private long ageOfLastShippedOp_;
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    public boolean hasAgeOfLastShippedOp() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required uint64 ageOfLastShippedOp = 2;</code>
+     */
+    public long getAgeOfLastShippedOp() {
+      return ageOfLastShippedOp_;
+    }
 
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getRegionLoadsFieldBuilder();
-          getCoprocessorsFieldBuilder();
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
+    // required uint32 sizeOfLogQueue = 3;
+    public static final int SIZEOFLOGQUEUE_FIELD_NUMBER = 3;
+    private int sizeOfLogQueue_;
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    public boolean hasSizeOfLogQueue() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required uint32 sizeOfLogQueue = 3;</code>
+     */
+    public int getSizeOfLogQueue() {
+      return sizeOfLogQueue_;
+    }
 
-      public Builder clear() {
-        super.clear();
-        numberOfRequests_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        totalNumberOfRequests_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000002);
-        usedHeapMB_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        maxHeapMB_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        if (regionLoadsBuilder_ == null) {
-          regionLoads_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000010);
-        } else {
-          regionLoadsBuilder_.clear();
-        }
-        if (coprocessorsBuilder_ == null) {
-          coprocessors_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000020);
-        } else {
-          coprocessorsBuilder_.clear();
-        }
-        reportStartTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000040);
-        reportEndTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000080);
-        infoServerPort_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000100);
-        return this;
-      }
+    // required uint64 timeStampOfLastShippedOp = 4;
+    public static final int TIMESTAMPOFLASTSHIPPEDOP_FIELD_NUMBER = 4;
+    private long timeStampOfLastShippedOp_;
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    public boolean hasTimeStampOfLastShippedOp() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+     */
+    public long getTimeStampOfLastShippedOp() {
+      return timeStampOfLastShippedOp_;
+    }
 
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
+    // required uint64 replicationLag = 5;
+    public static final int REPLICATIONLAG_FIELD_NUMBER = 5;
+    private long replicationLag_;
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    public boolean hasReplicationLag() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>required uint64 replicationLag = 5;</code>
+     */
+    public long getReplicationLag() {
+      return replicationLag_;
+    }
 
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ServerLoad_descriptor;
-      }
+    private void initFields() {
+      peerID_ = "";
+      ageOfLastShippedOp_ = 0L;
+      sizeOfLogQueue_ = 0;
+      timeStampOfLastShippedOp_ = 0L;
+      replicationLag_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
 
-      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad.getDefaultInstance();
+      if (!hasPeerID()) {
+        memoizedIsInitialized = 0;
+        return false;
       }
-
-      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad build() {
-        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad result = buildPartial();
+      if (!hasAgeOfLastShippedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasSizeOfLogQueue()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTimeStampOfLastShippedOp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasReplicationLag()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getPeerIDBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, ageOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt32(3, sizeOfLogQueue_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeUInt64(4, timeStampOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeUInt64(5, replicationLag_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getPeerIDBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, ageOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(3, sizeOfLogQueue_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(4, timeStampOfLastShippedOp_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(5, replicationLag_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource other = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource) obj;
+
+      boolean result = true;
+      result = result && (hasPeerID() == other.hasPeerID());
+      if (hasPeerID()) {
+        result = result && getPeerID()
+            .equals(other.getPeerID());
+      }
+      result = result && (hasAgeOfLastShippedOp() == other.hasAgeOfLastShippedOp());
+      if (hasAgeOfLastShippedOp()) {
+        result = result && (getAgeOfLastShippedOp()
+            == other.getAgeOfLastShippedOp());
+      }
+      result = result && (hasSizeOfLogQueue() == other.hasSizeOfLogQueue());
+      if (hasSizeOfLogQueue()) {
+        result = result && (getSizeOfLogQueue()
+            == other.getSizeOfLogQueue());
+      }
+      result = result && (hasTimeStampOfLastShippedOp() == other.hasTimeStampOfLastShippedOp());
+      if (hasTimeStampOfLastShippedOp()) {
+        result = result && (getTimeStampOfLastShippedOp()
+            == other.getTimeStampOfLastShippedOp());
+      }
+      result = result && (hasReplicationLag() == other.hasReplicationLag());
+      if (hasReplicationLag()) {
+        result = result && (getReplicationLag()
+            == other.getReplicationLag());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasPeerID()) {
+        hash = (37 * hash) + PEERID_FIELD_NUMBER;
+        hash = (53 * hash) + getPeerID().hashCode();
+      }
+      if (hasAgeOfLastShippedOp()) {
+        hash = (37 * hash) + AGEOFLASTSHIPPEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getAgeOfLastShippedOp());
+      }
+      if (hasSizeOfLogQueue()) {
+        hash = (37 * hash) + SIZEOFLOGQUEUE_FIELD_NUMBER;
+        hash = (53 * hash) + getSizeOfLogQueue();
+      }
+      if (hasTimeStampOfLastShippedOp()) {
+        hash = (37 * hash) + TIMESTAMPOFLASTSHIPPEDOP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTimeStampOfLastShippedOp());
+      }
+      if (hasReplicationLag()) {
+        hash = (37 * hash) + REPLICATIONLAG_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getReplicationLag());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code ReplicationLoadSource}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSourceOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.class, org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        peerID_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        ageOfLastShippedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        sizeOfLogQueue_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        timeStampOfLastShippedOp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        replicationLag_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.internal_static_ReplicationLoadSource_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource build() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad result = new org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ServerLoad(this);
+      public org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource result = new org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.numberOfRequests_ = numberOfRequests_;
+        result.peerID_ = peerID_;
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.totalNumberOfRequests_ = totalNumberOfRequests_;
+        result.ageOfLastShippedOp_ = ageOfLastShippedOp_;
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000004;
         }
-        result.usedHeapMB_ = usedHeapMB_;
+        result.sizeOfLogQueue_ = sizeOfLogQueue_;
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000008;
         }
-        result.maxHeapMB_ = maxHeapMB_;
+        result.timeStampOfLastShippedOp_ = timeStampOfLastShippedOp_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.replicationLag_ = replicationLag_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.getDefaultInstance()) return this;
+        if (other.hasPeerID()) {
+          bitField0_ |= 0x00000001;
+          peerID_ = other.peerID_;
+          onChanged();
+        }
+        if (other.hasAgeOfLastShippedOp()) {
+          setAgeOfLastShippedOp(other.getAgeOfLastShippedOp());
+        }
+        if (other.hasSizeOfLogQueue()) {
+          setSizeOfLogQueue(other.getSizeOfLogQueue());
+        }
+        if (other.hasTimeStampOfLastShippedOp()) {
+          setTimeStampOfLastShippedOp(other.getTimeStampOfLastShippedOp());
+        }
+        if (other.hasReplicationLag()) {
+          setReplicationLag(other.getReplicationLag());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasPeerID()) {
+          
+          return false;
+        }
+        if (!hasAgeOfLastShippedOp()) {
+          
+          return false;
+        }
+        if (!hasSizeOfLogQueue()) {
+          
+          return false;
+        }
+        if (!hasTimeStampOfLastShippedOp()) {
+          
+          return false;
+        }
+        if (!hasReplicationLag()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string peerID = 1;
+      private java.lang.Object peerID_ = "";
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public boolean hasPeerID() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public java.lang.String getPeerID() {
+        java.lang.Object ref = peerID_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          peerID_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getPeerIDBytes() {
+        java.lang.Object ref = peerID_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          peerID_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public Builder setPeerID(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        peerID_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public Builder clearPeerID() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        peerID_ = getDefaultInstance().getPeerID();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string peerID = 1;</code>
+       */
+      public Builder setPeerIDBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        peerID_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 ageOfLastShippedOp = 2;
+      private long ageOfLastShippedOp_ ;
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public boolean hasAgeOfLastShippedOp() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public long getAgeOfLastShippedOp() {
+        return ageOfLastShippedOp_;
+      }
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public Builder setAgeOfLastShippedOp(long value) {
+        bitField0_ |= 0x00000002;
+        ageOfLastShippedOp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 ageOfLastShippedOp = 2;</code>
+       */
+      public Builder clearAgeOfLastShippedOp() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        ageOfLastShippedOp_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required uint32 sizeOfLogQueue = 3;
+      private int sizeOfLogQueue_ ;
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public boolean hasSizeOfLogQueue() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public int getSizeOfLogQueue() {
+        return sizeOfLogQueue_;
+      }
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public Builder setSizeOfLogQueue(int value) {
+        bitField0_ |= 0x00000004;
+        sizeOfLogQueue_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint32 sizeOfLogQueue = 3;</code>
+       */
+      public Builder clearSizeOfLogQueue() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        sizeOfLogQueue_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 timeStampOfLastShippedOp = 4;
+      private long timeStampOfLastShippedOp_ ;
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public boolean hasTimeStampOfLastShippedOp() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public long getTimeStampOfLastShippedOp() {
+        return timeStampOfLastShippedOp_;
+      }
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public Builder setTimeStampOfLastShippedOp(long value) {
+        bitField0_ |= 0x00000008;
+        timeStampOfLastShippedOp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 timeStampOfLastShippedOp = 4;</code>
+       */
+      public Builder clearTimeStampOfLastShippedOp() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        timeStampOfLastShippedOp_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 replicationLag = 5;
+      private long replicationLag_ ;
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public boolean hasReplicationLag() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public long getReplicationLag() {
+        return replicationLag_;
+      }
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public Builder setReplicationLag(long value) {
+        bitField0_ |= 0x00000010;
+        replicationLag_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 replicationLag = 5;</code>
+       */
+      public Builder clearReplicationLag() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        replicationLag_ = 0

<TRUNCATED>