You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2021/05/06 12:05:14 UTC

[GitHub] [hbase] shahrs87 commented on a change in pull request #3222: HBASE-25612 [branch-1] HMaster should abort if ReplicationLogCleaner is not able to delete oldWALs.

shahrs87 commented on a change in pull request #3222:
URL: https://github.com/apache/hbase/pull/3222#discussion_r627357118



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
##########
@@ -132,42 +137,57 @@ public void setConf(Configuration config) {
       LOG.warn("Not configured - allowing all wals to be deleted");
       return;
     }
-    // Make my own Configuration.  Then I'll have my own connection to zk that
-    // I can close myself when comes time.
-    Configuration conf = new Configuration(config);
-    try {
-      setConf(conf, new ZooKeeperWatcher(conf, "replicationLogCleaner", null));
-    } catch (IOException e) {
-      LOG.error("Error while configuring " + this.getClass().getName(), e);
-    }
+    super.setConf(config);
   }
 
-  @InterfaceAudience.Private
-  public void setConf(Configuration conf, ZooKeeperWatcher zk) {
-    super.setConf(conf);
-    try {
-      this.zkw = zk;
-      this.replicationQueues = ReplicationFactory.getReplicationQueuesClient(zkw, conf,
-          new WarnOnlyAbortable());
-      this.replicationQueues.init();
-    } catch (ReplicationException e) {
-      LOG.error("Error while configuring " + this.getClass().getName(), e);
+  @Override
+  public void init(Map<String, Object> params) {
+    if (getConf() == null) {
+      // Replication is disabled so do nothing.
+      return;
+    }
+
+    if (MapUtils.isNotEmpty(params)) {
+      Object master = params.get(HMaster.MASTER);
+      if (master != null && master instanceof HMaster) {
+        this.master = (HMaster)master;
+        zkw = ((HMaster) master).getZooKeeper();
+        shareZK = true;
+      }
     }
+    init(getConf(), this.zkw, null);
   }
 
   @InterfaceAudience.Private
-  public void setConf(Configuration conf, ZooKeeperWatcher zk, 
+  public void init(Configuration conf, ZooKeeperWatcher zk,
       ReplicationQueuesClient replicationQueuesClient) {
     super.setConf(conf);
-    this.zkw = zk;
-    this.replicationQueues = replicationQueuesClient;
+    try {
+      if (zk != null) {
+        this.zkw = zk;
+      } else {
+        this.zkw = new ZooKeeperWatcher(getConf(), "replicationLogCleaner", null);
+      }
+      Preconditions.checkNotNull(this.zkw, "Zookeeper watcher cannot be null");
+      if (replicationQueuesClient != null) {
+        this.replicationQueues = replicationQueuesClient;
+      } else {
+        this.replicationQueues =
+          ReplicationFactory.getReplicationQueuesClient(zkw, getConf(), master);
+        this.replicationQueues.init();
+      }
+      Preconditions.checkNotNull(this.replicationQueues,
+        "ReplicationQueues cannot be null");
+    } catch (IOException | ReplicationException e) {
+      LOG.error("Error while configuring " + this.getClass().getName(), e);

Review comment:
       I kept the same behavior as before. https://github.com/apache/hbase/blob/master/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java#L114




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