You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/08/28 23:50:20 UTC

[GitHub] [hbase] bharathv commented on a change in pull request #2284: HBASE-24764: Add support of adding default peer configs via hbase-site.xml for all replication peers.

bharathv commented on a change in pull request #2284:
URL: https://github.com/apache/hbase/pull/2284#discussion_r479576404



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -232,6 +233,9 @@ public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean ena
       // this should be a retry, just return
       return;
     }
+    ReplicationPeerConfig updatedPeerConfig = ReplicationPeerConfigUtil.
+      addBasePeerConfigsIfNotPresent(conf,peerConfig);
+    peerConfig = updatedPeerConfig;

Review comment:
       merge into a single line.
   
   peerConfig = ReplicatinPeerConfigUtil..addPeer..IfNotPresent()

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
##########
@@ -450,6 +453,45 @@ public static ReplicationPeerConfig appendTableCFsToReplicationPeerConfig(
     return builder.build();
   }
 
+  /**
+   * Helper method to add base peer configs from Configuration to ReplicationPeerConfig
+   * if not present in latter.
+   *
+   * This merges the user supplied peer configuration
+   * {@link org.apache.hadoop.hbase.replication.ReplicationPeerConfig} with peer configs
+   * provided as property hbase.replication.peer.base.configs in hbase configuration.
+   * Expected format for this hbase configuration is "k1=v1;k2=v2,v2_1". Original value
+   * of conf is retained if already present in ReplicationPeerConfig.
+   *
+   * @param conf Configuration
+   * @return ReplicationPeerConfig containing updated configs.
+   */
+  public static ReplicationPeerConfig addBasePeerConfigsIfNotPresent(Configuration conf,
+    ReplicationPeerConfig receivedPeerConfig) {
+    String basePeerConfigs = conf.get(HBASE_REPLICATION_PEER_BASE_CONFIG, null);

Review comment:
       nit: Use the default as empty string? That way we can avoid null check in if, not a big deal, just to be consistent with other similar usages.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
##########
@@ -450,6 +453,45 @@ public static ReplicationPeerConfig appendTableCFsToReplicationPeerConfig(
     return builder.build();
   }
 
+  /**
+   * Helper method to add base peer configs from Configuration to ReplicationPeerConfig
+   * if not present in latter.
+   *
+   * This merges the user supplied peer configuration
+   * {@link org.apache.hadoop.hbase.replication.ReplicationPeerConfig} with peer configs
+   * provided as property hbase.replication.peer.base.configs in hbase configuration.
+   * Expected format for this hbase configuration is "k1=v1;k2=v2,v2_1". Original value
+   * of conf is retained if already present in ReplicationPeerConfig.
+   *
+   * @param conf Configuration
+   * @return ReplicationPeerConfig containing updated configs.
+   */
+  public static ReplicationPeerConfig addBasePeerConfigsIfNotPresent(Configuration conf,
+    ReplicationPeerConfig receivedPeerConfig) {
+    String basePeerConfigs = conf.get(HBASE_REPLICATION_PEER_BASE_CONFIG, null);
+
+    ReplicationPeerConfigBuilder copiedPeerConfigBuilder = ReplicationPeerConfig.
+      newBuilder(receivedPeerConfig);
+    Map<String,String> receivedPeerConfigMap = receivedPeerConfig.getConfiguration();
+

Review comment:
       nit: Remove multiple extraneous new lines in this method.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -546,9 +550,13 @@ public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf, St
     ConcurrentMap<String, ReplicationPeerDescription> peers = new ConcurrentHashMap<>();
     for (String peerId : peerStorage.listPeerIds()) {
       ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
+
+      ReplicationPeerConfig updatedPeerConfig = ReplicationPeerConfigUtil.

Review comment:
       peerConfig = ReplicationpeerConfigUtil.....() (avoid unnecessary temp variable)

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -546,9 +550,13 @@ public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf, St
     ConcurrentMap<String, ReplicationPeerDescription> peers = new ConcurrentHashMap<>();
     for (String peerId : peerStorage.listPeerIds()) {
       ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
+
+      ReplicationPeerConfig updatedPeerConfig = ReplicationPeerConfigUtil.
+        addBasePeerConfigsIfNotPresent(conf,peerConfig);
+      peerStorage.updatePeerConfig(peerId,updatedPeerConfig);

Review comment:
       nit: , updatedPeerConfig (space).

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
##########
@@ -232,6 +233,9 @@ public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean ena
       // this should be a retry, just return
       return;
     }
+    ReplicationPeerConfig updatedPeerConfig = ReplicationPeerConfigUtil.
+      addBasePeerConfigsIfNotPresent(conf,peerConfig);

Review comment:
       nit: config, peerConfig (space)

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
##########
@@ -450,6 +453,45 @@ public static ReplicationPeerConfig appendTableCFsToReplicationPeerConfig(
     return builder.build();
   }
 
+  /**
+   * Helper method to add base peer configs from Configuration to ReplicationPeerConfig
+   * if not present in latter.
+   *
+   * This merges the user supplied peer configuration
+   * {@link org.apache.hadoop.hbase.replication.ReplicationPeerConfig} with peer configs
+   * provided as property hbase.replication.peer.base.configs in hbase configuration.
+   * Expected format for this hbase configuration is "k1=v1;k2=v2,v2_1". Original value
+   * of conf is retained if already present in ReplicationPeerConfig.
+   *
+   * @param conf Configuration
+   * @return ReplicationPeerConfig containing updated configs.
+   */
+  public static ReplicationPeerConfig addBasePeerConfigsIfNotPresent(Configuration conf,
+    ReplicationPeerConfig receivedPeerConfig) {
+    String basePeerConfigs = conf.get(HBASE_REPLICATION_PEER_BASE_CONFIG, null);
+
+    ReplicationPeerConfigBuilder copiedPeerConfigBuilder = ReplicationPeerConfig.
+      newBuilder(receivedPeerConfig);
+    Map<String,String> receivedPeerConfigMap = receivedPeerConfig.getConfiguration();
+
+    if (basePeerConfigs != null && basePeerConfigs.length() != 0) {
+
+      Map<String, String> basePeerConfigMap = Splitter.on(';').trimResults().omitEmptyStrings()
+        .withKeyValueSeparator("=").split(basePeerConfigs);
+
+      for (Map.Entry<String,String> entry : basePeerConfigMap.entrySet()) {
+        String configName = entry.getKey();
+        String configValue = entry.getValue();
+        // Only override if base config does not exist in existing peer configs
+        if (!receivedPeerConfigMap.containsKey(configName)) {
+          copiedPeerConfigBuilder.putConfiguration(configName,configValue);

Review comment:
       nit:, configValue (space)




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

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