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 2018/10/02 01:51:26 UTC
[5/5] hbase git commit: HBASE-18549 Add metrics for failed
replication queue recovery
HBASE-18549 Add metrics for failed replication queue recovery
Signed-off-by: Andrew Purtell <ap...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b22095ee
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b22095ee
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b22095ee
Branch: refs/heads/branch-1.4
Commit: b22095eea1590d6a59ca1430ffaaf1c1bff3b0c0
Parents: 964a1d6
Author: Xu Cang <xc...@salesforce.com>
Authored: Wed Aug 29 16:39:57 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon Oct 1 18:43:01 2018 -0700
----------------------------------------------------------------------
.../hadoop/hbase/replication/ReplicationQueuesZKImpl.java | 5 ++++-
.../regionserver/MetricsReplicationSourceSource.java | 3 +++
.../regionserver/MetricsReplicationGlobalSourceSource.java | 8 +++++++-
.../regionserver/MetricsReplicationSourceSourceImpl.java | 3 +++
.../hbase/replication/regionserver/MetricsSource.java | 5 +++++
.../replication/regionserver/ReplicationSourceManager.java | 3 +++
.../hadoop/hbase/replication/TestReplicationEndpoint.java | 9 +++++++--
7 files changed, 32 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/b22095ee/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index dda9adf..f88ae53 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -499,7 +499,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
}
return new Pair<>(newCluster, logQueue);
} catch (KeeperException e) {
- LOG.warn("Got exception in copyQueueFromLockedRS: ", e);
+ LOG.warn("Got exception in copyQueueFromLockedRS: "+
+ " Possible problem: check if znode size exceeds jute.maxBuffer value. "
+ + "If so, increase it for both client and server side." ,e);
+
} catch (InterruptedException e) {
LOG.warn(e);
Thread.currentThread().interrupt();
http://git-wip-us.apache.org/repos/asf/hbase/blob/b22095ee/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 9075a68..25d72af 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
@@ -49,6 +49,7 @@ public interface MetricsReplicationSourceSource extends BaseSource {
public static final String SOURCE_REPEATED_LOG_FILE_BYTES = "source.repeatedLogFileBytes";
public static final String SOURCE_COMPLETED_LOGS = "source.completedLogs";
public static final String SOURCE_COMPLETED_RECOVERY_QUEUES = "source.completedRecoverQueues";
+ public static final String SOURCE_FAILED_RECOVERY_QUEUES = "source.failedRecoverQueues";
void setLastShippedAge(long age);
void incrSizeOfLogQueue(int size);
@@ -72,4 +73,6 @@ public interface MetricsReplicationSourceSource extends BaseSource {
void incrRepeatedFileBytes(final long bytes);
void incrCompletedWAL();
void incrCompletedRecoveryQueue();
+ void incrFailedRecoveryQueue();
+
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/b22095ee/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 0e5c07f..64585fa 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
@@ -45,6 +45,7 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
private final MutableFastCounter repeatedFileBytes;
private final MutableFastCounter completedWAL;
private final MutableFastCounter completedRecoveryQueue;
+ private final MutableFastCounter failedRecoveryQueue;
public MetricsReplicationGlobalSourceSource(MetricsReplicationSourceImpl rms) {
this.rms = rms;
@@ -79,6 +80,8 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
repeatedFileBytes = rms.getMetricsRegistry().getCounter(SOURCE_REPEATED_LOG_FILE_BYTES, 0L);
completedWAL = rms.getMetricsRegistry().getCounter(SOURCE_COMPLETED_LOGS, 0L);
completedRecoveryQueue = rms.getMetricsRegistry().getCounter(SOURCE_COMPLETED_RECOVERY_QUEUES, 0L);
+ failedRecoveryQueue = rms.getMetricsRegistry()
+ .getCounter(SOURCE_FAILED_RECOVERY_QUEUES, 0L);
}
@Override public void setLastShippedAge(long age) {
@@ -189,7 +192,10 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
public void incrCompletedRecoveryQueue() {
completedRecoveryQueue.incr(1L);
}
-
+ @Override
+ public void incrFailedRecoveryQueue() {
+ failedRecoveryQueue.incr(1L);
+ }
@Override
public void init() {
rms.init();
http://git-wip-us.apache.org/repos/asf/hbase/blob/b22095ee/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 9838e42..0078a97 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
@@ -254,6 +254,9 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
}
@Override
+ public void incrFailedRecoveryQueue() {/*no op*/}
+
+ @Override
public void init() {
rms.init();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/b22095ee/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 2d99018..c08b187 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
@@ -278,6 +278,11 @@ public class MetricsSource implements BaseSource {
globalSourceSource.incrCompletedRecoveryQueue();
}
+ public void incrFailedRecoveryQueue() {
+ globalSourceSource.incrFailedRecoveryQueue();
+ }
+
+
@Override
public void init() {
singleSourceSource.init();
http://git-wip-us.apache.org/repos/asf/hbase/blob/b22095ee/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 5686fa6..d5b5c63 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -47,6 +47,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableDescriptors;
@@ -536,6 +537,8 @@ public class ReplicationSourceManager implements ReplicationListener {
try {
this.executor.execute(transfer);
} catch (RejectedExecutionException ex) {
+ CompatibilitySingletonFactory.getInstance(MetricsReplicationSourceFactory.class)
+ .getGlobalSource().incrFailedRecoveryQueue();
LOG.info("Cancelling the transfer of " + rsZnode + " because of " + ex.getMessage());
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/b22095ee/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
index 277d876..f2a5d58 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.replication;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -53,7 +52,9 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@@ -306,7 +307,9 @@ public class TestReplicationEndpoint extends TestReplicationBase {
MetricsReplicationSourceSource singleSourceSource = new MetricsReplicationSourceSourceImpl(singleRms, id);
MetricsReplicationSourceSource globalSourceSource = new MetricsReplicationGlobalSourceSource(globalRms);
- MetricsSource source = new MetricsSource(id, singleSourceSource, globalSourceSource);
+ MetricsReplicationSourceSource spyglobalSourceSource = spy(globalSourceSource);
+ doNothing().when(spyglobalSourceSource).incrFailedRecoveryQueue();
+ MetricsSource source = new MetricsSource(id, singleSourceSource, spyglobalSourceSource);
String gaugeName = "gauge";
String singleGaugeName = "source.id." + gaugeName;
long delta = 1;
@@ -324,6 +327,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
source.removeMetric(gaugeName);
source.setGauge(gaugeName, delta);
source.updateHistogram(counterName, count);
+ source.incrFailedRecoveryQueue();
verify(singleRms).decGauge(singleGaugeName, delta);
verify(globalRms).decGauge(gaugeName, delta);
@@ -341,6 +345,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
verify(globalRms).setGauge(gaugeName, delta);
verify(singleRms).updateHistogram(singleCounterName, count);
verify(globalRms).updateHistogram(counterName, count);
+ verify(spyglobalSourceSource).incrFailedRecoveryQueue();
}
private void doPut(byte[] row) throws IOException {