You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/08/04 20:57:31 UTC

[GitHub] [hbase] saintstack opened a new pull request #2198: HBASE-24817 Allow configuring WALEntry filters on ReplicationSource

saintstack opened a new pull request #2198:
URL: https://github.com/apache/hbase/pull/2198


   Allow specifying base WALEntry filter on construction of
   ReplicationSource. Add means of being able to filter WALs by name.
   
   hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
    Add constructor that allows passing a predicate for filtering *in* WALs
    and a list of filters for filtering *out* WALEntries. The latter was
    hardcoded to filter out system-table WALEntries. The former did not
    exist but we'll need it if Replication takes in more than just the
    default Provider.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] anoopsjohn commented on a change in pull request #2198: HBASE-24817 Allow configuring WALEntry filters on ReplicationSource

Posted by GitBox <gi...@apache.org>.
anoopsjohn commented on a change in pull request #2198:
URL: https://github.com/apache/hbase/pull/2198#discussion_r465907752



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
##########
@@ -194,30 +232,34 @@ private void decorateConf() {
   }
 
   @Override
-  public void enqueueLog(Path log) {
-    String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(log.getName());
+  public void enqueueLog(Path wal) {
+    if (!this.filterInWALs.test(wal)) {
+      LOG.trace("NOT replicating {}", wal);
+      return;
+    }
+    String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(wal.getName());
     PriorityBlockingQueue<Path> queue = queues.get(logPrefix);
     if (queue == null) {
       queue = new PriorityBlockingQueue<>(queueSizePerGroup, new LogsComparator());
       queues.put(logPrefix, queue);
       if (this.isSourceActive() && this.walEntryFilter != null) {
         // new wal group observed after source startup, start a new worker thread to track it
-        // notice: it's possible that log enqueued when this.running is set but worker thread
+        // notice: it's possible that wal enqueued when this.running is set but worker thread
         // still not launched, so it's necessary to check workerThreads before start the worker
         tryStartNewShipper(logPrefix, queue);
       }
     }
-    queue.put(log);
+    queue.put(wal);
     if (LOG.isTraceEnabled()) {
-      LOG.trace("{} Added log file {} to queue of source {}.", logPeerId(), logPrefix,
+      LOG.trace("{} Added wal {} to queue of source {}.", logPeerId(), logPrefix,
         this.replicationQueueInfo.getQueueId());
     }
     this.metrics.incrSizeOfLogQueue();
-    // This will log a warning for each new log that gets created above the warn threshold
+    // This will wal a warning for each new wal that gets created above the warn threshold
     int queueSize = queue.size();
     if (queueSize > this.logQueueWarnThreshold) {
       LOG.warn("{} WAL group {} queue size: {} exceeds value of "
-          + "replication.source.log.queue.warn: {}", logPeerId(),
+          + "replication.source.wal.queue.warn: {}", logPeerId(),

Review comment:
       Are we changing this config name also in this patch?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2198: HBASE-24817 Allow configuring WALEntry filters on ReplicationSource

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2198:
URL: https://github.com/apache/hbase/pull/2198#issuecomment-668840966


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 42s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ branch-2 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 57s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 10s |  branch-2 passed  |
   | +1 :green_heart: |  spotbugs  |   2m  7s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 23s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  7s |  hbase-server: The patch generated 2 new + 1 unchanged - 15 fixed = 3 total (was 16)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m  9s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 16s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 14s |  The patch does not generate ASF License warnings.  |
   |  |   |  35m 30s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2198 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 27f06bc51dd0 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 8979202c7a |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #2198: HBASE-24817 Allow configuring WALEntry filters on ReplicationSource

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #2198:
URL: https://github.com/apache/hbase/pull/2198#discussion_r465995263



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/WALEntryFilter.java
##########
@@ -49,5 +47,5 @@
    * @return a (possibly modified) Entry to use. Returning null or an entry with no cells will cause
    *         the entry to be skipped for replication.
    */
-  public Entry filter(Entry entry);
+  Entry filter(Entry entry);

Review comment:
       This is how it works now.... no Optional. Will leave it in this patch. Thanks.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on pull request #2198: HBASE-24817 Allow configuring WALEntry filters on ReplicationSource

Posted by GitBox <gi...@apache.org>.
saintstack commented on pull request #2198:
URL: https://github.com/apache/hbase/pull/2198#issuecomment-669501322


   The failures seem unrelated (too many and the few I checked pass locally)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2198: HBASE-24817 Allow configuring WALEntry filters on ReplicationSource

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2198:
URL: https://github.com/apache/hbase/pull/2198#discussion_r465794580



##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java
##########
@@ -295,7 +356,40 @@ protected void stopServiceThreads() {
     }
   }
 
-  // Test HBASE-20497
+  /**
+   * Deadend Endpoint. Does nothing.
+   */
+  public static class DoNothingReplicationEndpoint extends HBaseInterClusterReplicationEndpoint {
+    private final UUID uuid = UUID.randomUUID();
+
+    @Override public void init(Context context) throws IOException {
+      this.ctx = context;
+      return;

Review comment:
       nit: redundant

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java
##########
@@ -108,16 +107,100 @@ public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniDFSCluster();
   }
 
+  /**
+   * Test the default ReplicationSource skips queuing hbase:meta WAL files.
+   */
+  @Test
+  public void testDefaultSkipsMetaWAL() throws IOException {
+    ReplicationSource rs = new ReplicationSource();
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    conf.setInt("replication.source.maxretriesmultiplier", 1);
+    ReplicationPeer mockPeer = Mockito.mock(ReplicationPeer.class);
+    Mockito.when(mockPeer.getConfiguration()).thenReturn(conf);
+    Mockito.when(mockPeer.getPeerBandwidth()).thenReturn(0L);
+    ReplicationPeerConfig peerConfig = Mockito.mock(ReplicationPeerConfig.class);
+    Mockito.when(peerConfig.getReplicationEndpointImpl()).
+      thenReturn(DoNothingReplicationEndpoint.class.getName());
+    Mockito.when(mockPeer.getPeerConfig()).thenReturn(peerConfig);
+    ReplicationSourceManager manager = Mockito.mock(ReplicationSourceManager.class);
+    Mockito.when(manager.getTotalBufferUsed()).thenReturn(new AtomicLong());
+    String queueId = "qid";
+    RegionServerServices rss =
+      TEST_UTIL.createMockRegionServerService(ServerName.parseServerName("a.b.c,1,1"));
+    rs.init(conf, null, manager, null, mockPeer, rss, queueId, null,
+      p -> OptionalLong.empty(), new MetricsSource(queueId));
+    try {
+      rs.startup();
+      assertTrue(rs.isSourceActive());
+      assertEquals(0, rs.getSourceMetrics().getSizeOfLogQueue());
+      rs.enqueueLog(new Path("a.1" + META_WAL_PROVIDER_ID));
+      assertEquals(0, rs.getSourceMetrics().getSizeOfLogQueue());
+      rs.enqueueLog(new Path("a.1"));
+      assertEquals(1, rs.getSourceMetrics().getSizeOfLogQueue());
+    } finally {
+      rs.terminate("Done");
+      rss.stop("Done");
+    }
+  }
+
+  /**
+   * Test that we filter out meta edits, etc.
+   */
+  @Test
+  public void testWALEntryFilter() throws IOException {
+    // To get the fully constructed default WALEntryFilter, need to create a ReplicationSource
+    // instance and init it.
+    ReplicationSource rs = new ReplicationSource();
+    UUID uuid = UUID.randomUUID();
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    ReplicationPeer mockPeer = Mockito.mock(ReplicationPeer.class);
+    Mockito.when(mockPeer.getConfiguration()).thenReturn(conf);
+    Mockito.when(mockPeer.getPeerBandwidth()).thenReturn(0L);
+    ReplicationPeerConfig peerConfig = Mockito.mock(ReplicationPeerConfig.class);
+    Mockito.when(peerConfig.getReplicationEndpointImpl()).
+      thenReturn(DoNothingReplicationEndpoint.class.getName());
+    Mockito.when(mockPeer.getPeerConfig()).thenReturn(peerConfig);
+    ReplicationSourceManager manager = Mockito.mock(ReplicationSourceManager.class);
+    Mockito.when(manager.getTotalBufferUsed()).thenReturn(new AtomicLong());
+    String queueId = "qid";
+    RegionServerServices rss =
+      TEST_UTIL.createMockRegionServerService(ServerName.parseServerName("a.b.c,1,1"));
+    rs.init(conf, null, manager, null, mockPeer, rss, queueId,
+      uuid, p -> OptionalLong.empty(), new MetricsSource(queueId));
+    try {
+      rs.startup();
+      TEST_UTIL.waitFor(30000, () -> rs.getWalEntryFilter() != null);
+      WALEntryFilter wef = rs.getWalEntryFilter();
+      // Test non-system WAL edit.
+      WAL.Entry e = new WAL.Entry(new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY,
+        TableName.valueOf("test"), -1), new WALEdit());
+      assertTrue(wef.filter(e) == e);

Review comment:
       nit: just in case if you like `assertSame(e, wef.filter(e))`

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/WALEntryFilter.java
##########
@@ -49,5 +47,5 @@
    * @return a (possibly modified) Entry to use. Returning null or an entry with no cells will cause
    *         the entry to be skipped for replication.
    */
-  public Entry filter(Entry entry);
+  Entry filter(Entry entry);

Review comment:
       For system tables we return null Entry. Good to consider return type `Optional<Entry>` here?
   Maybe as follow up task?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2198: HBASE-24817 Allow configuring WALEntry filters on ReplicationSource

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2198:
URL: https://github.com/apache/hbase/pull/2198#issuecomment-669118511


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m 11s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  7s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ branch-2 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 14s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   1m  4s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   5m 49s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in branch-2 failed.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 54s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  5s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 51s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 38s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 653m 34s |  hbase-server in the patch failed.  |
   |  |   | 805m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2198 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux f21d637f4ba0 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 8979202c7a |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/1/testReport/ |
   | Max. process+thread count | 4505 (vs. ulimit of 12500) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack merged pull request #2198: HBASE-24817 Allow configuring WALEntry filters on ReplicationSource

Posted by GitBox <gi...@apache.org>.
saintstack merged pull request #2198:
URL: https://github.com/apache/hbase/pull/2198


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #2198: HBASE-24817 Allow configuring WALEntry filters on ReplicationSource

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #2198:
URL: https://github.com/apache/hbase/pull/2198#discussion_r465994449



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
##########
@@ -194,30 +232,34 @@ private void decorateConf() {
   }
 
   @Override
-  public void enqueueLog(Path log) {
-    String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(log.getName());
+  public void enqueueLog(Path wal) {
+    if (!this.filterInWALs.test(wal)) {
+      LOG.trace("NOT replicating {}", wal);
+      return;
+    }
+    String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(wal.getName());
     PriorityBlockingQueue<Path> queue = queues.get(logPrefix);
     if (queue == null) {
       queue = new PriorityBlockingQueue<>(queueSizePerGroup, new LogsComparator());
       queues.put(logPrefix, queue);
       if (this.isSourceActive() && this.walEntryFilter != null) {
         // new wal group observed after source startup, start a new worker thread to track it
-        // notice: it's possible that log enqueued when this.running is set but worker thread
+        // notice: it's possible that wal enqueued when this.running is set but worker thread
         // still not launched, so it's necessary to check workerThreads before start the worker
         tryStartNewShipper(logPrefix, queue);
       }
     }
-    queue.put(log);
+    queue.put(wal);
     if (LOG.isTraceEnabled()) {
-      LOG.trace("{} Added log file {} to queue of source {}.", logPeerId(), logPrefix,
+      LOG.trace("{} Added wal {} to queue of source {}.", logPeerId(), logPrefix,
         this.replicationQueueInfo.getQueueId());
     }
     this.metrics.incrSizeOfLogQueue();
-    // This will log a warning for each new log that gets created above the warn threshold
+    // This will wal a warning for each new wal that gets created above the warn threshold
     int queueSize = queue.size();
     if (queueSize > this.logQueueWarnThreshold) {
       LOG.warn("{} WAL group {} queue size: {} exceeds value of "
-          + "replication.source.log.queue.warn: {}", logPeerId(),
+          + "replication.source.wal.queue.warn: {}", logPeerId(),

Review comment:
       Good catch. Thanks. Let me restore.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #2198: HBASE-24817 Allow configuring WALEntry filters on ReplicationSource

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #2198:
URL: https://github.com/apache/hbase/pull/2198#discussion_r465994744



##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java
##########
@@ -295,7 +356,40 @@ protected void stopServiceThreads() {
     }
   }
 
-  // Test HBASE-20497
+  /**
+   * Deadend Endpoint. Does nothing.
+   */
+  public static class DoNothingReplicationEndpoint extends HBaseInterClusterReplicationEndpoint {
+    private final UUID uuid = UUID.randomUUID();
+
+    @Override public void init(Context context) throws IOException {
+      this.ctx = context;
+      return;

Review comment:
       Yes. Fixed. Thanks.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2198: HBASE-24817 Allow configuring WALEntry filters on ReplicationSource

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2198:
URL: https://github.com/apache/hbase/pull/2198#issuecomment-669524398


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 36s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ branch-2 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 43s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   1m  9s |  branch-2 passed  |
   | +1 :green_heart: |  spotbugs  |   2m  0s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 14s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  5s |  hbase-server: The patch generated 2 new + 1 unchanged - 15 fixed = 3 total (was 16)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 33s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m  8s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 14s |  The patch does not generate ASF License warnings.  |
   |  |   |  33m 53s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2198 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 7704ed5e54c4 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | branch-2 / 64fe71dba3 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2198/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org