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 2022/12/12 23:46:57 UTC

[GitHub] [hbase] virajjasani opened a new pull request, #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

virajjasani opened a new pull request, #4924:
URL: https://github.com/apache/hbase/pull/4924

   HBase provides coproc ability to enhance WALKey attributes (a.k.a. WAL annotations) in order for the replication sink cluster to build required metadata with the mutations. The endpoint is preWALAppend(). This ability was provided by HBASE-22622. The map of extended attributes is optional and hence not directly used by hbase internally. 
   
   For any hbase downstreamers to build CDC (Change Data Capture) like functionality, it might required additional metadata in addition to the ones being used by hbase already (replication scope, list of cluster ids, seq id, table name, region id etc). For instance, Phoenix uses many additional attributes like tenant id, schema name, table type etc.
   We already have this extended map of attributed available in WAL protobuf, to provide us the capability to (de)serialize it. While creating new ReplicateWALEntryRequest from the list of WAL entires, we are able to serialize the additional attributes. Similarly, at the replication sink side, the deserialized WALEntry has the extended attributed available.
   
   At the sink cluster, we should be able to attach the deserialized extended attributes to the newly generated mutations so that the peer cluster can utilize the mutation attributes to re-build required metadata.


-- 
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] virajjasani commented on pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on PR #4924:
URL: https://github.com/apache/hbase/pull/4924#issuecomment-1371339786

   > I think introducing a default conversion from WAL entry attribute to mutation attribute would sufficient
   
   This is exactly what the purpose of this patch was initially.
   
   
   > What I would not like to see here is a new config. It is quite use case specific and the accumulation of these makes problems due to configuration errors all the more likely.
   
   I agree here and that's why I kept the initial version without including any configs.
   
   
   > but to @Apache9 's point, a symmetric flow here would have a pair of coprocessors involved: the first transforming attribute to metadata at the source client, the latter transforming metadata to attribute at the sink client.
   
   Yeah this also doesn't seem bad idea @Apache9.


-- 
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] apurtell commented on a diff in pull request #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
apurtell commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1067359503


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java:
##########
@@ -137,4 +140,33 @@ default void preExecuteProcedures(ObserverContext<RegionServerCoprocessorEnviron
   default void postExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
     throws IOException {
   }
+
+  /**
+   * This will be called before replication sink mutations are executed on the sink table as part of
+   * batch call.
+   * @param ctx      the environment to interact with the framework and region server.
+   * @param walEntry wal entry from which mutation is formed.
+   * @param mutation mutation to be applied at sink cluster.
+   * @throws IOException if something goes wrong.
+   */
+  default void preReplicationSinkBatchMutate(
+    ObserverContext<RegionServerCoprocessorEnvironment> ctx, AdminProtos.WALEntry walEntry,

Review Comment:
   I wish we didn't have these protobuf types in the LimitedPrivate coprocessor API. We are going to have to live within compatibility constraints. However it would be prohibitively expensive to convert from protobuf types to HBase types (and back) just for coprocessors.



-- 
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] apurtell commented on pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
apurtell commented on PR #4924:
URL: https://github.com/apache/hbase/pull/4924#issuecomment-1371249713

   > I do not think adding WAL attributes to mutation directly is the correct way to deal with this.
   
   For what it's worth, I think it would be fine, but I see your point.
   
   Consider what we are designing end to end. 
   
   On the client side, the client creates a mutation with an attribute containing some metadata. 
   On the server side, a coprocessor extracts the metadata and decorates the associated WAL entry via an attribute.
   Then the WAL entry is replicated. 
   On the sink side, the WAL entry is processed and the metadata should be recovered from the attribute.
   
   The question is how to do that last part. I think introducing a default conversion from WAL entry attribute to mutation attribute would sufficient, but to @Apache9 's point, a symmetric flow here would have a pair of coprocessors involved: the first transforming attribute to metadata at the source client, the latter transforming metadata to attribute at the sink client. 
   
   What I would not like to see here is a new config.


-- 
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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 38s |  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  |   2m 34s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 24s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 39s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 31s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 28s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 28s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 33s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   9m 34s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.4.  |
   | +1 :green_heart: |  spotless  |   0m 39s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 32s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  8s |  The patch does not generate ASF License warnings.  |
   |  |   |  31m 27s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux c7b976e1553c 5.4.0-1088-aws #96~18.04.1-Ubuntu SMP Mon Oct 17 02:57:48 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8b4e134f8c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | Max. process+thread count | 77 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/4/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046593544


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   > And also, you can read it in ReplicationSink at the sink cluster, no problem.
   
   But once sink reads it, how else can it pass the attributes to coproc at sink side? 
   
   > The problem here is you want to add the WAL attributes to mutation's attributes.
   
   What if that's the behavior coproc expects? Would you be fine with configurable behavior?



-- 
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 #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 57s |  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  |   2m 23s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 58s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m  4s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 35s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m  0s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 209m 10s |  hbase-server in the patch passed.  |
   |  |   | 227m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/8/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 5fcd925cd654 5.4.0-1092-aws #100~18.04.2-Ubuntu SMP Tue Nov 29 08:39:52 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 991ca9598b |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/8/testReport/ |
   | Max. process+thread count | 2441 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/8/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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 #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

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

   :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  |   2m 32s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 48s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 29s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 47s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 47s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 46s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 211m 40s |  hbase-server in the patch passed.  |
   |  |   | 232m 10s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/8/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 9f5b07a00ce2 5.4.0-135-generic #152-Ubuntu SMP Wed Nov 23 20:19:22 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 991ca9598b |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/8/testReport/ |
   | Max. process+thread count | 2689 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/8/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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] apurtell commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
apurtell commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1065102733


##########
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithWALExtendedAttributes.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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 static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+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.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+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.codec.KeyValueCodecWithTags;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.junit.AfterClass;
+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;
+
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestReplicationWithWALExtendedAttributes {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestReplicationWithWALExtendedAttributes.class);
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(TestReplicationWithWALExtendedAttributes.class);
+
+  private static Configuration conf1 = HBaseConfiguration.create();
+
+  private static Admin replicationAdmin;
+
+  private static Connection connection1;
+
+  private static Table htable1;
+  private static Table htable2;
+
+  private static HBaseTestingUtil utility1;
+  private static HBaseTestingUtil utility2;
+  private static final long SLEEP_TIME = 500;
+  private static final int NB_RETRIES = 10;
+
+  private static final TableName TABLE_NAME = TableName.valueOf("TestReplicationWithWALAnnotation");
+  private static final byte[] FAMILY = Bytes.toBytes("f");
+  private static final byte[] ROW = Bytes.toBytes("row");
+  private static final byte[] ROW2 = Bytes.toBytes("row2");
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
+    conf1.setInt("replication.source.size.capacity", 10240);
+    conf1.setLong("replication.source.sleepforretries", 100);
+    conf1.setInt("hbase.regionserver.maxlogs", 10);
+    conf1.setLong("hbase.master.logcleaner.ttl", 10);
+    conf1.setInt("zookeeper.recovery.retry", 1);
+    conf1.setInt("zookeeper.recovery.retry.intervalmill", 10);
+    conf1.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
+    conf1.setInt("replication.stats.thread.period.seconds", 5);
+    conf1.setBoolean("hbase.tests.use.shortcircuit.reads", false);
+    conf1.setStrings(HConstants.REPLICATION_CODEC_CONF_KEY, KeyValueCodecWithTags.class.getName());
+    conf1.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
+      TestCoprocessorForWALAnnotationAtSource.class.getName());
+
+    utility1 = new HBaseTestingUtil(conf1);
+    utility1.startMiniZKCluster();
+    MiniZooKeeperCluster miniZK = utility1.getZkCluster();
+    // Have to reget conf1 in case zk cluster location different
+    // than default
+    conf1 = utility1.getConfiguration();
+    LOG.info("Setup first Zk");
+
+    // Base conf2 on conf1 so it gets the right zk cluster.
+    Configuration conf2 = HBaseConfiguration.create(conf1);
+    conf2.setInt("hfile.format.version", 3);
+    conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
+    conf2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
+    conf2.setBoolean("hbase.tests.use.shortcircuit.reads", false);
+    conf2.setStrings(HConstants.REPLICATION_CODEC_CONF_KEY, KeyValueCodecWithTags.class.getName());
+    conf2.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
+      TestCoprocessorForWALAnnotationAtSink.class.getName());
+    conf2.setStrings(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
+      TestReplicationSinkRegionServerEndpoint.class.getName());
+
+    utility2 = new HBaseTestingUtil(conf2);
+    utility2.setZkCluster(miniZK);
+
+    LOG.info("Setup second Zk");
+    utility1.startMiniCluster(2);
+    utility2.startMiniCluster(2);
+
+    connection1 = ConnectionFactory.createConnection(conf1);
+    replicationAdmin = connection1.getAdmin();
+    ReplicationPeerConfig rpc =
+      ReplicationPeerConfig.newBuilder().setClusterKey(utility2.getClusterKey()).build();
+    replicationAdmin.addReplicationPeer("2", rpc);
+
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(3)
+        .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
+      .build();
+    try (Connection conn = ConnectionFactory.createConnection(conf1);
+      Admin admin = conn.getAdmin()) {
+      admin.createTable(tableDescriptor, HBaseTestingUtil.KEYS_FOR_HBA_CREATE_TABLE);
+    }
+    try (Connection conn = ConnectionFactory.createConnection(conf2);
+      Admin admin = conn.getAdmin()) {
+      admin.createTable(tableDescriptor, HBaseTestingUtil.KEYS_FOR_HBA_CREATE_TABLE);
+    }
+    htable1 = utility1.getConnection().getTable(TABLE_NAME);
+    htable2 = utility2.getConnection().getTable(TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    Closeables.close(replicationAdmin, true);
+    Closeables.close(connection1, true);
+    utility2.shutdownMiniCluster();
+    utility1.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testReplicationWithWALExtendedAttributes() throws Exception {
+    Put put = new Put(ROW);
+    put.addColumn(FAMILY, ROW, ROW);
+
+    htable1 = utility1.getConnection().getTable(TABLE_NAME);
+    htable1.put(put);
+
+    Put put2 = new Put(ROW2);
+    put2.addColumn(FAMILY, ROW2, ROW2);
+
+    htable1.batch(Collections.singletonList(put2), new Object[1]);
+
+    assertGetValues(new Get(ROW), ROW);
+    assertGetValues(new Get(ROW2), ROW2);
+  }
+
+  private static void assertGetValues(Get get, byte[] value)
+    throws IOException, InterruptedException {
+    for (int i = 0; i < NB_RETRIES; i++) {
+      if (i == NB_RETRIES - 1) {
+        fail("Waited too much time for put replication");
+      }
+      Result res = htable2.get(get);
+      if (res.isEmpty()) {
+        LOG.info("Row not available");
+        Thread.sleep(SLEEP_TIME);
+      } else {
+        assertArrayEquals(value, res.value());
+        break;
+      }
+    }
+  }
+
+  public static class TestCoprocessorForWALAnnotationAtSource
+    implements RegionCoprocessor, RegionObserver {
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void preWALAppend(ObserverContext<RegionCoprocessorEnvironment> ctx, WALKey key,
+      WALEdit edit) throws IOException {
+      key.addExtendedAttribute("extendedAttr1", Bytes.toBytes("Value of Extended attribute 01"));
+      key.addExtendedAttribute("extendedAttr2", Bytes.toBytes("Value of Extended attribute 02"));
+    }
+  }
+
+  public static class TestCoprocessorForWALAnnotationAtSink
+    implements RegionCoprocessor, RegionObserver {
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit)
+      throws IOException {
+      String attrVal1 = Bytes.toString(put.getAttribute("extendedAttr1"));
+      String attrVal2 = Bytes.toString(put.getAttribute("extendedAttr2"));
+      if (attrVal1 == null || attrVal2 == null) {
+        throw new IOException("Failed to retrieve WAL annotations");
+      }
+      if (
+        attrVal1.equals("Value of Extended attribute 01")
+          && attrVal2.equals("Value of Extended attribute 02")
+      ) {
+        return;
+      }
+      throw new IOException("Failed to retrieve WAL annotations..");
+    }
+
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      String attrVal1 = Bytes.toString(miniBatchOp.getOperation(0).getAttribute("extendedAttr1"));
+      String attrVal2 = Bytes.toString(miniBatchOp.getOperation(0).getAttribute("extendedAttr2"));
+      if (attrVal1 == null || attrVal2 == null) {
+        throw new IOException("Failed to retrieve WAL annotations");
+      }
+      if (
+        attrVal1.equals("Value of Extended attribute 01")
+          && attrVal2.equals("Value of Extended attribute 02")
+      ) {
+        return;
+      }
+      throw new IOException("Failed to retrieve WAL annotations..");
+    }
+  }
+
+  public static final class TestReplicationSinkRegionServerEndpoint

Review Comment:
   Sure, we have a new coprocessor, so it must be registered. This is not a new configuration key-value, though. hbase.coprocessor.regionserver.classes already exists.



-- 
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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046599163


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   > And if phoenix wants this feature, please describe more about at lease one usage. Let's see how to better support it.
   
   So let me take a simple example. Phoenix supports multi-tenancy by providing multi-tenant connection. Hence, same table can be shared by multiple tenants. This requires tenant id to be a rowkey prefix in hbase rowkey. When a particular tenant creates tenant connection and writes data, those data are not visible to other tenants.
   Tenant id is very basic entity in phoenix. Now let's say we want to introduce some level of caching in phoenix server side, the cache would contain tenant id.
   
   For HBase sink cluster, mutation is already created for specific table because table name is available in WALKey. For phoenix, let's say table name is available and mutation has full rowkey as well but it's very difficult to derive tenant id from this rowkey. Hence, phoenix coproc needs to know of tenant id attribute with mutation so that it can form tenant level connection or derive further attributes at sink side and so on.
   
   In addition to tenant id, there are a few more imp metadata that is required at sink side coproc, without which it's not possible to be aware of how the data is meant to be used.
   
   > How do you read these attributes at the source cluster's coproc?
   
   If we take same example as above, multi tenant connection is already created by client and hence tenant id is always available with all operations performed at source cluster coproc. Besides some of the imp attributes are already attached as mutation attributes at source side. But when the mutations are generated at sink side, while hbase metadata is available, coproc like phoenix metadata is not available and it would be great if we can attach these metadata attributes to sink mutation, otherwise sink coproc would not be able to derive them.
   Making this configurable feature with proper documentation is also fine, if you don't mind. WDYT?



-- 
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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 55s |  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  |   2m 59s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 46s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 33s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 47s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 47s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 45s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 210m 44s |  hbase-server in the patch failed.  |
   |  |   | 231m  1s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/3/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 61c5b61dcde5 5.4.0-135-generic #152-Ubuntu SMP Wed Nov 23 20:19:22 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8b4e134f8c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/3/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/3/testReport/ |
   | Max. process+thread count | 2700 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/3/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  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  |   2m  0s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 55s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m  2s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 35s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 56s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 202m 10s |  hbase-server in the patch passed.  |
   |  |   | 220m  8s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux aa347d546ba9 5.4.0-1092-aws #100~18.04.2-Ubuntu SMP Tue Nov 29 08:39:52 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8b4e134f8c |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/3/testReport/ |
   | Max. process+thread count | 2479 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/3/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046590676


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   > The goal of the WAL extended attribute is to pass some attributes down to the replication implementation from the coproc implementation
   
   That is correct but coproc would add it only so that it can consume it at sink side correct? Otherwise who else would read WAL attributes from WALEntry at sink side? HBase does it, while preparing for sink side mutations.
   
   At sink side, once corpoc receives mutations generated by replication, it cannot correlate the mutation with WAL extended attributes on it's own. Getting these attributes consumed at sink side is the primary reason why corpoc would like to pass some attributes down to replication implementation.



-- 
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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  4s |  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  |   2m 44s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 45s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 33s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 46s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 46s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 44s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 213m 42s |  hbase-server in the patch passed.  |
   |  |   | 234m 23s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/6/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 31651c26b248 5.4.0-135-generic #152-Ubuntu SMP Wed Nov 23 20:19:22 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 3f1087fe82 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/6/testReport/ |
   | Max. process+thread count | 2472 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/6/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 39s |  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  |   2m 48s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 27s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 42s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 28s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 36s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 23s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 33s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   9m 43s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.4.  |
   | +1 :green_heart: |  spotless  |   0m 39s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 32s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 10s |  The patch does not generate ASF License warnings.  |
   |  |   |  32m 16s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 4e4212434f8d 5.4.0-1088-aws #96~18.04.1-Ubuntu SMP Mon Oct 17 02:57:48 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 37c82a6209 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/1/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046600026


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   For the sake of this example, let's say that tenant id (part of the extended attribute in WALKey) is as important to phoenix, as is rowkey to hbase. It's quite basic entity.



-- 
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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +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  |   2m 34s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 24s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 39s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 24s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 29s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 33s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   9m 39s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.4.  |
   | +1 :green_heart: |  spotless  |   0m 39s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 33s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  8s |  The patch does not generate ASF License warnings.  |
   |  |   |  31m 48s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 9236b368470c 5.4.0-1088-aws #96~18.04.1-Ubuntu SMP Mon Oct 17 02:57:48 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 37c82a6209 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | Max. process+thread count | 78 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/2/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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 #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

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

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  3s |  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  |   2m 40s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 48s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 46s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 46s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 46s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 212m 30s |  hbase-server in the patch failed.  |
   |  |   | 233m  1s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/7/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3ae9412455ae 5.4.0-135-generic #152-Ubuntu SMP Wed Nov 23 20:19:22 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 4add5250ed |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/7/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/7/testReport/ |
   | Max. process+thread count | 2684 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/7/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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 #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 39s |  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  |   2m 38s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 30s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 41s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 27s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 32s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 29s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 30s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   9m 47s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.4.  |
   | +1 :green_heart: |  spotless  |   0m 40s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  8s |  The patch does not generate ASF License warnings.  |
   |  |   |  32m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/8/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 8c41773305b7 5.4.0-1088-aws #96~18.04.1-Ubuntu SMP Mon Oct 17 02:57:48 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 991ca9598b |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/8/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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 #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  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  |   2m 41s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 23s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 43s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 23s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 21s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 21s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 21s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 32s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   8m 55s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.4.  |
   | +1 :green_heart: |  spotless  |   0m 37s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 28s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 10s |  The patch does not generate ASF License warnings.  |
   |  |   |  30m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/9/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux e5b767f6961e 5.4.0-1093-aws #102~18.04.2-Ubuntu SMP Wed Dec 7 00:31:59 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2a7c69d30e |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/9/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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 #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  1s |  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  |   2m 33s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 46s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 46s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 46s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 45s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 210m 50s |  hbase-server in the patch passed.  |
   |  |   | 230m 45s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/9/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e10ed094eccd 5.4.0-135-generic #152-Ubuntu SMP Wed Nov 23 20:19:22 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2a7c69d30e |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/9/testReport/ |
   | Max. process+thread count | 2633 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/9/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046586266


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   There has to be a consumer of WAL extended attribute. The primary purpose for anything associated with WALKey is meant for deriving appropriate Mutation behavior or replication related behavior (like list of cluster ids). Hence if we allow coproc to set the WAL extended attributes, we should also let the attributes be consumed at sink side. Otherwise we won't be able to have any consumer of these attributes, correct?



-- 
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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 59s |  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  |   2m 49s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 46s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 47s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 46s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 46s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 49s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 211m  3s |  hbase-server in the patch failed.  |
   |  |   | 231m 36s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 772fdb158213 5.4.0-135-generic #152-Ubuntu SMP Wed Nov 23 20:19:22 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 37c82a6209 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/1/testReport/ |
   | Max. process+thread count | 2631 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/1/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046590782


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   If required, we can make this behavior configurable?



-- 
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] apurtell commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
apurtell commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1065100096


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java:
##########
@@ -166,6 +169,16 @@ public void call(RegionServerObserver observer) throws IOException {
     });
   }
 
+  public void preReplicationSinkBatchMutate(AdminProtos.WALEntry walEntry, Mutation mutation)

Review Comment:
   The coprocessor API is symmetric. Whenever there is a preXXX action, giving the user control before a thing happens, there must also be a postXXX action, letting the user get control after the thing happened. There can be case by case exceptions when no postXXX condition actually exists, like after a shutdown. 



-- 
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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   3m 20s |  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  |   2m 24s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m  0s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 12s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 35s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m  0s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 210m 15s |  hbase-server in the patch passed.  |
   |  |   | 232m  1s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/6/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 106bee083b12 5.4.0-1092-aws #100~18.04.2-Ubuntu SMP Tue Nov 29 08:39:52 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 3f1087fe82 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/6/testReport/ |
   | Max. process+thread count | 2369 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/6/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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 diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046594456


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   How do you read these attributes at the source cluster's coproc?



-- 
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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +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  |   2m 36s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 26s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 30s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 40s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 33s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   9m 34s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.4.  |
   | +1 :green_heart: |  spotless  |   0m 41s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 31s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  7s |  The patch does not generate ASF License warnings.  |
   |  |   |  31m 26s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 36a8a226e3f3 5.4.0-1088-aws #96~18.04.1-Ubuntu SMP Mon Oct 17 02:57:48 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8b4e134f8c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | Max. process+thread count | 78 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/3/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  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  |   2m  4s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 58s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m  2s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 35s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 58s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 202m 41s |  hbase-server in the patch passed.  |
   |  |   | 220m 57s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e1f9e8ef57be 5.4.0-1092-aws #100~18.04.2-Ubuntu SMP Tue Nov 29 08:39:52 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 37c82a6209 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/2/testReport/ |
   | Max. process+thread count | 2468 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/2/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 38s |  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  |   2m 33s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 32s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 31s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 40s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 24s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 24s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 24s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   9m 35s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.4.  |
   | +1 :green_heart: |  spotless  |   0m 40s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 34s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  8s |  The patch does not generate ASF License warnings.  |
   |  |   |  31m 51s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/5/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux f2f7945aba9f 5.4.0-1088-aws #96~18.04.1-Ubuntu SMP Mon Oct 17 02:57:48 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 222ec684d6 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/5/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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 diff in pull request #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1067794535


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java:
##########
@@ -137,4 +140,33 @@ default void preExecuteProcedures(ObserverContext<RegionServerCoprocessorEnviron
   default void postExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
     throws IOException {
   }
+
+  /**
+   * This will be called before replication sink mutations are executed on the sink table as part of
+   * batch call.
+   * @param ctx      the environment to interact with the framework and region server.
+   * @param walEntry wal entry from which mutation is formed.
+   * @param mutation mutation to be applied at sink cluster.
+   * @throws IOException if something goes wrong.
+   */
+  default void preReplicationSinkBatchMutate(
+    ObserverContext<RegionServerCoprocessorEnvironment> ctx, AdminProtos.WALEntry walEntry,

Review Comment:
   Agree with @apurtell that exposing pb message in CP is not very good but I checked the code, we never convert the WALEntry to a none pb one so there is no good way for us to not use pb message here. Let's keep it like this for now.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -178,16 +179,19 @@ private void decorateConf() {
   /**
    * Replicate this array of entries directly into the local cluster using the native client. Only
    * operates against raw protobuf type saving on a conversion from pb to pojo.
+   * @param entries                    WAL entries to be replicated.
+   * @param cells                      cell scanner for iteration.
    * @param replicationClusterId       Id which will uniquely identify source cluster FS client
    *                                   configurations in the replication configuration directory
    * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
    *                                   directory
    * @param sourceHFileArchiveDirPath  Path that point to the source cluster hfile archive directory
+   * @param rsServerHost               regionserver coproc host.
    * @throws IOException If failed to replicate the data
    */
   public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
     String replicationClusterId, String sourceBaseNamespaceDirPath,
-    String sourceHFileArchiveDirPath) throws IOException {
+    String sourceHFileArchiveDirPath, RegionServerCoprocessorHost rsServerHost) throws IOException {

Review Comment:
   I suppose the RegionServerCroprocessorHost should be initialized when creating ReplicationSink?
   
   I checked the code, maybe in ReplicationSinkServiceImpl, we could check whether the Server is an instance of HRegionServer, if so, we get its RegionServerCoprocessorHost for creating the ReplicationSink. And we could also reuse the AsyncClusterConnection of the Server, instead of creating a new one 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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1069068913


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -178,16 +179,19 @@ private void decorateConf() {
   /**
    * Replicate this array of entries directly into the local cluster using the native client. Only
    * operates against raw protobuf type saving on a conversion from pb to pojo.
+   * @param entries                    WAL entries to be replicated.
+   * @param cells                      cell scanner for iteration.
    * @param replicationClusterId       Id which will uniquely identify source cluster FS client
    *                                   configurations in the replication configuration directory
    * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
    *                                   directory
    * @param sourceHFileArchiveDirPath  Path that point to the source cluster hfile archive directory
+   * @param rsServerHost               regionserver coproc host.
    * @throws IOException If failed to replicate the data
    */
   public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
     String replicationClusterId, String sourceBaseNamespaceDirPath,
-    String sourceHFileArchiveDirPath) throws IOException {
+    String sourceHFileArchiveDirPath, RegionServerCoprocessorHost rsServerHost) throws IOException {

Review Comment:
   Ah I see, yes that sounds good, will make these changes.



-- 
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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1062009281


##########
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithWALExtendedAttributes.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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 static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+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.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+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.codec.KeyValueCodecWithTags;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.junit.AfterClass;
+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;
+
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestReplicationWithWALExtendedAttributes {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestReplicationWithWALExtendedAttributes.class);
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(TestReplicationWithWALExtendedAttributes.class);
+
+  private static Configuration conf1 = HBaseConfiguration.create();
+
+  private static Admin replicationAdmin;
+
+  private static Connection connection1;
+
+  private static Table htable1;
+  private static Table htable2;
+
+  private static HBaseTestingUtil utility1;
+  private static HBaseTestingUtil utility2;
+  private static final long SLEEP_TIME = 500;
+  private static final int NB_RETRIES = 10;
+
+  private static final TableName TABLE_NAME = TableName.valueOf("TestReplicationWithWALAnnotation");
+  private static final byte[] FAMILY = Bytes.toBytes("f");
+  private static final byte[] ROW = Bytes.toBytes("row");
+  private static final byte[] ROW2 = Bytes.toBytes("row2");
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
+    conf1.setInt("replication.source.size.capacity", 10240);
+    conf1.setLong("replication.source.sleepforretries", 100);
+    conf1.setInt("hbase.regionserver.maxlogs", 10);
+    conf1.setLong("hbase.master.logcleaner.ttl", 10);
+    conf1.setInt("zookeeper.recovery.retry", 1);
+    conf1.setInt("zookeeper.recovery.retry.intervalmill", 10);
+    conf1.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
+    conf1.setInt("replication.stats.thread.period.seconds", 5);
+    conf1.setBoolean("hbase.tests.use.shortcircuit.reads", false);
+    conf1.setStrings(HConstants.REPLICATION_CODEC_CONF_KEY, KeyValueCodecWithTags.class.getName());
+    conf1.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
+      TestCoprocessorForWALAnnotationAtSource.class.getName());
+
+    utility1 = new HBaseTestingUtil(conf1);
+    utility1.startMiniZKCluster();
+    MiniZooKeeperCluster miniZK = utility1.getZkCluster();
+    // Have to reget conf1 in case zk cluster location different
+    // than default
+    conf1 = utility1.getConfiguration();
+    LOG.info("Setup first Zk");
+
+    // Base conf2 on conf1 so it gets the right zk cluster.
+    Configuration conf2 = HBaseConfiguration.create(conf1);
+    conf2.setInt("hfile.format.version", 3);
+    conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
+    conf2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
+    conf2.setBoolean("hbase.tests.use.shortcircuit.reads", false);
+    conf2.setStrings(HConstants.REPLICATION_CODEC_CONF_KEY, KeyValueCodecWithTags.class.getName());
+    conf2.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
+      TestCoprocessorForWALAnnotationAtSink.class.getName());
+    conf2.setStrings(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
+      TestReplicationSinkRegionServerEndpoint.class.getName());
+
+    utility2 = new HBaseTestingUtil(conf2);
+    utility2.setZkCluster(miniZK);
+
+    LOG.info("Setup second Zk");
+    utility1.startMiniCluster(2);
+    utility2.startMiniCluster(2);
+
+    connection1 = ConnectionFactory.createConnection(conf1);
+    replicationAdmin = connection1.getAdmin();
+    ReplicationPeerConfig rpc =
+      ReplicationPeerConfig.newBuilder().setClusterKey(utility2.getClusterKey()).build();
+    replicationAdmin.addReplicationPeer("2", rpc);
+
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(3)
+        .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
+      .build();
+    try (Connection conn = ConnectionFactory.createConnection(conf1);
+      Admin admin = conn.getAdmin()) {
+      admin.createTable(tableDescriptor, HBaseTestingUtil.KEYS_FOR_HBA_CREATE_TABLE);
+    }
+    try (Connection conn = ConnectionFactory.createConnection(conf2);
+      Admin admin = conn.getAdmin()) {
+      admin.createTable(tableDescriptor, HBaseTestingUtil.KEYS_FOR_HBA_CREATE_TABLE);
+    }
+    htable1 = utility1.getConnection().getTable(TABLE_NAME);
+    htable2 = utility2.getConnection().getTable(TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    Closeables.close(replicationAdmin, true);
+    Closeables.close(connection1, true);
+    utility2.shutdownMiniCluster();
+    utility1.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testReplicationWithWALExtendedAttributes() throws Exception {
+    Put put = new Put(ROW);
+    put.addColumn(FAMILY, ROW, ROW);
+
+    htable1 = utility1.getConnection().getTable(TABLE_NAME);
+    htable1.put(put);
+
+    Put put2 = new Put(ROW2);
+    put2.addColumn(FAMILY, ROW2, ROW2);
+
+    htable1.batch(Collections.singletonList(put2), new Object[1]);
+
+    assertGetValues(new Get(ROW), ROW);
+    assertGetValues(new Get(ROW2), ROW2);
+  }
+
+  private static void assertGetValues(Get get, byte[] value)
+    throws IOException, InterruptedException {
+    for (int i = 0; i < NB_RETRIES; i++) {
+      if (i == NB_RETRIES - 1) {
+        fail("Waited too much time for put replication");
+      }
+      Result res = htable2.get(get);
+      if (res.isEmpty()) {
+        LOG.info("Row not available");
+        Thread.sleep(SLEEP_TIME);
+      } else {
+        assertArrayEquals(value, res.value());
+        break;
+      }
+    }
+  }
+
+  public static class TestCoprocessorForWALAnnotationAtSource
+    implements RegionCoprocessor, RegionObserver {
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void preWALAppend(ObserverContext<RegionCoprocessorEnvironment> ctx, WALKey key,
+      WALEdit edit) throws IOException {
+      key.addExtendedAttribute("extendedAttr1", Bytes.toBytes("Value of Extended attribute 01"));
+      key.addExtendedAttribute("extendedAttr2", Bytes.toBytes("Value of Extended attribute 02"));
+    }
+  }
+
+  public static class TestCoprocessorForWALAnnotationAtSink
+    implements RegionCoprocessor, RegionObserver {
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit)
+      throws IOException {
+      String attrVal1 = Bytes.toString(put.getAttribute("extendedAttr1"));
+      String attrVal2 = Bytes.toString(put.getAttribute("extendedAttr2"));
+      if (attrVal1 == null || attrVal2 == null) {
+        throw new IOException("Failed to retrieve WAL annotations");
+      }
+      if (
+        attrVal1.equals("Value of Extended attribute 01")
+          && attrVal2.equals("Value of Extended attribute 02")
+      ) {
+        return;
+      }
+      throw new IOException("Failed to retrieve WAL annotations..");
+    }
+
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      String attrVal1 = Bytes.toString(miniBatchOp.getOperation(0).getAttribute("extendedAttr1"));
+      String attrVal2 = Bytes.toString(miniBatchOp.getOperation(0).getAttribute("extendedAttr2"));
+      if (attrVal1 == null || attrVal2 == null) {
+        throw new IOException("Failed to retrieve WAL annotations");
+      }
+      if (
+        attrVal1.equals("Value of Extended attribute 01")
+          && attrVal2.equals("Value of Extended attribute 02")
+      ) {
+        return;
+      }
+      throw new IOException("Failed to retrieve WAL annotations..");
+    }
+  }
+
+  public static final class TestReplicationSinkRegionServerEndpoint

Review Comment:
   @Apache9 @apurtell Even with this approach, we at least need config change to allow adding new class value to `hbase.coprocessor.regionserver.classes` config.



-- 
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] apurtell commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
apurtell commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1065100096


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java:
##########
@@ -166,6 +169,16 @@ public void call(RegionServerObserver observer) throws IOException {
     });
   }
 
+  public void preReplicationSinkBatchMutate(AdminProtos.WALEntry walEntry, Mutation mutation)

Review Comment:
   The coprocessor API has a symmetry around upcall points by design principle. Whenever there is a preXXX action, giving the user control before a thing happens, there must also be a postXXX action, letting the user get control after the thing happened. There can be case by case exceptions when no postXXX condition actually exists, like after a shutdown. 
   
   The pattern is like this:
   
       doPreXXX(ctx, /* context specific upcall arguments */); 
       if (!ctx.shouldBypass()) {
           try {
               // do stuff
           } finally {
               doPostXXX(ctx, /* context specific upcall arguments */);
           }
       }
   



-- 
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] apurtell commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
apurtell commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1065100096


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java:
##########
@@ -166,6 +169,16 @@ public void call(RegionServerObserver observer) throws IOException {
     });
   }
 
+  public void preReplicationSinkBatchMutate(AdminProtos.WALEntry walEntry, Mutation mutation)

Review Comment:
   The coprocessor API has a symmetry around upcall points by design principle. Whenever there is a preXXX action, giving the user control before a thing happens, there must also be a postXXX action, letting the user get control after the thing happened. There can be case by case exceptions when no postXXX condition actually exists, like after a shutdown. 
   
   The pattern is like this:
   
       preXXX(ctx, /* context specific upcall arguments */); 
       if (!ctx.shouldBypass()) {
           try {
               // do stuff
           } finally {
               postXXX(ctx, /* context specific upcall arguments */);
           }
       }
   



-- 
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 diff in pull request #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1068971919


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -178,16 +179,19 @@ private void decorateConf() {
   /**
    * Replicate this array of entries directly into the local cluster using the native client. Only
    * operates against raw protobuf type saving on a conversion from pb to pojo.
+   * @param entries                    WAL entries to be replicated.
+   * @param cells                      cell scanner for iteration.
    * @param replicationClusterId       Id which will uniquely identify source cluster FS client
    *                                   configurations in the replication configuration directory
    * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
    *                                   directory
    * @param sourceHFileArchiveDirPath  Path that point to the source cluster hfile archive directory
+   * @param rsServerHost               regionserver coproc host.
    * @throws IOException If failed to replicate the data
    */
   public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
     String replicationClusterId, String sourceBaseNamespaceDirPath,
-    String sourceHFileArchiveDirPath) throws IOException {
+    String sourceHFileArchiveDirPath, RegionServerCoprocessorHost rsServerHost) throws IOException {

Review Comment:
   I mean we should pass RegionServerCoprocessorHost in when creating ReplicationSink, and store it as the a class field, so we do not need to pass it everytime when calling replicateEntries...



-- 
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 #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 46s |  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  |   2m 29s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m  1s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m  6s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 35s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m  0s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 208m  4s |  hbase-server in the patch passed.  |
   |  |   | 226m 52s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/9/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a92a330374d0 5.4.0-1092-aws #100~18.04.2-Ubuntu SMP Tue Nov 29 08:39:52 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2a7c69d30e |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/9/testReport/ |
   | Max. process+thread count | 2449 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/9/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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] apurtell commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
apurtell commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1061745477


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -107,6 +107,14 @@ public class ReplicationSink {
   private SourceFSConfigurationProvider provider;
   private WALEntrySinkFilter walEntrySinkFilter;
 
+  /**
+   * If enabled at sink cluster site config, extended WAL attributes would be attached as Mutation
+   * attributes. This is useful for source cluster coproc to provide coproc specific metadata as WAL
+   * annotations and have them attached back to Mutations generated from WAL entries at sink side.
+   */
+  public static final String HBASE_REPLICATION_SINK_ATTRIBUTES_WAL_TO_MUTATIONS =

Review Comment:
   This doesn't need to be configurable. Mutation attributes will be ignored if they are not expected so this is harmless to clients. And IMHO the right thing to be doing. I agree with your premise.
   
   Normally WAL entries will not have attributes, so this is doubly harmless for standard use cases. 



-- 
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 diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046591100


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   No, you do not understand the problem here. While you say that
   
   > the sink side mutation attributes can be expected to be set up by HBase since hbase prepares mutations from WAL entries.
   
   But will we set these attributes to mutation's attributes at source cluster? If not, why should we set them at the sink cluster?



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

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 diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046571171


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   This is WAL extended attribute, not mutation attribute? I do not think we should just set them as attributes 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] Apache-HBase commented on pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   3m 15s |  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  |   2m  5s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m  8s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 38s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m  9s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 214m 15s |  hbase-server in the patch passed.  |
   |  |   | 235m 53s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/5/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3bba769aa263 5.4.0-1092-aws #100~18.04.2-Ubuntu SMP Tue Nov 29 08:39:52 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 222ec684d6 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/5/testReport/ |
   | Max. process+thread count | 2428 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/5/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :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  |   2m 32s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 48s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 50s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 31s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 46s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 46s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 49s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 209m 59s |  hbase-server in the patch passed.  |
   |  |   | 230m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 22fe83e79be4 5.4.0-135-generic #152-Ubuntu SMP Wed Nov 23 20:19:22 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 37c82a6209 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/2/testReport/ |
   | Max. process+thread count | 2558 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/2/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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] apurtell commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
apurtell commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1065100096


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java:
##########
@@ -166,6 +169,16 @@ public void call(RegionServerObserver observer) throws IOException {
     });
   }
 
+  public void preReplicationSinkBatchMutate(AdminProtos.WALEntry walEntry, Mutation mutation)

Review Comment:
   The coprocessor API has a symmetry around upcall points by design principle. Whenever there is a preXXX action, giving the user control before a thing happens, there must also be a postXXX action, letting the user get control after the thing happened. There can be case by case exceptions when no postXXX condition actually exists, like after a shutdown. 



-- 
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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1066266739


##########
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithWALExtendedAttributes.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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 static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+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.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+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.codec.KeyValueCodecWithTags;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.junit.AfterClass;
+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;
+
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestReplicationWithWALExtendedAttributes {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestReplicationWithWALExtendedAttributes.class);
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(TestReplicationWithWALExtendedAttributes.class);
+
+  private static Configuration conf1 = HBaseConfiguration.create();
+
+  private static Admin replicationAdmin;
+
+  private static Connection connection1;
+
+  private static Table htable1;
+  private static Table htable2;
+
+  private static HBaseTestingUtil utility1;
+  private static HBaseTestingUtil utility2;
+  private static final long SLEEP_TIME = 500;
+  private static final int NB_RETRIES = 10;
+
+  private static final TableName TABLE_NAME = TableName.valueOf("TestReplicationWithWALAnnotation");
+  private static final byte[] FAMILY = Bytes.toBytes("f");
+  private static final byte[] ROW = Bytes.toBytes("row");
+  private static final byte[] ROW2 = Bytes.toBytes("row2");
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
+    conf1.setInt("replication.source.size.capacity", 10240);
+    conf1.setLong("replication.source.sleepforretries", 100);
+    conf1.setInt("hbase.regionserver.maxlogs", 10);
+    conf1.setLong("hbase.master.logcleaner.ttl", 10);
+    conf1.setInt("zookeeper.recovery.retry", 1);
+    conf1.setInt("zookeeper.recovery.retry.intervalmill", 10);
+    conf1.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
+    conf1.setInt("replication.stats.thread.period.seconds", 5);
+    conf1.setBoolean("hbase.tests.use.shortcircuit.reads", false);
+    conf1.setStrings(HConstants.REPLICATION_CODEC_CONF_KEY, KeyValueCodecWithTags.class.getName());
+    conf1.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
+      TestCoprocessorForWALAnnotationAtSource.class.getName());
+
+    utility1 = new HBaseTestingUtil(conf1);
+    utility1.startMiniZKCluster();
+    MiniZooKeeperCluster miniZK = utility1.getZkCluster();
+    // Have to reget conf1 in case zk cluster location different
+    // than default
+    conf1 = utility1.getConfiguration();
+    LOG.info("Setup first Zk");
+
+    // Base conf2 on conf1 so it gets the right zk cluster.
+    Configuration conf2 = HBaseConfiguration.create(conf1);
+    conf2.setInt("hfile.format.version", 3);
+    conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
+    conf2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
+    conf2.setBoolean("hbase.tests.use.shortcircuit.reads", false);
+    conf2.setStrings(HConstants.REPLICATION_CODEC_CONF_KEY, KeyValueCodecWithTags.class.getName());
+    conf2.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
+      TestCoprocessorForWALAnnotationAtSink.class.getName());
+    conf2.setStrings(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
+      TestReplicationSinkRegionServerEndpoint.class.getName());
+
+    utility2 = new HBaseTestingUtil(conf2);
+    utility2.setZkCluster(miniZK);
+
+    LOG.info("Setup second Zk");
+    utility1.startMiniCluster(2);
+    utility2.startMiniCluster(2);
+
+    connection1 = ConnectionFactory.createConnection(conf1);
+    replicationAdmin = connection1.getAdmin();
+    ReplicationPeerConfig rpc =
+      ReplicationPeerConfig.newBuilder().setClusterKey(utility2.getClusterKey()).build();
+    replicationAdmin.addReplicationPeer("2", rpc);
+
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(3)
+        .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
+      .build();
+    try (Connection conn = ConnectionFactory.createConnection(conf1);
+      Admin admin = conn.getAdmin()) {
+      admin.createTable(tableDescriptor, HBaseTestingUtil.KEYS_FOR_HBA_CREATE_TABLE);
+    }
+    try (Connection conn = ConnectionFactory.createConnection(conf2);
+      Admin admin = conn.getAdmin()) {
+      admin.createTable(tableDescriptor, HBaseTestingUtil.KEYS_FOR_HBA_CREATE_TABLE);
+    }
+    htable1 = utility1.getConnection().getTable(TABLE_NAME);
+    htable2 = utility2.getConnection().getTable(TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    Closeables.close(replicationAdmin, true);
+    Closeables.close(connection1, true);
+    utility2.shutdownMiniCluster();
+    utility1.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testReplicationWithWALExtendedAttributes() throws Exception {
+    Put put = new Put(ROW);
+    put.addColumn(FAMILY, ROW, ROW);
+
+    htable1 = utility1.getConnection().getTable(TABLE_NAME);
+    htable1.put(put);
+
+    Put put2 = new Put(ROW2);
+    put2.addColumn(FAMILY, ROW2, ROW2);
+
+    htable1.batch(Collections.singletonList(put2), new Object[1]);
+
+    assertGetValues(new Get(ROW), ROW);
+    assertGetValues(new Get(ROW2), ROW2);
+  }
+
+  private static void assertGetValues(Get get, byte[] value)
+    throws IOException, InterruptedException {
+    for (int i = 0; i < NB_RETRIES; i++) {
+      if (i == NB_RETRIES - 1) {
+        fail("Waited too much time for put replication");
+      }
+      Result res = htable2.get(get);
+      if (res.isEmpty()) {
+        LOG.info("Row not available");
+        Thread.sleep(SLEEP_TIME);
+      } else {
+        assertArrayEquals(value, res.value());
+        break;
+      }
+    }
+  }
+
+  public static class TestCoprocessorForWALAnnotationAtSource
+    implements RegionCoprocessor, RegionObserver {
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void preWALAppend(ObserverContext<RegionCoprocessorEnvironment> ctx, WALKey key,
+      WALEdit edit) throws IOException {
+      key.addExtendedAttribute("extendedAttr1", Bytes.toBytes("Value of Extended attribute 01"));
+      key.addExtendedAttribute("extendedAttr2", Bytes.toBytes("Value of Extended attribute 02"));
+    }
+  }
+
+  public static class TestCoprocessorForWALAnnotationAtSink
+    implements RegionCoprocessor, RegionObserver {
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit)
+      throws IOException {
+      String attrVal1 = Bytes.toString(put.getAttribute("extendedAttr1"));
+      String attrVal2 = Bytes.toString(put.getAttribute("extendedAttr2"));
+      if (attrVal1 == null || attrVal2 == null) {
+        throw new IOException("Failed to retrieve WAL annotations");
+      }
+      if (
+        attrVal1.equals("Value of Extended attribute 01")
+          && attrVal2.equals("Value of Extended attribute 02")
+      ) {
+        return;
+      }
+      throw new IOException("Failed to retrieve WAL annotations..");
+    }
+
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      String attrVal1 = Bytes.toString(miniBatchOp.getOperation(0).getAttribute("extendedAttr1"));
+      String attrVal2 = Bytes.toString(miniBatchOp.getOperation(0).getAttribute("extendedAttr2"));
+      if (attrVal1 == null || attrVal2 == null) {
+        throw new IOException("Failed to retrieve WAL annotations");
+      }
+      if (
+        attrVal1.equals("Value of Extended attribute 01")
+          && attrVal2.equals("Value of Extended attribute 02")
+      ) {
+        return;
+      }
+      throw new IOException("Failed to retrieve WAL annotations..");
+    }
+  }
+
+  public static final class TestReplicationSinkRegionServerEndpoint

Review Comment:
   Sounds good, better than a new config key-value pair to toggle the behavior.



-- 
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 #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 39s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  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  |   2m 51s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 29s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 35s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 42s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 34s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 35s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 26s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 26s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 32s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   9m 53s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.4.  |
   | +1 :green_heart: |  spotless  |   0m 41s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  8s |  The patch does not generate ASF License warnings.  |
   |  |   |  32m 37s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/7/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux c7f68424bf56 5.4.0-1088-aws #96~18.04.1-Ubuntu SMP Mon Oct 17 02:57:48 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 4add5250ed |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/7/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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] virajjasani merged pull request #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani merged PR #4924:
URL: https://github.com/apache/hbase/pull/4924


-- 
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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   Let's get back to this.
   
   For me, I do not think adding WAL attributes to mutation directly is the correct way to deal with this. Even introducing a flag seems weird,
   
   Maybe we could add coprocessor support or add new type of filter for ReplicationSink, so Phoenix could add the special logic by your own, i.e, adding the WAL attributes to mutation.
   
   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] apurtell commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
apurtell commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1061745477


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -107,6 +107,14 @@ public class ReplicationSink {
   private SourceFSConfigurationProvider provider;
   private WALEntrySinkFilter walEntrySinkFilter;
 
+  /**
+   * If enabled at sink cluster site config, extended WAL attributes would be attached as Mutation
+   * attributes. This is useful for source cluster coproc to provide coproc specific metadata as WAL
+   * annotations and have them attached back to Mutations generated from WAL entries at sink side.
+   */
+  public static final String HBASE_REPLICATION_SINK_ATTRIBUTES_WAL_TO_MUTATIONS =

Review Comment:
   This doesn't need to be configurable. Mutation attributes will be ignored if they are not expected so this is harmless to clients. And IMHO the right thing to be doing. I agree with your premise.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +273,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (this.conf.getBoolean(HBASE_REPLICATION_SINK_ATTRIBUTES_WAL_TO_MUTATIONS, false)) {

Review Comment:
   Does not need to be conditional behavior.



-- 
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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046589604


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   While WALKey can be used by coproc (IA.LP), but since coprocs reply on hbase cross-cluster replication, the sink side mutation attributes can be expected to be set up by HBase since hbase prepares mutations from WAL entries.
   
   Only if WALKey extended attributes are set by coproc, hbase would attach the attributes to mutation. Does this sound good to you @Apache9 ?



-- 
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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  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  |   2m 28s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 59s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m  9s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 36s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 57s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 202m 51s |  hbase-server in the patch passed.  |
   |  |   | 221m 40s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 7ca95f074bf2 5.4.0-1092-aws #100~18.04.2-Ubuntu SMP Tue Nov 29 08:39:52 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 37c82a6209 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/1/testReport/ |
   | Max. process+thread count | 2366 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/1/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +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  |   3m  1s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 29s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 36s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 44s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 28s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 35s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 31s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 31s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   9m 49s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.4.  |
   | +1 :green_heart: |  spotless  |   0m 40s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 35s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 10s |  The patch does not generate ASF License warnings.  |
   |  |   |  33m  1s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/6/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux b2b898be0204 5.4.0-1088-aws #96~18.04.1-Ubuntu SMP Mon Oct 17 02:57:48 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 3f1087fe82 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/6/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 58s |  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  |   2m 40s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 46s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 35s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 48s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 48s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 45s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 210m 19s |  hbase-server in the patch passed.  |
   |  |   | 230m 27s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/4/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 87673ccd3b54 5.4.0-135-generic #152-Ubuntu SMP Wed Nov 23 20:19:22 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8b4e134f8c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/4/testReport/ |
   | Max. process+thread count | 2487 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/4/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1070525428


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -178,16 +179,19 @@ private void decorateConf() {
   /**
    * Replicate this array of entries directly into the local cluster using the native client. Only
    * operates against raw protobuf type saving on a conversion from pb to pojo.
+   * @param entries                    WAL entries to be replicated.
+   * @param cells                      cell scanner for iteration.
    * @param replicationClusterId       Id which will uniquely identify source cluster FS client
    *                                   configurations in the replication configuration directory
    * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
    *                                   directory
    * @param sourceHFileArchiveDirPath  Path that point to the source cluster hfile archive directory
+   * @param rsServerHost               regionserver coproc host.
    * @throws IOException If failed to replicate the data
    */
   public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
     String replicationClusterId, String sourceBaseNamespaceDirPath,
-    String sourceHFileArchiveDirPath) throws IOException {
+    String sourceHFileArchiveDirPath, RegionServerCoprocessorHost rsServerHost) throws IOException {

Review Comment:
   This is addressed. Thanks @Apache9!



-- 
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 #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

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

   Be lated +1.
   
   Thanks @virajjasani !


-- 
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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Provide RS coproc ability to attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1068598414


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -178,16 +179,19 @@ private void decorateConf() {
   /**
    * Replicate this array of entries directly into the local cluster using the native client. Only
    * operates against raw protobuf type saving on a conversion from pb to pojo.
+   * @param entries                    WAL entries to be replicated.
+   * @param cells                      cell scanner for iteration.
    * @param replicationClusterId       Id which will uniquely identify source cluster FS client
    *                                   configurations in the replication configuration directory
    * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
    *                                   directory
    * @param sourceHFileArchiveDirPath  Path that point to the source cluster hfile archive directory
+   * @param rsServerHost               regionserver coproc host.
    * @throws IOException If failed to replicate the data
    */
   public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
     String replicationClusterId, String sourceBaseNamespaceDirPath,
-    String sourceHFileArchiveDirPath) throws IOException {
+    String sourceHFileArchiveDirPath, RegionServerCoprocessorHost rsServerHost) throws IOException {

Review Comment:
   Correct, that's what I have added as part of this patch:
   
   ```
     @Override
     public void replicateLogEntries(List<AdminProtos.WALEntry> entries, CellScanner cells,
       String replicationClusterId, String sourceBaseNamespaceDirPath,
       String sourceHFileArchiveDirPath) throws IOException {
       RegionServerCoprocessorHost rsServerHost = null;
       if (server instanceof HRegionServer) {
         rsServerHost = ((HRegionServer) server).getRegionServerCoprocessorHost();
       }
       this.replicationSink.replicateEntries(entries, cells, replicationClusterId,
         sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath, rsServerHost);
     }
   
   ```
   
   So we do check for server instance to be regionserver, only then we pass non-null rs coproc host instance to this method.



-- 
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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1066265196


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java:
##########
@@ -166,6 +169,16 @@ public void call(RegionServerObserver observer) throws IOException {
     });
   }
 
+  public void preReplicationSinkBatchMutate(AdminProtos.WALEntry walEntry, Mutation mutation)

Review Comment:
   Added postXXX()



-- 
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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 42s |  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  |   2m  2s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 34s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 56s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 19s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 34s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 34s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 58s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 203m 30s |  hbase-server in the patch passed.  |
   |  |   | 221m 35s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/4/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 5ed485d31bf2 5.4.0-1092-aws #100~18.04.2-Ubuntu SMP Tue Nov 29 08:39:52 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8b4e134f8c |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/4/testReport/ |
   | Max. process+thread count | 2466 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/4/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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 #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

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

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   5m 19s |  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  |   2m 33s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 47s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 47s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 32s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 48s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 48s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 47s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 214m  7s |  hbase-server in the patch failed.  |
   |  |   | 238m 58s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/5/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4924 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e50a766ac1bc 5.4.0-135-generic #152-Ubuntu SMP Wed Nov 23 20:19:22 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 222ec684d6 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/5/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/5/testReport/ |
   | Max. process+thread count | 2565 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4924/5/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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 diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046588667


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   The goal of the WAL extended attribute is to pass some attributes down to the replication implementation from the coproc implementation, not to add it to the mutation's attribute. We should not mix things up.



-- 
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 diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046591975


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   > > The goal of the WAL extended attribute is to pass some attributes down to the replication implementation from the coproc implementation
   > 
   > That is correct but coproc would add it only so that it can consume it at sink side correct? Otherwise who else would read WAL attributes from WALEntry at sink side? HBase does it, while preparing for sink side mutations.
   
   The original design is to read it in replication endpoint, IIRC. And also, you can read it in ReplicationSink at the sink cluster, no problem.
   
   The problem here is you want to add the WAL attributes to mutation's attributes. This is a no no. We should not mix things up.



-- 
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] virajjasani commented on a diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
virajjasani commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046590676


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   > The goal of the WAL extended attribute is to pass some attributes down to the replication implementation from the coproc implementation
   
   That is correct but coproc would add it only so that it can consume it at sink side correct? Otherwise who else would read WAL attributes from WALEntry at sink side? HBase does it, while preparing for sink side mutations.



-- 
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 diff in pull request #4924: HBASE-27529 Attach WAL extended attributes to mutations at replication sink

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4924:
URL: https://github.com/apache/hbase/pull/4924#discussion_r1046592582


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
##########
@@ -265,6 +266,11 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
               mutation.setClusterIds(clusterIds);
               mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
                 HConstants.EMPTY_BYTE_ARRAY);
+              if (attributeList != null) {

Review Comment:
   And if phoenix wants this feature, please describe more about at lease one usage. Let's see how to better support it.



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