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 2023/01/12 20:08:20 UTC

[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

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