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/10/29 09:56:56 UTC

[GitHub] [hbase] ddupg opened a new pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

ddupg opened a new pull request #3806:
URL: https://github.com/apache/hbase/pull/3806


   Failed to add a peer replicating to non-HBase(like MQ) by implementing custom ReplicationEndpoint,  got exception like this in my UT: 
   ```
   2021-10-29T15:14:47,632 INFO  [RPCClient-NioEventLoopGroup-5-3] client.RawAsyncHBaseAdmin$ReplicationProcedureBiConsumer(2761): Operation: ADD_REPLICATION_PEER, peerId: 1 failed with Invalid cluster key: , should not replicate to itself for HBaseInterClusterReplicationEndpoint2021-10-29T15:14:47,632 INFO  [RPCClient-NioEventLoopGroup-5-3] client.RawAsyncHBaseAdmin$ReplicationProcedureBiConsumer(2761): Operation: ADD_REPLICATION_PEER, peerId: 1 failed with Invalid cluster key: , should not replicate to itself for HBaseInterClusterReplicationEndpoint
   org.apache.hadoop.hbase.DoNotRetryIOException: Invalid cluster key: , should not replicate to itself for HBaseInterClusterReplicationEndpoint
    at java.lang.Thread.getStackTrace(Thread.java:1559)
    at org.apache.hadoop.hbase.util.FutureUtils.setStackTrace(FutureUtils.java:130)
    at org.apache.hadoop.hbase.util.FutureUtils.rethrow(FutureUtils.java:149)
    at org.apache.hadoop.hbase.util.FutureUtils.get(FutureUtils.java:186)
    at org.apache.hadoop.hbase.client.Admin.addReplicationPeer(Admin.java:1948)
    at org.apache.hadoop.hbase.client.Admin.addReplicationPeer(Admin.java:1936)
    at org.apache.hadoop.hbase.replication.TestNonHBaseReplicationEndpoint.test(TestNonHBaseReplicationEndpoint.java:97)
    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    at java.lang.reflect.Method.invoke(Method.java:498)
    at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
    at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
    at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
    at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
    at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
    at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
    at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
    at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
    at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
    at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
    at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
    at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
    at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
    at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
    at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
    at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
    at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
    at org.apache.hadoop.hbase.SystemExitRule$1.evaluate(SystemExitRule.java:38)
    at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:288)
    at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:282)
    at java.util.concurrent.FutureTask.run(FutureTask.java:266)
    at java.lang.Thread.run(Thread.java:748)
    at --------Future.get--------(Unknown Source)
    at org.apache.hadoop.hbase.master.replication.ReplicationPeerManager.checkClusterId(ReplicationPeerManager.java:527)
    at org.apache.hadoop.hbase.master.replication.ReplicationPeerManager.checkPeerConfig(ReplicationPeerManager.java:367)
    at org.apache.hadoop.hbase.master.replication.ReplicationPeerManager.preAddPeer(ReplicationPeerManager.java:123)
    at org.apache.hadoop.hbase.master.replication.AddPeerProcedure.prePeerModification(AddPeerProcedure.java:101)
    at org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure.executeFromState(ModifyPeerProcedure.java:162)
    at org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure.executeFromState(ModifyPeerProcedure.java:43)
    at org.apache.hadoop.hbase.procedure2.StateMachineProcedure.execute(StateMachineProcedure.java:190)
    at org.apache.hadoop.hbase.procedure2.Procedure.doExecute(Procedure.java:953)
    at org.apache.hadoop.hbase.procedure2.ProcedureExecutor.execProcedure(ProcedureExecutor.java:1667)
    at org.apache.hadoop.hbase.procedure2.ProcedureExecutor.executeProcedure(ProcedureExecutor.java:1414)
    at org.apache.hadoop.hbase.procedure2.ProcedureExecutor.access$1100(ProcedureExecutor.java:78)
    at org.apache.hadoop.hbase.procedure2.ProcedureExecutor$WorkerThread.run(ProcedureExecutor.java:1981)
   ```
   HBASE-24743 ignored this situation and introduced this bug.


-- 
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] ddupg commented on a change in pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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



##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java
##########
@@ -0,0 +1,145 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ MediumTests.class, ReplicationTests.class })
+public class TestNonHBaseReplicationEndpoint {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestNonHBaseReplicationEndpoint.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestNonHBaseReplicationEndpoint.class);
+
+  private static final HBaseTestingUtil UTIL = new HBaseTestingUtil();
+
+  private static Admin ADMIN;
+
+  private static final TableName tableName = TableName.valueOf("test");
+  private static final byte[] famName = Bytes.toBytes("f");
+
+  private static final AtomicBoolean REPLICATED = new AtomicBoolean();
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    UTIL.startMiniCluster();
+    ADMIN = UTIL.getAdmin();
+  }
+
+  @AfterClass
+  public static void teardownAfterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setup() {
+    REPLICATED.set(false);
+  }
+
+  @Test
+  public void test() throws IOException {
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(famName)
+        .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
+      .build();
+    Table table = UTIL.createTable(td, HBaseTestingUtil.KEYS_FOR_HBA_CREATE_TABLE);
+
+    ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder()
+      .setReplicationEndpointImpl(NonHBaseReplicationEndpoint.class.getName())
+      .setReplicateAllUserTables(false)
+      .setTableCFsMap(new HashMap<TableName, List<String>>() {{
+        put(tableName, new ArrayList<>());
+      }}).build();
+
+    ADMIN.addReplicationPeer("1", peerConfig);
+    loadData(table);
+
+    UTIL.waitFor(10000L, () -> REPLICATED.get());
+  }
+
+  protected static void loadData(Table table) throws IOException {
+    for (int i = 0; i < 100; i++) {
+      Put put = new Put(Bytes.toBytes(Integer.toString(i)));
+      put.addColumn(famName, famName, Bytes.toBytes(i));
+      table.put(put);
+    }
+  }
+
+  public static class NonHBaseReplicationEndpoint extends BaseReplicationEndpoint {

Review comment:
       The answer should be in HBASE-15982, made HBaseReplicationEndpoint and BaseReplicationEndpoint as Private since want to internalize Guava's API, so custom Endpoint should implement `ReplicationEndpoint`. 
   I was not aware of this problem before, I should make NonHBaseReplicationEndpoint implement `ReplicationEndpoint`.




-- 
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 #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   :confetti_ball: **+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 _ |
   | +1 :green_heart: |  mvninstall  |   5m  4s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 19s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m  4s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 42s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m  4s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m  4s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 44s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 205m 33s |  hbase-server in the patch passed.  |
   |  |   | 240m 34s |   |
   
   
   | 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-3806/3/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3806 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux cef40cbb4104 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 / ff11f1115f |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3806/3/testReport/ |
   | Max. process+thread count | 2507 (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-3806/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] Apache-HBase commented on pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  8s |  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 55s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 13s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m 49s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 43s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 44s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 13s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 13s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m 47s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 243m 34s |  hbase-server in the patch failed.  |
   |  |   | 279m 55s |   |
   
   
   | 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-3806/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3806 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux de9a59cbb321 4.15.0-142-generic #146-Ubuntu SMP Tue Apr 13 01:11:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e0eaec6813 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3806/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-3806/1/testReport/ |
   | Max. process+thread count | 2863 (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-3806/1/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] ddupg commented on a change in pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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



##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java
##########
@@ -0,0 +1,145 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ MediumTests.class, ReplicationTests.class })
+public class TestNonHBaseReplicationEndpoint {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestNonHBaseReplicationEndpoint.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestNonHBaseReplicationEndpoint.class);

Review comment:
       OK, I'll remove the `LOG`




-- 
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 #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  4s |  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 59s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 12s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m 47s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 46s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 43s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 14s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 14s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m 51s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 238m 52s |  hbase-server in the patch passed.  |
   |  |   | 275m  7s |   |
   
   
   | 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-3806/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3806 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux b6164e814746 4.15.0-142-generic #146-Ubuntu SMP Tue Apr 13 01:11:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ff11f1115f |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3806/2/testReport/ |
   | Max. process+thread count | 2827 (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-3806/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] Apache-HBase commented on pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 25s |  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  8s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 14s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m  3s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m  4s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 49s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 12s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 12s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  3s |  hbase-server: The patch generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  19m 17s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  spotbugs  |   2m 16s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 16s |  The patch does not generate ASF License warnings.  |
   |  |   |  48m 47s |   |
   
   
   | 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-3806/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3806 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 60f353440fb4 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 / ff11f1115f |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3806/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 95 (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-3806/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] shahrs87 commented on a change in pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -358,13 +358,13 @@ private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws DoNotRetry
           e);
       }
     }
-    // Default is HBaseInterClusterReplicationEndpoint and only it need to check cluster key
-    if (endpoint == null || endpoint instanceof HBaseInterClusterReplicationEndpoint) {
+    // Endpoints implementing HBaseReplicationEndpoint need to check cluster key
+    if (endpoint == null || endpoint instanceof HBaseReplicationEndpoint) {

Review comment:
       As for this issue, it doesn't matter whether it is an instance of `HBaseInterClusterReplicationEndpoint` or `HBaseReplicationEndpoint`, correct ? since the test is skipping the check completely. 




-- 
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 #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  1s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  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  |   5m  6s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 19s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m  7s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 43s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m  4s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 19s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 19s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m  5s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 205m 47s |  hbase-server in the patch passed.  |
   |  |   | 240m 53s |   |
   
   
   | 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-3806/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3806 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 5fd6a8da5a45 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 / ff11f1115f |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3806/2/testReport/ |
   | Max. process+thread count | 2623 (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-3806/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] Apache-HBase commented on pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 39s |  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  |   6m 59s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 55s |  master passed  |
   | +1 :green_heart: |  shadedjars  |  10m 30s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 51s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 51s |  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  |  11m  2s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 51s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 228m 15s |  hbase-server in the patch passed.  |
   |  |   | 271m 36s |   |
   
   
   | 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-3806/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3806 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 79506bf4ec14 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 / e0eaec6813 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3806/1/testReport/ |
   | Max. process+thread count | 2581 (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-3806/1/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] Apache9 commented on a change in pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -358,13 +358,13 @@ private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws DoNotRetry
           e);
       }
     }
-    // Default is HBaseInterClusterReplicationEndpoint and only it need to check cluster key
-    if (endpoint == null || endpoint instanceof HBaseInterClusterReplicationEndpoint) {
+    // Endpoints implementing HBaseReplicationEndpoint need to check cluster key
+    if (endpoint == null || endpoint instanceof HBaseReplicationEndpoint) {

Review comment:
       Sound reasonable.




-- 
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 pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   Please fix the checkstyle issue before merging.


-- 
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 #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  3s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 22s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m  4s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 16s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m  2s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 216m 26s |  hbase-server in the patch failed.  |
   |  |   | 249m 19s |   |
   
   
   | 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-3806/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3806 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 2597ed89f969 4.15.0-142-generic #146-Ubuntu SMP Tue Apr 13 01:11:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ff11f1115f |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3806/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-3806/3/testReport/ |
   | Max. process+thread count | 2738 (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-3806/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] Apache-HBase commented on pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  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 24s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 34s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m  2s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m  5s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 19s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 19s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  7s |  hbase-server: The patch generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  20m 32s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  spotbugs  |   2m 27s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 14s |  The patch does not generate ASF License warnings.  |
   |  |   |  52m  3s |   |
   
   
   | 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-3806/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3806 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 3c479afc1c00 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 / e0eaec6813 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3806/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 96 (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-3806/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] shahrs87 commented on a change in pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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



##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java
##########
@@ -0,0 +1,145 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ MediumTests.class, ReplicationTests.class })
+public class TestNonHBaseReplicationEndpoint {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestNonHBaseReplicationEndpoint.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestNonHBaseReplicationEndpoint.class);

Review comment:
       nit: LOG not used anywhere.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -510,7 +510,7 @@ private void checkClusterKey(String clusterKey) throws DoNotRetryIOException {
     }
   }
 
-  private void checkClusterId(String clusterKey) throws DoNotRetryIOException {
+  private void checkSameClusterKey(String clusterKey) throws DoNotRetryIOException {

Review comment:
       This method is checking whether cluster id is same or not, so IMO the method name is correct. We can change it to `checkSameClusterId` if you like.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -358,13 +358,13 @@ private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws DoNotRetry
           e);
       }
     }
-    // Default is HBaseInterClusterReplicationEndpoint and only it need to check cluster key
-    if (endpoint == null || endpoint instanceof HBaseInterClusterReplicationEndpoint) {
+    // Endpoints implementing HBaseReplicationEndpoint need to check cluster key
+    if (endpoint == null || endpoint instanceof HBaseReplicationEndpoint) {

Review comment:
       As for this issue, it doesn't matter whether it is an instance of HBaseReplicationEndpoint or HBaseReplicationEndpoint, correct ? since the test is skipping the check completely. 

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java
##########
@@ -0,0 +1,145 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ MediumTests.class, ReplicationTests.class })
+public class TestNonHBaseReplicationEndpoint {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestNonHBaseReplicationEndpoint.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestNonHBaseReplicationEndpoint.class);
+
+  private static final HBaseTestingUtil UTIL = new HBaseTestingUtil();
+
+  private static Admin ADMIN;
+
+  private static final TableName tableName = TableName.valueOf("test");
+  private static final byte[] famName = Bytes.toBytes("f");
+
+  private static final AtomicBoolean REPLICATED = new AtomicBoolean();
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    UTIL.startMiniCluster();
+    ADMIN = UTIL.getAdmin();
+  }
+
+  @AfterClass
+  public static void teardownAfterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setup() {
+    REPLICATED.set(false);
+  }
+
+  @Test
+  public void test() throws IOException {
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(famName)
+        .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
+      .build();
+    Table table = UTIL.createTable(td, HBaseTestingUtil.KEYS_FOR_HBA_CREATE_TABLE);
+
+    ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder()
+      .setReplicationEndpointImpl(NonHBaseReplicationEndpoint.class.getName())
+      .setReplicateAllUserTables(false)
+      .setTableCFsMap(new HashMap<TableName, List<String>>() {{
+        put(tableName, new ArrayList<>());
+      }}).build();
+
+    ADMIN.addReplicationPeer("1", peerConfig);
+    loadData(table);
+
+    UTIL.waitFor(10000L, () -> REPLICATED.get());
+  }
+
+  protected static void loadData(Table table) throws IOException {
+    for (int i = 0; i < 100; i++) {
+      Put put = new Put(Bytes.toBytes(Integer.toString(i)));
+      put.addColumn(famName, famName, Bytes.toBytes(i));
+      table.put(put);
+    }
+  }
+
+  public static class NonHBaseReplicationEndpoint extends BaseReplicationEndpoint {

Review comment:
       Just curious, BaseReplicationEndpoint's InterfaceAudience is Private and so is HBaseInterClusterReplicationEndpoint and HBaseReplicationEndpoint's IA. If anyone wants to implement its custom ReplicationEndPoint, should we think to make it LimitedPrivate with HBaseInterfaceAudience as REPLICATION.




-- 
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 #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  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 48s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 56s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 16s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   3m  1s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 22s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 16s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 16s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  19m  6s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  spotbugs  |   2m 16s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 13s |  The patch does not generate ASF License warnings.  |
   |  |   |  51m 51s |   |
   
   
   | 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-3806/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3806 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 6c5cb6ca4541 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 / ff11f1115f |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | Max. process+thread count | 96 (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-3806/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



[GitHub] [hbase] ddupg merged pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   


-- 
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] ddupg merged pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   


-- 
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] ddupg merged pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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


   


-- 
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] ddupg commented on a change in pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -358,13 +358,13 @@ private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws DoNotRetry
           e);
       }
     }
-    // Default is HBaseInterClusterReplicationEndpoint and only it need to check cluster key
-    if (endpoint == null || endpoint instanceof HBaseInterClusterReplicationEndpoint) {
+    // Endpoints implementing HBaseReplicationEndpoint need to check cluster key
+    if (endpoint == null || endpoint instanceof HBaseReplicationEndpoint) {

Review comment:
       The reason I modified here is that the target of HBaseReplicationEndpoint is an HBase cluster, for all its implementations including HBaseInterClusterReplicationEndpoint, we should check clusterKey.




-- 
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] ddupg commented on a change in pull request #3806: HBASE-26406 Can not add peer replicating to non-HBase

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -510,7 +510,7 @@ private void checkClusterKey(String clusterKey) throws DoNotRetryIOException {
     }
   }
 
-  private void checkClusterId(String clusterKey) throws DoNotRetryIOException {
+  private void checkSameClusterKey(String clusterKey) throws DoNotRetryIOException {

Review comment:
       The original method name `checkClusterId` is inappropriate, clusterId and clusterKey are not the same thing in replication, So I kept the same style as the above method `checkClusterKey` and renamed it `checkSameClusterKey`.




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