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/12/14 00:15:11 UTC
[43/50] [abbrv] 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>
Conflicts:
hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/1ecfca3a
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/1ecfca3a
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/1ecfca3a
Branch: refs/heads/branch-1.3
Commit: 1ecfca3a5c7587a3a69f718ffbeec0d36cbcf4d1
Parents: 9675ad3
Author: Xu Cang <xc...@salesforce.com>
Authored: Wed Aug 29 16:39:57 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 19:25:33 2018 -0800
----------------------------------------------------------------------
.../hadoop/hbase/replication/ReplicationQueuesZKImpl.java | 5 ++++-
.../regionserver/MetricsReplicationSourceSource.java | 3 +++
.../regionserver/MetricsReplicationGlobalSourceSource.java | 8 ++++++++
.../regionserver/MetricsReplicationSourceSourceImpl.java | 5 +++++
.../hadoop/hbase/replication/regionserver/MetricsSource.java | 4 ++++
.../replication/regionserver/ReplicationSourceManager.java | 3 +++
.../hadoop/hbase/replication/TestReplicationEndpoint.java | 8 +++++++-
7 files changed, 34 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/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 3554a08..5ae2f5b 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
@@ -490,7 +490,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/1ecfca3a/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 1ed5a6b..4a2c807 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
@@ -47,6 +47,7 @@ public interface MetricsReplicationSourceSource {
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);
@@ -70,4 +71,6 @@ public interface MetricsReplicationSourceSource {
void incrRepeatedFileBytes(final long bytes);
void incrCompletedWAL();
void incrCompletedRecoveryQueue();
+ void incrFailedRecoveryQueue();
+
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/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 f3a4ae5..4b62382 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
@@ -46,6 +46,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;
@@ -82,6 +83,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,9 +192,14 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
public void incrCompletedWAL() {
completedWAL.incr(1L);
}
+
@Override
public void incrCompletedRecoveryQueue() {
completedRecoveryQueue.incr(1L);
}
+ @Override
+ public void incrFailedRecoveryQueue() {
+ failedRecoveryQueue.incr(1L);
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/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 783a61f..58f091c 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
@@ -256,4 +256,9 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
public void incrCompletedRecoveryQueue() {
completedRecoveryQueue.incr(1L);
}
+
+ @Override
+ public void incrFailedRecoveryQueue() {
+ /*no op*/
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/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 56baa05..4db48ca 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
@@ -290,6 +290,10 @@ public class MetricsSource {
globalSourceSource.incrCompletedRecoveryQueue();
}
+ public void incrFailedRecoveryQueue() {
+ globalSourceSource.incrFailedRecoveryQueue();
+ }
+
public Map<String, MetricsReplicationSourceSource> getSingleSourceSourceByTable() {
return singleSourceSourceByTable;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/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 8d19e22..d619f48 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;
@@ -535,6 +536,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/1ecfca3a/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 c3822c1..dd2e2f0 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
@@ -18,9 +18,11 @@
package org.apache.hadoop.hbase.replication;
+import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.ArrayList;
@@ -285,6 +287,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
MetricsReplicationSourceSource singleSourceSource = new MetricsReplicationSourceSourceImpl(singleRms, id);
MetricsReplicationSourceSource globalSourceSource = new MetricsReplicationGlobalSourceSource(globalRms);
MetricsReplicationSourceSource spyglobalSourceSource = spy(globalSourceSource);
+ doNothing().when(spyglobalSourceSource).incrFailedRecoveryQueue();
Map<String, MetricsReplicationSourceSource> singleSourceSourceByTable = new HashMap<>();
MetricsSource source = new MetricsSource(id, singleSourceSource, spyglobalSourceSource,
singleSourceSourceByTable);
@@ -304,6 +307,9 @@ public class TestReplicationEndpoint extends TestReplicationBase {
// cannot put more concreate value here to verify because the age is arbitrary.
// as long as it's greater than 0, we see it as correct answer.
Assert.assertTrue(msr.getLastShippedAge() > 0);
+
+ source.incrFailedRecoveryQueue();
+ verify(spyglobalSourceSource).incrFailedRecoveryQueue();
}
private void doPut(byte[] row) throws IOException {