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 2021/12/02 04:51:15 UTC

[GitHub] [hbase] thangTang opened a new pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

thangTang opened a new pull request #3911:
URL: https://github.com/apache/hbase/pull/3911


   https://issues.apache.org/jira/browse/HBASE-26532


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  7s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 52s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 31s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 26s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  1s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 53s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 30s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 30s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 19s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 38s |  hbase-server generated 2 new + 21 unchanged - 0 fixed = 23 total (was 21)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 17s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 155m 19s |  hbase-server in the patch passed.  |
   |  |   | 190m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3911 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 887f8b346edf 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 | master / 19b0b2e8fc |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/3/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/3/testReport/ |
   | Max. process+thread count | 4757 (vs. ulimit of 30000) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/3/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 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.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] thangTang removed a comment on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

Posted by GitBox <gi...@apache.org>.
thangTang removed a comment on pull request #3911:
URL: https://github.com/apache/hbase/pull/3911#issuecomment-984519090


   @Reidddddd Could u pls help me to review this patch?


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Reidddddd commented on a change in pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -242,21 +276,150 @@ public boolean isAborted() {
     if (children == null) {
       return Collections.emptyList();
     }
+
+    Configuration conf = HBaseConfiguration.create();
+
+    /** if use other balancer, return all regionservers */
+    if (!conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS)
+      .equals(RSGroupBasedLoadBalancer.class.getName())
+      || hostServerName == null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication random choose policy...");
+      }
+      return parseServerNameFromList(children);
+    } else {
+      /** if use rsgroup balancer,
+       * just return regionservers belong to the same rsgroup or default rsgroup */
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication rsgroup choose policy...");
+      }
+      Map<String, String> serverNameHostPortMapping = new HashMap<>();
+      for (String serverName : children) {
+        String mappingKey =
+          serverName.split(",")[0] + ServerName.SERVERNAME_SEPARATOR + serverName.split(",")[1];
+        serverNameHostPortMapping.put(mappingKey, serverName);
+      }
+
+      String groupName = null;
+      RSGroupInfo rsGroupInfo = null;
+      try {
+        rsGroupInfo = getRSGroupInfoOfServer(conn.toConnection(), hostServerName.getAddress());
+      }catch (IOException e) {
+        e.printStackTrace();
+        LOG.error("rsGroupInfo error!", e);
+      }
+      if (rsGroupInfo != null) {
+        groupName = rsGroupInfo.getName();
+      }
+      try {
+        List<ServerName> serverList =
+          getGroupServerListFromTargetZkCluster(groupName, zkw, serverNameHostPortMapping);
+        if (serverList.size() > 0) {
+          // if target cluster open group balancer, serverList must has server(s)
+          LOG.debug("group list > 0");
+          threadLocal.get().getAndSet(true);
+          return serverList;
+        }
+        else {
+          // if not, choose sinkers from all regionservers
+          LOG.debug("target group list <= 0");

Review comment:
       ditto. Please add check `LOG.isDebugEnabled() {}` and elsewhere




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Reidddddd commented on a change in pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -242,21 +276,150 @@ public boolean isAborted() {
     if (children == null) {
       return Collections.emptyList();
     }
+
+    Configuration conf = HBaseConfiguration.create();
+
+    /** if use other balancer, return all regionservers */
+    if (!conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS)
+      .equals(RSGroupBasedLoadBalancer.class.getName())
+      || hostServerName == null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication random choose policy...");
+      }
+      return parseServerNameFromList(children);
+    } else {
+      /** if use rsgroup balancer,
+       * just return regionservers belong to the same rsgroup or default rsgroup */
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication rsgroup choose policy...");
+      }
+      Map<String, String> serverNameHostPortMapping = new HashMap<>();
+      for (String serverName : children) {
+        String mappingKey =
+          serverName.split(",")[0] + ServerName.SERVERNAME_SEPARATOR + serverName.split(",")[1];
+        serverNameHostPortMapping.put(mappingKey, serverName);
+      }
+
+      String groupName = null;
+      RSGroupInfo rsGroupInfo = null;
+      try {
+        rsGroupInfo = getRSGroupInfoOfServer(conn.toConnection(), hostServerName.getAddress());
+      }catch (IOException e) {
+        e.printStackTrace();
+        LOG.error("rsGroupInfo error!", e);
+      }
+      if (rsGroupInfo != null) {
+        groupName = rsGroupInfo.getName();
+      }
+      try {
+        List<ServerName> serverList =
+          getGroupServerListFromTargetZkCluster(groupName, zkw, serverNameHostPortMapping);
+        if (serverList.size() > 0) {
+          // if target cluster open group balancer, serverList must has server(s)
+          LOG.debug("group list > 0");
+          threadLocal.get().getAndSet(true);
+          return serverList;
+        }
+        else {
+          // if not, choose sinkers from all regionservers
+          LOG.debug("target group list <= 0");
+          return parseServerNameFromList(children);
+        }
+      }catch (IOException | KeeperException e) {
+        LOG.error("Get server list from target zk error", e);
+        return Collections.emptyList();
+      }
+    }
+  }
+
+  protected List<ServerName> parseServerNameFromList(List<String> children) {
+    if (children == null) {
+      return Collections.emptyList();
+    }
+    StringBuffer sb = new StringBuffer();

Review comment:
       Can use StringBuilder here?




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache9 commented on a change in pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -242,21 +276,150 @@ public boolean isAborted() {
     if (children == null) {
       return Collections.emptyList();
     }
+
+    Configuration conf = HBaseConfiguration.create();
+
+    /** if use other balancer, return all regionservers */
+    if (!conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS)
+      .equals(RSGroupBasedLoadBalancer.class.getName())
+      || hostServerName == null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication random choose policy...");
+      }
+      return parseServerNameFromList(children);
+    } else {
+      /** if use rsgroup balancer,
+       * just return regionservers belong to the same rsgroup or default rsgroup */
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication rsgroup choose policy...");
+      }
+      Map<String, String> serverNameHostPortMapping = new HashMap<>();
+      for (String serverName : children) {
+        String mappingKey =
+          serverName.split(",")[0] + ServerName.SERVERNAME_SEPARATOR + serverName.split(",")[1];
+        serverNameHostPortMapping.put(mappingKey, serverName);
+      }
+
+      String groupName = null;
+      RSGroupInfo rsGroupInfo = null;
+      try {
+        rsGroupInfo = getRSGroupInfoOfServer(conn.toConnection(), hostServerName.getAddress());
+      }catch (IOException e) {
+        e.printStackTrace();
+        LOG.error("rsGroupInfo error!", e);
+      }
+      if (rsGroupInfo != null) {
+        groupName = rsGroupInfo.getName();
+      }
+      try {
+        List<ServerName> serverList =
+          getGroupServerListFromTargetZkCluster(groupName, zkw, serverNameHostPortMapping);
+        if (serverList.size() > 0) {
+          // if target cluster open group balancer, serverList must has server(s)
+          LOG.debug("group list > 0");
+          threadLocal.get().getAndSet(true);
+          return serverList;
+        }
+        else {
+          // if not, choose sinkers from all regionservers
+          LOG.debug("target group list <= 0");
+          return parseServerNameFromList(children);
+        }
+      }catch (IOException | KeeperException e) {
+        LOG.error("Get server list from target zk error", e);
+        return Collections.emptyList();
+      }
+    }
+  }
+
+  protected List<ServerName> parseServerNameFromList(List<String> children) {
+    if (children == null) {
+      return Collections.emptyList();
+    }
+    StringBuffer sb = new StringBuffer();
     List<ServerName> addresses = new ArrayList<>(children.size());
     for (String child : children) {
       addresses.add(ServerName.parseServerName(child));
+      sb.append(ServerName.parseServerName(child)).append("/");
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+        "Find " + addresses.size() + " child znodes from target cluster zk. " + sb.toString());
     }
     return addresses;
   }
 
+  protected List<ServerName> getGroupServerListFromTargetZkCluster(String groupName,

Review comment:
       On master branch we do not use zk for storing the rs group any more...

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -77,9 +90,13 @@
    */
   public static final float DEFAULT_REPLICATION_SOURCE_RATIO = 0.5f;
 
+  static final float DEFAULT_REPLICATION_SOURCE_GROUP_RATIO = 1f;
+
   // Ratio of total number of potential peer region servers to be used
   private float ratio;
 
+  private float groupRatio;

Review comment:
       Yes, please add some explaination.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -88,6 +105,17 @@
 
   private List<ServerName> sinkServers = new ArrayList<>(0);
 
+  private static ThreadLocal<AtomicBoolean> threadLocal = new ThreadLocal<AtomicBoolean>() {

Review comment:
       We need to use AtomicBoolean for a thread local variable?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
##########
@@ -32,6 +32,7 @@
  */
 @InterfaceAudience.Private
 public interface ReplicationService {
+

Review comment:
       Please avoid touching unnecessary file.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -242,21 +276,150 @@ public boolean isAborted() {
     if (children == null) {
       return Collections.emptyList();
     }
+
+    Configuration conf = HBaseConfiguration.create();
+
+    /** if use other balancer, return all regionservers */
+    if (!conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS)

Review comment:
       And another problem is do we need to check the configuration here? I think it is the version at the source cluster, but we need to check the target cluster?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -242,21 +276,150 @@ public boolean isAborted() {
     if (children == null) {
       return Collections.emptyList();
     }
+
+    Configuration conf = HBaseConfiguration.create();
+
+    /** if use other balancer, return all regionservers */
+    if (!conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS)

Review comment:
       On master this is not the case now. The balancer will always be a RSGroupBasedLoadBalancer, if we do not enable rs group feature, DisabledRSGroupInfoManager will be used and there will be only one group, which will act as there is no rs group. So I think here we should check for RSGroupUtil.isRSGroupEnabled.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Reidddddd commented on a change in pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -242,21 +276,150 @@ public boolean isAborted() {
     if (children == null) {
       return Collections.emptyList();
     }
+
+    Configuration conf = HBaseConfiguration.create();
+
+    /** if use other balancer, return all regionservers */
+    if (!conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS)
+      .equals(RSGroupBasedLoadBalancer.class.getName())
+      || hostServerName == null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication random choose policy...");
+      }
+      return parseServerNameFromList(children);
+    } else {
+      /** if use rsgroup balancer,
+       * just return regionservers belong to the same rsgroup or default rsgroup */
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication rsgroup choose policy...");
+      }
+      Map<String, String> serverNameHostPortMapping = new HashMap<>();
+      for (String serverName : children) {
+        String mappingKey =
+          serverName.split(",")[0] + ServerName.SERVERNAME_SEPARATOR + serverName.split(",")[1];
+        serverNameHostPortMapping.put(mappingKey, serverName);
+      }
+
+      String groupName = null;
+      RSGroupInfo rsGroupInfo = null;
+      try {
+        rsGroupInfo = getRSGroupInfoOfServer(conn.toConnection(), hostServerName.getAddress());
+      }catch (IOException e) {
+        e.printStackTrace();
+        LOG.error("rsGroupInfo error!", e);
+      }
+      if (rsGroupInfo != null) {
+        groupName = rsGroupInfo.getName();
+      }
+      try {
+        List<ServerName> serverList =
+          getGroupServerListFromTargetZkCluster(groupName, zkw, serverNameHostPortMapping);
+        if (serverList.size() > 0) {
+          // if target cluster open group balancer, serverList must has server(s)
+          LOG.debug("group list > 0");
+          threadLocal.get().getAndSet(true);
+          return serverList;
+        }
+        else {
+          // if not, choose sinkers from all regionservers
+          LOG.debug("target group list <= 0");
+          return parseServerNameFromList(children);
+        }
+      }catch (IOException | KeeperException e) {
+        LOG.error("Get server list from target zk error", e);
+        return Collections.emptyList();
+      }
+    }
+  }
+
+  protected List<ServerName> parseServerNameFromList(List<String> children) {
+    if (children == null) {
+      return Collections.emptyList();
+    }
+    StringBuffer sb = new StringBuffer();
     List<ServerName> addresses = new ArrayList<>(children.size());
     for (String child : children) {
       addresses.add(ServerName.parseServerName(child));
+      sb.append(ServerName.parseServerName(child)).append("/");
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+        "Find " + addresses.size() + " child znodes from target cluster zk. " + sb.toString());
     }
     return addresses;
   }
 
+  protected List<ServerName> getGroupServerListFromTargetZkCluster(String groupName,
+    ZKWatcher zkw, Map<String, String> serverNameHostPortMapping)
+    throws KeeperException, IOException {
+    /** get group info from slave cluster zk */
+    List<String> groupInfos = ZKUtil.listChildrenAndWatchForNewChildren(
+        zkw, ZNodePaths.joinZNode(zkw.getZNodePaths().baseZNode, "rsgroup"));
+    /** if target cluster have same name group */
+    if(groupInfos == null){
+      if(LOG.isDebugEnabled()){
+        LOG.debug("groupInfos == null");
+      }
+      return Collections.emptyList();
+    }else{
+      if (groupInfos.size() > 0) {
+        if (groupInfos.contains(groupName)) {
+          return getServerListFromWithRSGroupName(groupName, zkw, serverNameHostPortMapping);
+        } else if (!groupInfos.contains(groupName)) {
+          /** if target cluster does not have same name group, return a empty list */
+          return Collections.emptyList();
+        }
+      } else {
+        /** if target cluster does not use group balancer, return a empty list */
+        return Collections.emptyList();
+      }
+    }
+
+    return Collections.emptyList();
+  }
+
+  protected List<ServerName> getServerListFromWithRSGroupName(
+    String groupName, ZKWatcher zkw, Map<String, String> serverNameHostPortMapping)
+    throws IOException {
+    List<ServerName> serverList = new ArrayList<>();
+    RSGroupInfo detail = retrieveGroupInfo(
+      zkw, ZNodePaths.joinZNode(zkw.getZNodePaths().baseZNode, "rsgroup"), groupName);
+    // choose server from rsZNode children which also in same group with local mashine
+    for (Address serverInfo : detail.getServers()) {
+      String serverPrefix =
+        serverInfo.getHostname() + ServerName.SERVERNAME_SEPARATOR + serverInfo.getPort();
+      if (serverNameHostPortMapping.containsKey(serverPrefix)) {
+        ServerName sn = ServerName.parseServerName(serverNameHostPortMapping.get(serverPrefix));
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Match server in " + groupName + " success " + serverPrefix + "/" + sn);
+        }
+        serverList.add(sn);
+      }
+    }
+    return serverList;
+  }
+
   protected synchronized void chooseSinks() {
     List<ServerName> slaveAddresses = fetchSlavesAddresses();
     if (slaveAddresses.isEmpty()) {
       LOG.warn("No sinks available at peer. Will not be able to replicate");
     }
     Collections.shuffle(slaveAddresses, ThreadLocalRandom.current());
-    int numSinks = (int) Math.ceil(slaveAddresses.size() * ratio);
+    float actualRatio=ratio;
+    if(getIsGroup() && conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, LoadBalancerFactory
+      .getDefaultLoadBalancerClass().getName()).equals(RSGroupBasedLoadBalancer.class.getName())
+      && hostServerName != null) {
+      actualRatio=groupRatio;
+    }
+
+    int numSinks = (int) Math.ceil(slaveAddresses.size() * actualRatio);
     this.sinkServers = slaveAddresses.subList(0, numSinks);
+    StringBuffer sb = new StringBuffer();

Review comment:
       ditto.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Reidddddd commented on a change in pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -88,6 +105,17 @@
 
   private List<ServerName> sinkServers = new ArrayList<>(0);
 
+  private static ThreadLocal<AtomicBoolean> threadLocal = new ThreadLocal<AtomicBoolean>() {
+    @Override
+    protected AtomicBoolean initialValue() {
+      return new AtomicBoolean(false);
+    }
+  };
+
+  public  boolean getIsGroup() {

Review comment:
       nit. redundant space




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 29s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 13s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 14s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 41s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 29s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 15s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 14s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 13s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 41s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 139m 33s |  hbase-server in the patch failed.  |
   |  |   | 171m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/4/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3911 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 531045b9408b 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 | master / 19b0b2e8fc |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/4/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-3911/4/testReport/ |
   | Max. process+thread count | 4234 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/4/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 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.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Reidddddd commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   ping @Apache9, you probably would want to take a look at this one? 


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] thangTang commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   > Seems there are still some legacy code, which is for the old rs group implementation. Better provide a simple design doc first, we can discuss the approach first, and then you can modify the PR here.
   > 
   > Thanks~
   
   Yes, actually i did this on 1.x...
   Try to upload a doc later.
   Thanks.


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 38s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 43s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 24s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  8s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 28s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 42s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 42s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 14s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  9s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 34s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  |   9m  2s |  hbase-server in the patch failed.  |
   |  |   |  44m 33s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3911 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux eeb96bb36c5d 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 | master / 19b0b2e8fc |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/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-3911/2/testReport/ |
   | Max. process+thread count | 825 (vs. ulimit of 30000) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 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.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Reidddddd commented on a change in pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -242,21 +276,150 @@ public boolean isAborted() {
     if (children == null) {
       return Collections.emptyList();
     }
+
+    Configuration conf = HBaseConfiguration.create();
+
+    /** if use other balancer, return all regionservers */
+    if (!conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS)
+      .equals(RSGroupBasedLoadBalancer.class.getName())
+      || hostServerName == null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication random choose policy...");
+      }
+      return parseServerNameFromList(children);
+    } else {
+      /** if use rsgroup balancer,
+       * just return regionservers belong to the same rsgroup or default rsgroup */
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication rsgroup choose policy...");
+      }
+      Map<String, String> serverNameHostPortMapping = new HashMap<>();
+      for (String serverName : children) {
+        String mappingKey =
+          serverName.split(",")[0] + ServerName.SERVERNAME_SEPARATOR + serverName.split(",")[1];
+        serverNameHostPortMapping.put(mappingKey, serverName);
+      }
+
+      String groupName = null;
+      RSGroupInfo rsGroupInfo = null;
+      try {
+        rsGroupInfo = getRSGroupInfoOfServer(conn.toConnection(), hostServerName.getAddress());
+      }catch (IOException e) {
+        e.printStackTrace();
+        LOG.error("rsGroupInfo error!", e);
+      }
+      if (rsGroupInfo != null) {
+        groupName = rsGroupInfo.getName();
+      }
+      try {
+        List<ServerName> serverList =
+          getGroupServerListFromTargetZkCluster(groupName, zkw, serverNameHostPortMapping);
+        if (serverList.size() > 0) {
+          // if target cluster open group balancer, serverList must has server(s)
+          LOG.debug("group list > 0");
+          threadLocal.get().getAndSet(true);
+          return serverList;
+        }
+        else {
+          // if not, choose sinkers from all regionservers
+          LOG.debug("target group list <= 0");
+          return parseServerNameFromList(children);
+        }
+      }catch (IOException | KeeperException e) {
+        LOG.error("Get server list from target zk error", e);
+        return Collections.emptyList();
+      }
+    }
+  }
+
+  protected List<ServerName> parseServerNameFromList(List<String> children) {
+    if (children == null) {
+      return Collections.emptyList();
+    }
+    StringBuffer sb = new StringBuffer();
     List<ServerName> addresses = new ArrayList<>(children.size());
     for (String child : children) {
       addresses.add(ServerName.parseServerName(child));
+      sb.append(ServerName.parseServerName(child)).append("/");
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+        "Find " + addresses.size() + " child znodes from target cluster zk. " + sb.toString());
     }
     return addresses;
   }
 
+  protected List<ServerName> getGroupServerListFromTargetZkCluster(String groupName,
+    ZKWatcher zkw, Map<String, String> serverNameHostPortMapping)
+    throws KeeperException, IOException {
+    /** get group info from slave cluster zk */
+    List<String> groupInfos = ZKUtil.listChildrenAndWatchForNewChildren(
+        zkw, ZNodePaths.joinZNode(zkw.getZNodePaths().baseZNode, "rsgroup"));
+    /** if target cluster have same name group */
+    if(groupInfos == null){
+      if(LOG.isDebugEnabled()){
+        LOG.debug("groupInfos == null");
+      }
+      return Collections.emptyList();
+    }else{

Review comment:
       nit. spaces between else {} are needed




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Reidddddd commented on a change in pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -242,21 +276,150 @@ public boolean isAborted() {
     if (children == null) {
       return Collections.emptyList();
     }
+
+    Configuration conf = HBaseConfiguration.create();
+
+    /** if use other balancer, return all regionservers */
+    if (!conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS)
+      .equals(RSGroupBasedLoadBalancer.class.getName())
+      || hostServerName == null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication random choose policy...");
+      }
+      return parseServerNameFromList(children);
+    } else {
+      /** if use rsgroup balancer,
+       * just return regionservers belong to the same rsgroup or default rsgroup */
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication rsgroup choose policy...");
+      }
+      Map<String, String> serverNameHostPortMapping = new HashMap<>();
+      for (String serverName : children) {
+        String mappingKey =
+          serverName.split(",")[0] + ServerName.SERVERNAME_SEPARATOR + serverName.split(",")[1];
+        serverNameHostPortMapping.put(mappingKey, serverName);
+      }
+
+      String groupName = null;
+      RSGroupInfo rsGroupInfo = null;
+      try {
+        rsGroupInfo = getRSGroupInfoOfServer(conn.toConnection(), hostServerName.getAddress());
+      }catch (IOException e) {
+        e.printStackTrace();
+        LOG.error("rsGroupInfo error!", e);
+      }
+      if (rsGroupInfo != null) {
+        groupName = rsGroupInfo.getName();
+      }
+      try {
+        List<ServerName> serverList =
+          getGroupServerListFromTargetZkCluster(groupName, zkw, serverNameHostPortMapping);
+        if (serverList.size() > 0) {
+          // if target cluster open group balancer, serverList must has server(s)
+          LOG.debug("group list > 0");
+          threadLocal.get().getAndSet(true);
+          return serverList;
+        }
+        else {
+          // if not, choose sinkers from all regionservers
+          LOG.debug("target group list <= 0");
+          return parseServerNameFromList(children);
+        }
+      }catch (IOException | KeeperException e) {
+        LOG.error("Get server list from target zk error", e);
+        return Collections.emptyList();
+      }
+    }
+  }
+
+  protected List<ServerName> parseServerNameFromList(List<String> children) {
+    if (children == null) {
+      return Collections.emptyList();
+    }
+    StringBuffer sb = new StringBuffer();
     List<ServerName> addresses = new ArrayList<>(children.size());
     for (String child : children) {
       addresses.add(ServerName.parseServerName(child));
+      sb.append(ServerName.parseServerName(child)).append("/");
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+        "Find " + addresses.size() + " child znodes from target cluster zk. " + sb.toString());

Review comment:
       Please use parameterized logging, and elsewhere.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Reidddddd commented on a change in pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -77,9 +90,13 @@
    */
   public static final float DEFAULT_REPLICATION_SOURCE_RATIO = 0.5f;
 
+  static final float DEFAULT_REPLICATION_SOURCE_GROUP_RATIO = 1f;
+
   // Ratio of total number of potential peer region servers to be used
   private float ratio;
 
+  private float groupRatio;

Review comment:
       here can add some descriptions/comments as well




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Reidddddd commented on a change in pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -242,21 +276,150 @@ public boolean isAborted() {
     if (children == null) {
       return Collections.emptyList();
     }
+
+    Configuration conf = HBaseConfiguration.create();
+
+    /** if use other balancer, return all regionservers */
+    if (!conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS)
+      .equals(RSGroupBasedLoadBalancer.class.getName())
+      || hostServerName == null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication random choose policy...");
+      }
+      return parseServerNameFromList(children);
+    } else {
+      /** if use rsgroup balancer,
+       * just return regionservers belong to the same rsgroup or default rsgroup */
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication rsgroup choose policy...");
+      }
+      Map<String, String> serverNameHostPortMapping = new HashMap<>();
+      for (String serverName : children) {
+        String mappingKey =
+          serverName.split(",")[0] + ServerName.SERVERNAME_SEPARATOR + serverName.split(",")[1];
+        serverNameHostPortMapping.put(mappingKey, serverName);
+      }
+
+      String groupName = null;
+      RSGroupInfo rsGroupInfo = null;
+      try {
+        rsGroupInfo = getRSGroupInfoOfServer(conn.toConnection(), hostServerName.getAddress());
+      }catch (IOException e) {
+        e.printStackTrace();

Review comment:
       still need`e.printStackTrace();`? 




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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






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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   3m 11s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 35s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 23s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 11s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 13s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 20s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 23s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 10s |  hbase-server: The patch generated 2 new + 7 unchanged - 1 fixed = 9 total (was 8)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  21m 39s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 24s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 14s |  The patch does not generate ASF License warnings.  |
   |  |   |  56m 32s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3911 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 74a6e7e79b9e 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 19b0b2e8fc |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/4/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/4/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 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.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   :confetti_ball: **+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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 14s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  3s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 19s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 58s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  3s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  3s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 36s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 149m 14s |  hbase-server in the patch passed.  |
   |  |   | 180m 17s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/4/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3911 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux fca40b4dfb59 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 | master / 19b0b2e8fc |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/4/testReport/ |
   | Max. process+thread count | 4639 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/4/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 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.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Reidddddd commented on a change in pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -242,21 +276,150 @@ public boolean isAborted() {
     if (children == null) {
       return Collections.emptyList();
     }
+
+    Configuration conf = HBaseConfiguration.create();
+
+    /** if use other balancer, return all regionservers */
+    if (!conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS)
+      .equals(RSGroupBasedLoadBalancer.class.getName())
+      || hostServerName == null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication random choose policy...");
+      }
+      return parseServerNameFromList(children);
+    } else {
+      /** if use rsgroup balancer,
+       * just return regionservers belong to the same rsgroup or default rsgroup */
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication rsgroup choose policy...");
+      }
+      Map<String, String> serverNameHostPortMapping = new HashMap<>();
+      for (String serverName : children) {

Review comment:
       can reuse parseServerNameFromList(children) ?  then the followings can just user ServerName#getHost and ServerName#getPort.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  2s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 53s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 34s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 36s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  4s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 32s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 32s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 27s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server generated 2 new + 21 unchanged - 0 fixed = 23 total (was 21)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 18s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  |   8m 21s |  hbase-server in the patch failed.  |
   |  |   |  43m  1s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3911 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 388fe08050f6 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 | master / 19b0b2e8fc |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/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-3911/2/testReport/ |
   | Max. process+thread count | 1036 (vs. ulimit of 30000) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 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.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Reidddddd commented on a change in pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
##########
@@ -242,21 +276,150 @@ public boolean isAborted() {
     if (children == null) {
       return Collections.emptyList();
     }
+
+    Configuration conf = HBaseConfiguration.create();
+
+    /** if use other balancer, return all regionservers */
+    if (!conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS)
+      .equals(RSGroupBasedLoadBalancer.class.getName())
+      || hostServerName == null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication random choose policy...");
+      }
+      return parseServerNameFromList(children);
+    } else {
+      /** if use rsgroup balancer,
+       * just return regionservers belong to the same rsgroup or default rsgroup */
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Use replication rsgroup choose policy...");
+      }
+      Map<String, String> serverNameHostPortMapping = new HashMap<>();
+      for (String serverName : children) {
+        String mappingKey =
+          serverName.split(",")[0] + ServerName.SERVERNAME_SEPARATOR + serverName.split(",")[1];
+        serverNameHostPortMapping.put(mappingKey, serverName);
+      }
+
+      String groupName = null;
+      RSGroupInfo rsGroupInfo = null;
+      try {
+        rsGroupInfo = getRSGroupInfoOfServer(conn.toConnection(), hostServerName.getAddress());
+      }catch (IOException e) {
+        e.printStackTrace();
+        LOG.error("rsGroupInfo error!", e);
+      }
+      if (rsGroupInfo != null) {
+        groupName = rsGroupInfo.getName();
+      }
+      try {
+        List<ServerName> serverList =
+          getGroupServerListFromTargetZkCluster(groupName, zkw, serverNameHostPortMapping);
+        if (serverList.size() > 0) {
+          // if target cluster open group balancer, serverList must has server(s)
+          LOG.debug("group list > 0");

Review comment:
       can log which group > 0?  




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  7s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 57s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 22s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 55s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   4m  2s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 58s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 13s |  the patch passed  |
   | +1 :green_heart: |  javac  |   5m 13s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 34s |  hbase-client: The patch generated 2 new + 17 unchanged - 0 fixed = 19 total (was 17)  |
   | -0 :warning: |  checkstyle  |   1m 21s |  hbase-server: The patch generated 16 new + 8 unchanged - 0 fixed = 24 total (was 8)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  24m 50s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | -1 :x: |  spotbugs  |   1m 39s |  hbase-client generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | -1 :x: |  spotbugs  |   3m  5s |  hbase-server generated 3 new + 0 unchanged - 0 fixed = 3 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 27s |  The patch does not generate ASF License warnings.  |
   |  |   |  70m 28s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-client |
   |  |  Write to static field org.apache.hadoop.hbase.zookeeper.ZNodePaths.groupInfo from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:[line 122] |
   |  |  Write to static field org.apache.hadoop.hbase.zookeeper.ZNodePaths.rsgroup from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:[line 124] |
   | FindBugs | module:hbase-server |
   |  |  Possible null pointer dereference of context in org.apache.hadoop.hbase.replication.BaseReplicationEndpoint.init(ReplicationEndpoint$Context)  Dereferenced at BaseReplicationEndpoint.java:context in org.apache.hadoop.hbase.replication.BaseReplicationEndpoint.init(ReplicationEndpoint$Context)  Dereferenced at BaseReplicationEndpoint.java:[line 60] |
   |  |  Write to static field org.apache.hadoop.hbase.replication.BaseReplicationEndpoint.hostServerName from instance method org.apache.hadoop.hbase.replication.BaseReplicationEndpoint.init(ReplicationEndpoint$Context)  At BaseReplicationEndpoint.java:from instance method org.apache.hadoop.hbase.replication.BaseReplicationEndpoint.init(ReplicationEndpoint$Context)  At BaseReplicationEndpoint.java:[line 60] |
   |  |  Potentially dangerous use of non-short-circuit logic in org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint.chooseSinks()  At HBaseReplicationEndpoint.java:logic in org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint.chooseSinks()  At HBaseReplicationEndpoint.java:[line 411] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3911 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 1fd3878a242f 4.15.0-147-generic #151-Ubuntu SMP Fri Jun 18 19:21:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 19b0b2e8fc |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/artifact/yetus-general-check/output/new-spotbugs-hbase-client.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/2/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 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.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] thangTang commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   @Reidddddd Could u pls help me to review this patch?


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 12s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   6m 10s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 20s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 55s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   4m  1s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   2m 22s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 53s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 53s |  hbase-server in the patch failed.  |
   | -0 :warning: |  checkstyle  |   0m 34s |  hbase-client: The patch generated 2 new + 17 unchanged - 0 fixed = 19 total (was 17)  |
   | -0 :warning: |  checkstyle  |   1m 19s |  hbase-server: The patch generated 18 new + 8 unchanged - 0 fixed = 26 total (was 8)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | -1 :x: |  hadoopcheck  |   2m 37s |  The patch causes 16 errors with Hadoop v3.1.2.  |
   | -1 :x: |  hadoopcheck  |   5m 25s |  The patch causes 16 errors with Hadoop v3.2.2.  |
   | -1 :x: |  hadoopcheck  |   8m 24s |  The patch causes 16 errors with Hadoop v3.3.1.  |
   | -1 :x: |  spotbugs  |   1m 34s |  hbase-client generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | -1 :x: |  spotbugs  |   0m 45s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 24s |  The patch does not generate ASF License warnings.  |
   |  |   |  38m 59s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-client |
   |  |  Write to static field org.apache.hadoop.hbase.zookeeper.ZNodePaths.groupInfo from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:[line 122] |
   |  |  Write to static field org.apache.hadoop.hbase.zookeeper.ZNodePaths.rsgroup from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:[line 124] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3911 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 3e3abe2d52fe 4.15.0-147-generic #151-Ubuntu SMP Fri Jun 18 19:21:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 19b0b2e8fc |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/artifact/yetus-general-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/artifact/yetus-general-check/output/patch-javac-3.1.2.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/artifact/yetus-general-check/output/patch-javac-3.2.2.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/artifact/yetus-general-check/output/patch-javac-3.3.1.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/artifact/yetus-general-check/output/new-spotbugs-hbase-client.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/artifact/yetus-general-check/output/patch-spotbugs-hbase-server.txt |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/1/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 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.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3911: HBASE-26532 Replication could choose the same named group if it is exist in the target cluster

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  7s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 53s |  master passed  |
   | +1 :green_heart: |  compile  |   5m 15s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 54s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   4m  0s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m  5s |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 22s |  the patch passed  |
   | +1 :green_heart: |  javac  |   5m 22s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 34s |  hbase-client: The patch generated 2 new + 17 unchanged - 0 fixed = 19 total (was 17)  |
   | -0 :warning: |  checkstyle  |   1m 30s |  hbase-server: The patch generated 17 new + 8 unchanged - 0 fixed = 25 total (was 8)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  24m 44s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | -1 :x: |  spotbugs  |   1m 37s |  hbase-client generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | -1 :x: |  spotbugs  |   3m  4s |  hbase-server generated 3 new + 0 unchanged - 0 fixed = 3 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 26s |  The patch does not generate ASF License warnings.  |
   |  |   |  70m 14s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-client |
   |  |  Write to static field org.apache.hadoop.hbase.zookeeper.ZNodePaths.groupInfo from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:[line 122] |
   |  |  Write to static field org.apache.hadoop.hbase.zookeeper.ZNodePaths.rsgroup from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:from instance method new org.apache.hadoop.hbase.zookeeper.ZNodePaths(Configuration)  At ZNodePaths.java:[line 124] |
   | FindBugs | module:hbase-server |
   |  |  Possible null pointer dereference of context in org.apache.hadoop.hbase.replication.BaseReplicationEndpoint.init(ReplicationEndpoint$Context)  Dereferenced at BaseReplicationEndpoint.java:context in org.apache.hadoop.hbase.replication.BaseReplicationEndpoint.init(ReplicationEndpoint$Context)  Dereferenced at BaseReplicationEndpoint.java:[line 60] |
   |  |  Write to static field org.apache.hadoop.hbase.replication.BaseReplicationEndpoint.hostServerName from instance method org.apache.hadoop.hbase.replication.BaseReplicationEndpoint.init(ReplicationEndpoint$Context)  At BaseReplicationEndpoint.java:from instance method org.apache.hadoop.hbase.replication.BaseReplicationEndpoint.init(ReplicationEndpoint$Context)  At BaseReplicationEndpoint.java:[line 60] |
   |  |  Potentially dangerous use of non-short-circuit logic in org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint.chooseSinks()  At HBaseReplicationEndpoint.java:logic in org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint.chooseSinks()  At HBaseReplicationEndpoint.java:[line 412] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3911 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 779a443e8ed1 4.15.0-147-generic #151-Ubuntu SMP Fri Jun 18 19:21:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 19b0b2e8fc |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/3/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/3/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/3/artifact/yetus-general-check/output/new-spotbugs-hbase-client.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/3/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3911/3/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 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.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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