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/09/24 09:46:27 UTC

[GitHub] [hbase] infraio opened a new pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

infraio opened a new pull request #2452:
URL: https://github.com/apache/hbase/pull/2452


   


----------------------------------------------------------------
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 #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 10s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 49s |  HBASE-24666 passed  |
   | +1 :green_heart: |  compile  |   1m 46s |  HBASE-24666 passed  |
   | +1 :green_heart: |  shadedjars  |   7m 17s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 49s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 47s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 47s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 11s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 48s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 48s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  | 210m  4s |  hbase-server in the patch passed.  |
   |  |   | 241m 47s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/11/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | unit javac javadoc shadedjars compile |
   | uname | Linux bdd0029df15d 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-24666 / f67c3dfc5a |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/11/testReport/ |
   | Max. process+thread count | 3559 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/11/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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -78,27 +104,44 @@
    */
   private ServerName serverName;
 
-  protected final Configuration conf;
+  private final Configuration conf;
 
-  private ReplicationSinkService replicationSinkService;
+  // zookeeper connection and watcher
+  private final ZKWatcher zooKeeper;
 
-  final int msgInterval;
-  // A sleeper that sleeps for msgInterval.
-  protected final Sleeper sleeper;
+  private final UUID clusterId;
 
-  // zookeeper connection and watcher
-  protected final ZKWatcher zooKeeper;
+  private HFileSystem walFs;
+  private Path walRootDir;
+
+  /**
+   * ChoreService used to schedule tasks that we want to run periodically
+   */
+  private ChoreService choreService;
 
   /**
    * The asynchronous cluster connection to be shared by services.
    */
-  protected AsyncClusterConnection asyncClusterConnection;
+  private AsyncClusterConnection asyncClusterConnection;
 
   private UserProvider userProvider;
 
-  protected final ReplicationServerRpcServices rpcServices;
+  private final ReplicationServerRpcServices rpcServices;
+
+  // Total buffer size on this RegionServer for holding batched edits to be shipped.
+  private final long totalBufferLimit;
+  private AtomicLong totalBufferUsed = new AtomicLong();
+
+  private final MetricsReplicationGlobalSourceSource globalMetrics;
+  private final Map<String, MetricsSource> sourceMetrics = new HashMap<>();
+  private final ConcurrentMap<String, ReplicationSourceInterface> sources = new ConcurrentHashMap<>();
+
+  private final ReplicationQueueStorage queueStorage;
+  private final ReplicationPeers replicationPeers;

Review comment:
       The key thing of this feature is here. The new HReplicationServer will be a independent server which manage some ReplicationSource.
   
   The assumption here is that: Replication is a producer-consumer module. The "producer" RegionServer write WAL to HDFS and write WAL name to a replication queue, which can be identified by <RegionServer ServerName, QueueId>. The "consumer" ReplicationSource fetch the WAL name from the replication queue and read the WAL from HDFS and replicate to other HBase Clusters or other Systems.
   
   When replication offload enabled, ReplicationSourceManager will only wirte the WAL name to the replication queue. But not start any ReplicationSource. HMaster get all replication queues and assign them to many ReplicationServers. Same with the region balancer, HMaster called a StartReplicationSource rpc to the ReplicationServer. Then the ReplicationServer will start a ReplicationSource thread to do the replication job.
   
   
   
   




----------------------------------------------------------------
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] infraio merged pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   


----------------------------------------------------------------
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 #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m 24s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 59s |  HBASE-24666 passed  |
   | +1 :green_heart: |  compile  |   1m 47s |  HBASE-24666 passed  |
   | +1 :green_heart: |  shadedjars  |   7m 12s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 49s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 51s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 47s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 47s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m  8s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 48s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 48s |  hbase-protocol-shaded in the patch passed.  |
   | -1 :x: |  unit  | 209m 50s |  hbase-server in the patch failed.  |
   |  |   | 245m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/10/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | unit javac javadoc shadedjars compile |
   | uname | Linux 8a631ab9fd67 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-24666 / f67c3dfc5a |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/10/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/10/testReport/ |
   | Max. process+thread count | 3585 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/10/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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
       One replication queue can be identified by <Producer ServerName, QueueId>.




----------------------------------------------------------------
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 #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m 18s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +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.  |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 28s |  HBASE-24666 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 36s |  HBASE-24666 passed  |
   | +1 :green_heart: |  spotbugs  |   6m 33s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 51s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 11s |  hbase-server: The patch generated 3 new + 56 unchanged - 0 fixed = 59 total (was 56)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  20m 49s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 47s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m 51s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 27s |  The patch does not generate ASF License warnings.  |
   |  |   |  62m 21s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/10/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | dupname asflicense cc hbaseprotoc prototool spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 69b1fc18422e 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 | HBASE-24666 / f67c3dfc5a |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/10/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 95 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/10/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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
          * @param server the server which start and run this replication source
      * @param producer the name of region server which produce WAL to the replication queue
   
   Is this comment clearly?




----------------------------------------------------------------
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] ddupg commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
##########
@@ -224,6 +226,35 @@ public void init(Configuration conf, FileSystem fs, Path walDir,
     this.abortOnError = this.conf.getBoolean("replication.source.regionserver.abort",
       true);
 
+    if (conf.getBoolean(HConstants.REPLICATION_OFFLOAD_ENABLE_KEY,
+      HConstants.REPLICATION_OFFLOAD_ENABLE_DEFAULT)) {
+      fetchWALsThread = new Thread(() -> {

Review comment:
       interrrupt the `fetchWALsThread` when the ReplicationServer exits or when the peer terminates?




----------------------------------------------------------------
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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -78,27 +104,44 @@
    */
   private ServerName serverName;
 
-  protected final Configuration conf;
+  private final Configuration conf;
 
-  private ReplicationSinkService replicationSinkService;
+  // zookeeper connection and watcher
+  private final ZKWatcher zooKeeper;
 
-  final int msgInterval;
-  // A sleeper that sleeps for msgInterval.
-  protected final Sleeper sleeper;
+  private final UUID clusterId;
 
-  // zookeeper connection and watcher
-  protected final ZKWatcher zooKeeper;
+  private HFileSystem walFs;
+  private Path walRootDir;
+
+  /**
+   * ChoreService used to schedule tasks that we want to run periodically
+   */
+  private ChoreService choreService;
 
   /**
    * The asynchronous cluster connection to be shared by services.
    */
-  protected AsyncClusterConnection asyncClusterConnection;
+  private AsyncClusterConnection asyncClusterConnection;
 
   private UserProvider userProvider;
 
-  protected final ReplicationServerRpcServices rpcServices;
+  private final ReplicationServerRpcServices rpcServices;

Review comment:
       I am ok with this. Decouple a RpcService can make the code more clearly.




----------------------------------------------------------------
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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
##########
@@ -52,15 +52,17 @@
    * @param queueStorage the replication queue storage
    * @param replicationPeer the replication peer
    * @param server the server which start and run this replication source
+   * @param producer the name of region server which produce the replication queue

Review comment:
       For all ReplicationSource. The assumption here is that: all replication is a producer-consumer module. 




----------------------------------------------------------------
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 #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto
##########
@@ -721,7 +721,8 @@ message ListReplicationSinkServersRequest {
 }
 
 message ListReplicationSinkServersResponse {
-  repeated ServerName server_name = 1;
+  required bool is_replication_server = 1;

Review comment:
       Is this right name for the attribute? The method name for a boolean should be isATTRIBUTE but the ATTRIBUTE itself should not have the 'is' prefix.

##########
File path: hbase-protocol-shaded/src/main/protobuf/server/replication/ReplicationServer.proto
##########
@@ -24,9 +24,21 @@ option java_generic_services = true;
 option java_generate_equals_and_hash = true;

Review comment:
       Does the replication server belong in the core? Should it be done outside in hbase-operator-tools? Just add the communication bits to core?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -78,27 +104,44 @@
    */
   private ServerName serverName;
 
-  protected final Configuration conf;
+  private final Configuration conf;
 
-  private ReplicationSinkService replicationSinkService;
+  // zookeeper connection and watcher
+  private final ZKWatcher zooKeeper;
 
-  final int msgInterval;
-  // A sleeper that sleeps for msgInterval.
-  protected final Sleeper sleeper;
+  private final UUID clusterId;
 
-  // zookeeper connection and watcher
-  protected final ZKWatcher zooKeeper;
+  private HFileSystem walFs;
+  private Path walRootDir;
+
+  /**
+   * ChoreService used to schedule tasks that we want to run periodically
+   */
+  private ChoreService choreService;
 
   /**
    * The asynchronous cluster connection to be shared by services.
    */
-  protected AsyncClusterConnection asyncClusterConnection;
+  private AsyncClusterConnection asyncClusterConnection;
 
   private UserProvider userProvider;
 
-  protected final ReplicationServerRpcServices rpcServices;
+  private final ReplicationServerRpcServices rpcServices;

Review comment:
       We have to do this? Just because we do it for HMaster and HRegionServer, it doesn't make it a good pattern. This is an Interface to pass instead of the impl?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -50,7 +72,7 @@
  */
 @InterfaceAudience.Private
 @SuppressWarnings({ "deprecation"})
-public class HReplicationServer extends Thread implements Server {
+public class HReplicationServer extends Thread implements Server, ReplicationSourceController  {

Review comment:
       Looks like ReplicationServer already committed (why have the 'H' prefix?)

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -118,6 +118,7 @@
   private boolean fetchServersUseZk = false;
   private FetchServersChore fetchServersChore;
   private int shortOperationTimeout;
+  private boolean isReplicationServer = false;

Review comment:
       Yeah, boolean should be named replicationServer and the method to access this data member is named isReplicationServer.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -226,13 +302,16 @@ protected final synchronized void setupClusterConnection() throws IOException {
   }
 
   /**
-   * Wait on all threads to finish. Presumption is that all closes and stops
+   * Wait on all threads to fnish. Presumption is that all closes and stops

Review comment:
       Undo this change. Misspelling.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -295,6 +296,7 @@ private synchronized AsyncClusterConnection getPeerConnection() throws IOExcepti
           .createBlockingRpcChannel(master, User.getCurrent(), shortOperationTimeout));
       ListReplicationSinkServersResponse resp = masterStub
         .listReplicationSinkServers(null, ListReplicationSinkServersRequest.newBuilder().build());
+      isReplicationServer = resp.getIsReplicationServer();

Review comment:
       Do you have to do hasReplicationServer first? And then read it if present? Or maybe this defaults false if not present?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -78,27 +104,44 @@
    */
   private ServerName serverName;
 
-  protected final Configuration conf;
+  private final Configuration conf;
 
-  private ReplicationSinkService replicationSinkService;
+  // zookeeper connection and watcher
+  private final ZKWatcher zooKeeper;
 
-  final int msgInterval;
-  // A sleeper that sleeps for msgInterval.
-  protected final Sleeper sleeper;
+  private final UUID clusterId;
 
-  // zookeeper connection and watcher
-  protected final ZKWatcher zooKeeper;
+  private HFileSystem walFs;
+  private Path walRootDir;
+
+  /**
+   * ChoreService used to schedule tasks that we want to run periodically
+   */
+  private ChoreService choreService;
 
   /**
    * The asynchronous cluster connection to be shared by services.
    */
-  protected AsyncClusterConnection asyncClusterConnection;
+  private AsyncClusterConnection asyncClusterConnection;
 
   private UserProvider userProvider;
 
-  protected final ReplicationServerRpcServices rpcServices;
+  private final ReplicationServerRpcServices rpcServices;

Review comment:
       Maybe it is to mock?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
       Perhaps say more about this?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -78,27 +104,44 @@
    */
   private ServerName serverName;
 
-  protected final Configuration conf;
+  private final Configuration conf;
 
-  private ReplicationSinkService replicationSinkService;
+  // zookeeper connection and watcher
+  private final ZKWatcher zooKeeper;
 
-  final int msgInterval;
-  // A sleeper that sleeps for msgInterval.
-  protected final Sleeper sleeper;
+  private final UUID clusterId;
 
-  // zookeeper connection and watcher
-  protected final ZKWatcher zooKeeper;
+  private HFileSystem walFs;
+  private Path walRootDir;
+
+  /**
+   * ChoreService used to schedule tasks that we want to run periodically
+   */
+  private ChoreService choreService;
 
   /**
    * The asynchronous cluster connection to be shared by services.
    */
-  protected AsyncClusterConnection asyncClusterConnection;
+  private AsyncClusterConnection asyncClusterConnection;
 
   private UserProvider userProvider;
 
-  protected final ReplicationServerRpcServices rpcServices;
+  private final ReplicationServerRpcServices rpcServices;
+
+  // Total buffer size on this RegionServer for holding batched edits to be shipped.
+  private final long totalBufferLimit;
+  private AtomicLong totalBufferUsed = new AtomicLong();
+
+  private final MetricsReplicationGlobalSourceSource globalMetrics;
+  private final Map<String, MetricsSource> sourceMetrics = new HashMap<>();
+  private final ConcurrentMap<String, ReplicationSourceInterface> sources = new ConcurrentHashMap<>();
+
+  private final ReplicationQueueStorage queueStorage;
+  private final ReplicationPeers replicationPeers;

Review comment:
       Is this stuff normally inside a ReplicationSourceManager? Or maybe the Replication instance? Can you not use these instead?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
##########
@@ -52,15 +52,17 @@
    * @param queueStorage the replication queue storage
    * @param replicationPeer the replication peer
    * @param server the server which start and run this replication source
+   * @param producer the name of region server which produce the replication queue

Review comment:
       This is for the RecoveryReplicationSource only?

##########
File path: hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto
##########
@@ -721,7 +721,8 @@ message ListReplicationSinkServersRequest {
 }
 
 message ListReplicationSinkServersResponse {
-  repeated ServerName server_name = 1;
+  required bool is_replication_server = 1;

Review comment:
       Oh, you can't change the protobuf index.... it breaks compatibility. Give the replication_server '2' and leave server_name as '1'.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
       There will be a time when the passed in 'server' differs from server.getServerName?




----------------------------------------------------------------
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 #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 56s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m  7s |  HBASE-24666 passed  |
   | +1 :green_heart: |  compile  |   2m 40s |  HBASE-24666 passed  |
   | +1 :green_heart: |  shadedjars  |   9m 16s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 19s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 54s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m 11s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  0s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 30s |  hbase-replication in the patch passed.  |
   | -1 :x: |  unit  | 173m  2s |  hbase-server in the patch failed.  |
   |  |   | 215m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | unit javac javadoc shadedjars compile |
   | uname | Linux 9300bdf7114c 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 | HBASE-24666 / 6008ffeb17 |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/1/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/1/testReport/ |
   | Max. process+thread count | 3694 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-replication hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 42s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +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.  |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 51s |  HBASE-24666 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 25s |  HBASE-24666 passed  |
   | +1 :green_heart: |  spotbugs  |   6m  7s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 39s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 11s |  hbase-server: The patch generated 1 new + 199 unchanged - 0 fixed = 200 total (was 199)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m  0s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 44s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   6m 31s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 27s |  The patch does not generate ASF License warnings.  |
   |  |   |  46m 31s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/5/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | dupname asflicense cc hbaseprotoc prototool spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 1dcf84dfbc5b 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 | HBASE-24666 / 6008ffeb17 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/5/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/5/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 27s |  HBASE-24666 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 28s |  HBASE-24666 passed  |
   | +1 :green_heart: |  spotbugs  |   5m 53s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 22s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  5s |  hbase-server: The patch generated 1 new + 56 unchanged - 0 fixed = 57 total (was 56)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  16m 54s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 58s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   6m 37s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 36s |  The patch does not generate ASF License warnings.  |
   |  |   |  50m 43s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/12/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | dupname asflicense cc hbaseprotoc prototool spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 24a7bab306c9 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 | HBASE-24666 / f67c3dfc5a |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/12/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-replication hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/12/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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto
##########
@@ -721,7 +721,8 @@ message ListReplicationSinkServersRequest {
 }
 
 message ListReplicationSinkServersResponse {
-  repeated ServerName server_name = 1;
+  required bool is_replication_server = 1;

Review comment:
       This is only for the feature branch. Not merged to any branch now.




----------------------------------------------------------------
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 #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 49s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 59s |  HBASE-24666 passed  |
   | +1 :green_heart: |  compile  |   2m  6s |  HBASE-24666 passed  |
   | +1 :green_heart: |  shadedjars  |   7m  7s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  2s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 13s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 13s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 30s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  8s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 52s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 40s |  hbase-replication in the patch passed.  |
   | -1 :x: |  unit  | 229m 22s |  hbase-server in the patch failed.  |
   |  |   | 264m 49s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/12/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | unit javac javadoc shadedjars compile |
   | uname | Linux 00bf3087f026 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-24666 / f67c3dfc5a |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/12/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/12/testReport/ |
   | Max. process+thread count | 3930 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-replication hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/12/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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






----------------------------------------------------------------
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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto
##########
@@ -721,7 +721,8 @@ message ListReplicationSinkServersRequest {
 }
 
 message ListReplicationSinkServersResponse {
-  repeated ServerName server_name = 1;
+  required bool is_replication_server = 1;

Review comment:
       This is only for the feature branch. Not merged to any branch now.

##########
File path: hbase-protocol-shaded/src/main/protobuf/server/replication/ReplicationServer.proto
##########
@@ -24,9 +24,21 @@ option java_generic_services = true;
 option java_generate_equals_and_hash = true;

Review comment:
       Yes, in the core. The HReplicationServer will be a new role in one HBase cluster.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -50,7 +72,7 @@
  */
 @InterfaceAudience.Private
 @SuppressWarnings({ "deprecation"})
-public class HReplicationServer extends Thread implements Server {
+public class HReplicationServer extends Thread implements Server, ReplicationSourceController  {

Review comment:
       Only for the feature branch HBASE-24666.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -78,27 +104,44 @@
    */
   private ServerName serverName;
 
-  protected final Configuration conf;
+  private final Configuration conf;
 
-  private ReplicationSinkService replicationSinkService;
+  // zookeeper connection and watcher
+  private final ZKWatcher zooKeeper;
 
-  final int msgInterval;
-  // A sleeper that sleeps for msgInterval.
-  protected final Sleeper sleeper;
+  private final UUID clusterId;
 
-  // zookeeper connection and watcher
-  protected final ZKWatcher zooKeeper;
+  private HFileSystem walFs;
+  private Path walRootDir;
+
+  /**
+   * ChoreService used to schedule tasks that we want to run periodically
+   */
+  private ChoreService choreService;
 
   /**
    * The asynchronous cluster connection to be shared by services.
    */
-  protected AsyncClusterConnection asyncClusterConnection;
+  private AsyncClusterConnection asyncClusterConnection;
 
   private UserProvider userProvider;
 
-  protected final ReplicationServerRpcServices rpcServices;
+  private final ReplicationServerRpcServices rpcServices;

Review comment:
       I am ok with this. Decouple a RpcService can make the code more clearly.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -78,27 +104,44 @@
    */
   private ServerName serverName;
 
-  protected final Configuration conf;
+  private final Configuration conf;
 
-  private ReplicationSinkService replicationSinkService;
+  // zookeeper connection and watcher
+  private final ZKWatcher zooKeeper;
 
-  final int msgInterval;
-  // A sleeper that sleeps for msgInterval.
-  protected final Sleeper sleeper;
+  private final UUID clusterId;
 
-  // zookeeper connection and watcher
-  protected final ZKWatcher zooKeeper;
+  private HFileSystem walFs;
+  private Path walRootDir;
+
+  /**
+   * ChoreService used to schedule tasks that we want to run periodically
+   */
+  private ChoreService choreService;
 
   /**
    * The asynchronous cluster connection to be shared by services.
    */
-  protected AsyncClusterConnection asyncClusterConnection;
+  private AsyncClusterConnection asyncClusterConnection;
 
   private UserProvider userProvider;
 
-  protected final ReplicationServerRpcServices rpcServices;
+  private final ReplicationServerRpcServices rpcServices;
+
+  // Total buffer size on this RegionServer for holding batched edits to be shipped.
+  private final long totalBufferLimit;
+  private AtomicLong totalBufferUsed = new AtomicLong();
+
+  private final MetricsReplicationGlobalSourceSource globalMetrics;
+  private final Map<String, MetricsSource> sourceMetrics = new HashMap<>();
+  private final ConcurrentMap<String, ReplicationSourceInterface> sources = new ConcurrentHashMap<>();
+
+  private final ReplicationQueueStorage queueStorage;
+  private final ReplicationPeers replicationPeers;

Review comment:
       The key thing of this feature is here. The new HReplicationServer will be a independent server which manage some ReplicationSource.
   
   The assumption here is that: Replication is a producer-consumer module. The "producer" RegionServer write WAL to HDFS and write WAL name to a replication queue, which can be identified by <RegionServer ServerName, QueueId>. The "consumer" ReplicationSource fetch the WAL name from the replication queue and read the WAL from HDFS and replicate to other HBase Clusters or other Systems.
   
   When replication offload enabled, ReplicationSourceManager will only wirte the WAL name to the replication queue. But not start any ReplicationSource. HMaster get all replication queues and assign them to many ReplicationServers. Same with the region balancer, HMaster called a StartReplicationSource rpc to the ReplicationServer. Then the ReplicationServer will start a ReplicationSource thread to do the replication job.
   
   
   
   

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
          * @param server the server which start and run this replication source
      * @param producer the name of region server which produce WAL to the replication queue
   
   Is this comment clearly?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
       One replication queue can be identified by <Producer ServerName, QueueId>.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
       And one ReplicationSource is responsible for one replication queue.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -295,6 +296,7 @@ private synchronized AsyncClusterConnection getPeerConnection() throws IOExcepti
           .createBlockingRpcChannel(master, User.getCurrent(), shortOperationTimeout));
       ListReplicationSinkServersResponse resp = masterStub
         .listReplicationSinkServers(null, ListReplicationSinkServersRequest.newBuilder().build());
+      isReplicationServer = resp.getIsReplicationServer();

Review comment:
       This field is required. So not need to check hasReplicationServer first?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
##########
@@ -52,15 +52,17 @@
    * @param queueStorage the replication queue storage
    * @param replicationPeer the replication peer
    * @param server the server which start and run this replication source
+   * @param producer the name of region server which produce the replication queue

Review comment:
       For all ReplicationSource. The assumption here is that: all replication is a producer-consumer module. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
       When replication offload disabled, the ReplicationSource will be started in RegionServer inside. So the producer name is same with the Server's name.




----------------------------------------------------------------
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 #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 22s |  HBASE-24666 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 15s |  HBASE-24666 passed  |
   | +1 :green_heart: |  spotbugs  |   5m 32s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 21s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  3s |  hbase-server: The patch generated 1 new + 56 unchanged - 0 fixed = 57 total (was 56)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  16m 56s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 39s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   5m 56s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 25s |  The patch does not generate ASF License warnings.  |
   |  |   |  48m 24s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/11/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | dupname asflicense cc hbaseprotoc prototool spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 61f8e8dcfd67 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 | HBASE-24666 / f67c3dfc5a |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/11/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/11/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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






----------------------------------------------------------------
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 #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto
##########
@@ -721,7 +721,8 @@ message ListReplicationSinkServersRequest {
 }
 
 message ListReplicationSinkServersResponse {
-  repeated ServerName server_name = 1;
+  required bool is_replication_server = 1;

Review comment:
       Is this right name for the attribute? The method name for a boolean should be isATTRIBUTE but the ATTRIBUTE itself should not have the 'is' prefix.

##########
File path: hbase-protocol-shaded/src/main/protobuf/server/replication/ReplicationServer.proto
##########
@@ -24,9 +24,21 @@ option java_generic_services = true;
 option java_generate_equals_and_hash = true;

Review comment:
       Does the replication server belong in the core? Should it be done outside in hbase-operator-tools? Just add the communication bits to core?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -78,27 +104,44 @@
    */
   private ServerName serverName;
 
-  protected final Configuration conf;
+  private final Configuration conf;
 
-  private ReplicationSinkService replicationSinkService;
+  // zookeeper connection and watcher
+  private final ZKWatcher zooKeeper;
 
-  final int msgInterval;
-  // A sleeper that sleeps for msgInterval.
-  protected final Sleeper sleeper;
+  private final UUID clusterId;
 
-  // zookeeper connection and watcher
-  protected final ZKWatcher zooKeeper;
+  private HFileSystem walFs;
+  private Path walRootDir;
+
+  /**
+   * ChoreService used to schedule tasks that we want to run periodically
+   */
+  private ChoreService choreService;
 
   /**
    * The asynchronous cluster connection to be shared by services.
    */
-  protected AsyncClusterConnection asyncClusterConnection;
+  private AsyncClusterConnection asyncClusterConnection;
 
   private UserProvider userProvider;
 
-  protected final ReplicationServerRpcServices rpcServices;
+  private final ReplicationServerRpcServices rpcServices;

Review comment:
       We have to do this? Just because we do it for HMaster and HRegionServer, it doesn't make it a good pattern. This is an Interface to pass instead of the impl?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -50,7 +72,7 @@
  */
 @InterfaceAudience.Private
 @SuppressWarnings({ "deprecation"})
-public class HReplicationServer extends Thread implements Server {
+public class HReplicationServer extends Thread implements Server, ReplicationSourceController  {

Review comment:
       Looks like ReplicationServer already committed (why have the 'H' prefix?)

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -118,6 +118,7 @@
   private boolean fetchServersUseZk = false;
   private FetchServersChore fetchServersChore;
   private int shortOperationTimeout;
+  private boolean isReplicationServer = false;

Review comment:
       Yeah, boolean should be named replicationServer and the method to access this data member is named isReplicationServer.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -226,13 +302,16 @@ protected final synchronized void setupClusterConnection() throws IOException {
   }
 
   /**
-   * Wait on all threads to finish. Presumption is that all closes and stops
+   * Wait on all threads to fnish. Presumption is that all closes and stops

Review comment:
       Undo this change. Misspelling.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -295,6 +296,7 @@ private synchronized AsyncClusterConnection getPeerConnection() throws IOExcepti
           .createBlockingRpcChannel(master, User.getCurrent(), shortOperationTimeout));
       ListReplicationSinkServersResponse resp = masterStub
         .listReplicationSinkServers(null, ListReplicationSinkServersRequest.newBuilder().build());
+      isReplicationServer = resp.getIsReplicationServer();

Review comment:
       Do you have to do hasReplicationServer first? And then read it if present? Or maybe this defaults false if not present?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -78,27 +104,44 @@
    */
   private ServerName serverName;
 
-  protected final Configuration conf;
+  private final Configuration conf;
 
-  private ReplicationSinkService replicationSinkService;
+  // zookeeper connection and watcher
+  private final ZKWatcher zooKeeper;
 
-  final int msgInterval;
-  // A sleeper that sleeps for msgInterval.
-  protected final Sleeper sleeper;
+  private final UUID clusterId;
 
-  // zookeeper connection and watcher
-  protected final ZKWatcher zooKeeper;
+  private HFileSystem walFs;
+  private Path walRootDir;
+
+  /**
+   * ChoreService used to schedule tasks that we want to run periodically
+   */
+  private ChoreService choreService;
 
   /**
    * The asynchronous cluster connection to be shared by services.
    */
-  protected AsyncClusterConnection asyncClusterConnection;
+  private AsyncClusterConnection asyncClusterConnection;
 
   private UserProvider userProvider;
 
-  protected final ReplicationServerRpcServices rpcServices;
+  private final ReplicationServerRpcServices rpcServices;

Review comment:
       Maybe it is to mock?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
       Perhaps say more about this?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -78,27 +104,44 @@
    */
   private ServerName serverName;
 
-  protected final Configuration conf;
+  private final Configuration conf;
 
-  private ReplicationSinkService replicationSinkService;
+  // zookeeper connection and watcher
+  private final ZKWatcher zooKeeper;
 
-  final int msgInterval;
-  // A sleeper that sleeps for msgInterval.
-  protected final Sleeper sleeper;
+  private final UUID clusterId;
 
-  // zookeeper connection and watcher
-  protected final ZKWatcher zooKeeper;
+  private HFileSystem walFs;
+  private Path walRootDir;
+
+  /**
+   * ChoreService used to schedule tasks that we want to run periodically
+   */
+  private ChoreService choreService;
 
   /**
    * The asynchronous cluster connection to be shared by services.
    */
-  protected AsyncClusterConnection asyncClusterConnection;
+  private AsyncClusterConnection asyncClusterConnection;
 
   private UserProvider userProvider;
 
-  protected final ReplicationServerRpcServices rpcServices;
+  private final ReplicationServerRpcServices rpcServices;
+
+  // Total buffer size on this RegionServer for holding batched edits to be shipped.
+  private final long totalBufferLimit;
+  private AtomicLong totalBufferUsed = new AtomicLong();
+
+  private final MetricsReplicationGlobalSourceSource globalMetrics;
+  private final Map<String, MetricsSource> sourceMetrics = new HashMap<>();
+  private final ConcurrentMap<String, ReplicationSourceInterface> sources = new ConcurrentHashMap<>();
+
+  private final ReplicationQueueStorage queueStorage;
+  private final ReplicationPeers replicationPeers;

Review comment:
       Is this stuff normally inside a ReplicationSourceManager? Or maybe the Replication instance? Can you not use these instead?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
##########
@@ -52,15 +52,17 @@
    * @param queueStorage the replication queue storage
    * @param replicationPeer the replication peer
    * @param server the server which start and run this replication source
+   * @param producer the name of region server which produce the replication queue

Review comment:
       This is for the RecoveryReplicationSource only?

##########
File path: hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto
##########
@@ -721,7 +721,8 @@ message ListReplicationSinkServersRequest {
 }
 
 message ListReplicationSinkServersResponse {
-  repeated ServerName server_name = 1;
+  required bool is_replication_server = 1;

Review comment:
       Oh, you can't change the protobuf index.... it breaks compatibility. Give the replication_server '2' and leave server_name as '1'.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
       There will be a time when the passed in 'server' differs from server.getServerName?




----------------------------------------------------------------
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 #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +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.  |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 23s |  HBASE-24666 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 28s |  HBASE-24666 passed  |
   | +1 :green_heart: |  spotbugs  |   5m 58s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 24s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 26s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  17m  0s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 57s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   6m 34s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 37s |  The patch does not generate ASF License warnings.  |
   |  |   |  50m 45s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/13/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | dupname asflicense cc hbaseprotoc prototool spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 2efb0aa3d0e6 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 | HBASE-24666 / f67c3dfc5a |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-replication hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/13/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 20s |  HBASE-24666 passed  |
   | +1 :green_heart: |  compile  |   2m  7s |  HBASE-24666 passed  |
   | +1 :green_heart: |  shadedjars  |   6m 46s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 55s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  9s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m  7s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m  7s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 54s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 57s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  | 142m 39s |  hbase-server in the patch passed.  |
   |  |   | 175m  7s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/11/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | unit javac javadoc shadedjars compile |
   | uname | Linux a7eadd63d83d 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 | HBASE-24666 / f67c3dfc5a |
   | Default Java | 2020-01-14 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/11/testReport/ |
   | Max. process+thread count | 4004 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/11/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 38s |  HBASE-24666 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 50s |  HBASE-24666 passed  |
   | +1 :green_heart: |  spotbugs  |   7m 42s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 48s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 26s |  hbase-server: The patch generated 11 new + 54 unchanged - 0 fixed = 65 total (was 54)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 13s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m 29s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   8m 43s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 42s |  The patch does not generate ASF License warnings.  |
   |  |   |  59m 54s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | dupname asflicense cc hbaseprotoc prototool spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 3ff3ec453b65 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 | HBASE-24666 / 6008ffeb17 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-replication hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 15s |  HBASE-24666 passed  |
   | +1 :green_heart: |  compile  |   2m  4s |  HBASE-24666 passed  |
   | +1 :green_heart: |  shadedjars  |   6m 42s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 56s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  3s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m  3s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m  3s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 38s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 58s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  | 135m 45s |  hbase-server in the patch passed.  |
   |  |   | 167m 53s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/10/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | unit javac javadoc shadedjars compile |
   | uname | Linux 1c6ad0b868ee 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 | HBASE-24666 / f67c3dfc5a |
   | Default Java | 2020-01-14 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/10/testReport/ |
   | Max. process+thread count | 3992 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/10/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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
       When replication offload disabled, the ReplicationSource will be started in RegionServer inside. So the producer name is same with the Server's name.




----------------------------------------------------------------
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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HReplicationServer.java
##########
@@ -50,7 +72,7 @@
  */
 @InterfaceAudience.Private
 @SuppressWarnings({ "deprecation"})
-public class HReplicationServer extends Thread implements Server {
+public class HReplicationServer extends Thread implements Server, ReplicationSourceController  {

Review comment:
       Only for the feature branch HBASE-24666.




----------------------------------------------------------------
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 #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 21s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 30s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 39s |  HBASE-24666 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 24s |  HBASE-24666 passed  |
   | +1 :green_heart: |  spotbugs  |   5m 34s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 21s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  8s |  hbase-server: The patch generated 3 new + 199 unchanged - 0 fixed = 202 total (was 199)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 29s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 39s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   5m 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 26s |  The patch does not generate ASF License warnings.  |
   |  |   |  44m 30s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | dupname asflicense cc hbaseprotoc prototool spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 51571e22b5c1 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 | HBASE-24666 / 6008ffeb17 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/3/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/3/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 21s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   2m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  5s |  HBASE-24666 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 26s |  HBASE-24666 passed  |
   | +1 :green_heart: |  spotbugs  |   6m  9s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 55s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 14s |  hbase-server: The patch generated 6 new + 199 unchanged - 0 fixed = 205 total (was 199)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  13m 37s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 47s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   6m 53s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 26s |  The patch does not generate ASF License warnings.  |
   |  |   |  53m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | dupname asflicense cc hbaseprotoc prototool spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 1b681dc56d0b 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-24666 / 6008ffeb17 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/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



[GitHub] [hbase] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 20s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 15s |  HBASE-24666 passed  |
   | +1 :green_heart: |  compile  |   2m 35s |  HBASE-24666 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 16s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 17s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 15s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 40s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 40s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 39s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 10s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 55s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 41s |  hbase-replication in the patch passed.  |
   | -1 :x: |  unit  | 229m 57s |  hbase-server in the patch failed.  |
   |  |   | 272m  5s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/13/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | unit javac javadoc shadedjars compile |
   | uname | Linux fa6421a08220 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-24666 / f67c3dfc5a |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/13/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/13/testReport/ |
   | Max. process+thread count | 3126 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-replication hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/13/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  HBASE-24666 passed  |
   | +1 :green_heart: |  compile  |   2m 21s |  HBASE-24666 passed  |
   | +1 :green_heart: |  shadedjars  |   6m 37s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 12s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  1s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 23s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 43s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 11s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 57s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 27s |  hbase-replication in the patch passed.  |
   | +1 :green_heart: |  unit  | 133m 15s |  hbase-server in the patch passed.  |
   |  |   | 167m  7s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/12/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | unit javac javadoc shadedjars compile |
   | uname | Linux e77e69bdd10b 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 | HBASE-24666 / f67c3dfc5a |
   | Default Java | 2020-01-14 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/12/testReport/ |
   | Max. process+thread count | 4354 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-replication hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/12/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  HBASE-24666 passed  |
   | +1 :green_heart: |  compile  |   2m 28s |  HBASE-24666 passed  |
   | +1 :green_heart: |  shadedjars  |   6m 55s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 12s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 14s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 41s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 12s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 58s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 26s |  hbase-replication in the patch passed.  |
   | +1 :green_heart: |  unit  | 136m  0s |  hbase-server in the patch passed.  |
   |  |   | 170m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/13/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | unit javac javadoc shadedjars compile |
   | uname | Linux a7b8d5f50c49 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 | HBASE-24666 / f67c3dfc5a |
   | Default Java | 2020-01-14 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/13/testReport/ |
   | Max. process+thread count | 4471 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-replication hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/13/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 50s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 29s |  HBASE-24666 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 35s |  HBASE-24666 passed  |
   | +1 :green_heart: |  spotbugs  |   6m 44s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 10s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 22s |  hbase-server: The patch generated 1 new + 199 unchanged - 0 fixed = 200 total (was 199)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 11s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 53s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m 37s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 24s |  The patch does not generate ASF License warnings.  |
   |  |   |  55m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | dupname asflicense cc hbaseprotoc prototool spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux c52b9c9bf73f 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-24666 / 6008ffeb17 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/4/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 85 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/4/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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-protocol-shaded/src/main/protobuf/server/replication/ReplicationServer.proto
##########
@@ -24,9 +24,21 @@ option java_generic_services = true;
 option java_generate_equals_and_hash = true;

Review comment:
       Yes, in the core. The HReplicationServer will be a new role in one HBase cluster.




----------------------------------------------------------------
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 #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 10s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 16s |  HBASE-24666 passed  |
   | +1 :green_heart: |  compile  |   2m 23s |  HBASE-24666 passed  |
   | +1 :green_heart: |  shadedjars  |   6m 55s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-replication in HBASE-24666 failed.  |
   | -0 :warning: |  javadoc  |   0m 42s |  hbase-server in HBASE-24666 failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  8s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 33s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 33s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 46s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-replication in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 43s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 57s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 42s |  hbase-replication in the patch passed.  |
   | -1 :x: |  unit  | 144m 58s |  hbase-server in the patch failed.  |
   |  |   | 180m 32s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | unit javac javadoc shadedjars compile |
   | uname | Linux a6f8bfe570cc 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-24666 / 6008ffeb17 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-replication.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/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-2452/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-replication.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/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-2452/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-2452/1/testReport/ |
   | Max. process+thread count | 3826 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-replication hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/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] Apache-HBase commented on pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-24666 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  3s |  HBASE-24666 passed  |
   | +1 :green_heart: |  compile  |   1m 46s |  HBASE-24666 passed  |
   | +1 :green_heart: |  shadedjars  |   7m 11s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 49s |  HBASE-24666 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 44s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 44s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m  8s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 49s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 47s |  hbase-protocol-shaded in the patch passed.  |
   | -1 :x: |  unit  | 203m  6s |  hbase-server in the patch failed.  |
   |  |   | 234m 18s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2452 |
   | Optional Tests | unit javac javadoc shadedjars compile |
   | uname | Linux d1103807a612 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-24666 / 6008ffeb17 |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/3/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/3/testReport/ |
   | Max. process+thread count | 3269 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2452/3/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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
##########
@@ -360,8 +360,8 @@ private ReplicationSourceInterface createSource(String queueId, ReplicationPeer
     MetricsSource metrics = new MetricsSource(queueId);
     sourceMetrics.put(queueId, metrics);
     // init replication source
-    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, queueId, clusterId,
-      walFileLengthProvider, metrics);
+    src.init(conf, fs, logDir, this, queueStorage, replicationPeer, server, server.getServerName(),

Review comment:
       And one ReplicationSource is responsible for one replication queue.




----------------------------------------------------------------
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] infraio commented on a change in pull request #2452: HBASE-25071 ReplicationServer support start ReplicationSource internal

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -295,6 +296,7 @@ private synchronized AsyncClusterConnection getPeerConnection() throws IOExcepti
           .createBlockingRpcChannel(master, User.getCurrent(), shortOperationTimeout));
       ListReplicationSinkServersResponse resp = masterStub
         .listReplicationSinkServers(null, ListReplicationSinkServersRequest.newBuilder().build());
+      isReplicationServer = resp.getIsReplicationServer();

Review comment:
       This field is required. So not need to check hasReplicationServer first?




----------------------------------------------------------------
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