You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2018/09/25 02:15:39 UTC

[bookkeeper] branch master updated: autorecovery-use-metadata-driver (part 2) : remove ZooKeeper from ReplicationWorker

This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new cd85888  autorecovery-use-metadata-driver (part 2) : remove ZooKeeper from ReplicationWorker
cd85888 is described below

commit cd85888086a819b20325d997680abe91ac3e786e
Author: Qi Wang <42...@users.noreply.github.com>
AuthorDate: Mon Sep 24 19:15:33 2018 -0700

    autorecovery-use-metadata-driver (part 2) : remove ZooKeeper from ReplicationWorker
    
    Descriptions of the changes in this PR:
    
    *Changes*
    
    This is the second part of changing AutoRecovery to use metadata driver. It removes
    the zookeeper reference from ReplicationWorker, because the zookeeper instance is not
    used anyway
    
    This change is based on #1693
    
    
    
    Author: Qi Wang <42...@users.noreply.github.com>
    Author: Qi Wang <co...@gmail.com>
    Author: Sijie Guo <gu...@gmail.com>
    Author: Charan Reddy Guttapalem <re...@gmail.com>
    
    Reviewers: Sijie Guo <si...@apache.org>, Enrico Olivelli <eo...@gmail.com>
    
    This closes #1694 from codingwangqi/autorecovery_metadata_part2
---
 .../bookkeeper/replication/AutoRecoveryMain.java   |  1 -
 .../bookkeeper/replication/ReplicationWorker.java  | 15 +++---------
 .../replication/AuditorPeriodicCheckTest.java      |  3 +--
 .../replication/TestReplicationWorker.java         | 28 +++++++++-------------
 4 files changed, 15 insertions(+), 32 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java
index 3369de8..b43ff85 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java
@@ -91,7 +91,6 @@ public class AutoRecoveryMain {
             getZooKeeperFromMetadataDriver(metadataBookieDriver),
             statsLogger.scope(AUDITOR_SCOPE));
         replicationWorker = new ReplicationWorker(
-            getZooKeeperFromMetadataDriver(metadataBookieDriver),
             conf,
             statsLogger.scope(REPLICATION_WORKER_SCOPE));
         deathWatcher = new AutoRecoveryDeathWatcher(this);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
index d5c85f1..b536fb0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
@@ -67,7 +67,6 @@ import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooKeeper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -83,7 +82,6 @@ public class ReplicationWorker implements Runnable {
 
     private final LedgerUnderreplicationManager underreplicationManager;
     private final ServerConfiguration conf;
-    private final ZooKeeper zkc;
     private volatile boolean workerRunning = false;
     private final BookKeeperAdmin admin;
     private final LedgerChecker ledgerChecker;
@@ -107,16 +105,13 @@ public class ReplicationWorker implements Runnable {
      * UnderReplicationManager to the targetBookie. This target bookie will be a
      * local bookie.
      *
-     * @param zkc
-     *            - ZK instance
      * @param conf
      *            - configurations
      */
-    public ReplicationWorker(final ZooKeeper zkc,
-                             final ServerConfiguration conf)
+    public ReplicationWorker(final ServerConfiguration conf)
             throws CompatibilityException, KeeperException,
             InterruptedException, IOException {
-        this(zkc, conf, NullStatsLogger.INSTANCE);
+        this(conf, NullStatsLogger.INSTANCE);
     }
 
     /**
@@ -124,19 +119,15 @@ public class ReplicationWorker implements Runnable {
      * UnderReplicationManager to the targetBookie. This target bookie will be a
      * local bookie.
      *
-     * @param zkc
-     *            - ZK instance
      * @param conf
      *            - configurations
      * @param statsLogger
      *            - stats logger
      */
-    public ReplicationWorker(final ZooKeeper zkc,
-                             final ServerConfiguration conf,
+    public ReplicationWorker(final ServerConfiguration conf,
                              StatsLogger statsLogger)
             throws CompatibilityException, KeeperException,
             InterruptedException, IOException {
-        this.zkc = zkc;
         this.conf = conf;
         try {
             this.bkc = BookKeeper.forConfig(new ClientConfiguration(conf))
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java
index 4e3bda8..0668088 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java
@@ -449,8 +449,7 @@ public class AuditorPeriodicCheckTest extends BookKeeperClusterTestCase {
         // now start the replication workers
         List<ReplicationWorker> l = new ArrayList<ReplicationWorker>();
         for (int i = 0; i < numBookies; i++) {
-            ReplicationWorker rw = new ReplicationWorker(
-                    zkc, bsConfs.get(i), NullStatsLogger.INSTANCE);
+            ReplicationWorker rw = new ReplicationWorker(bsConfs.get(i), NullStatsLogger.INSTANCE);
             rw.start();
             l.add(rw);
         }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
index e4b1232..a4b280a 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
@@ -52,7 +52,6 @@ import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
 import org.apache.bookkeeper.util.BookKeeperConstants;
 import org.apache.bookkeeper.zookeeper.ZooKeeperClient;
-import org.apache.zookeeper.ZooKeeper;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -153,7 +152,7 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
             lh.addEntry(data);
         }
 
-        ReplicationWorker rw = new ReplicationWorker(zkc, baseConf);
+        ReplicationWorker rw = new ReplicationWorker(baseConf);
 
         rw.start();
         try {
@@ -198,7 +197,7 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
         LOG.info("New Bookie addr :" + newBkAddr);
 
         killAllBookies(lh, newBkAddr);
-        ReplicationWorker rw = new ReplicationWorker(zkc, baseConf);
+        ReplicationWorker rw = new ReplicationWorker(baseConf);
 
         rw.start();
         try {
@@ -248,16 +247,12 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
         // Starte RW1
         BookieSocketAddress newBkAddr1 = startNewBookieAndReturnAddress();
         LOG.info("New Bookie addr : {}", newBkAddr1);
-        ReplicationWorker rw1 = new ReplicationWorker(zkc, baseConf);
+        ReplicationWorker rw1 = new ReplicationWorker(baseConf);
 
         // Starte RW2
         BookieSocketAddress newBkAddr2 = startNewBookieAndReturnAddress();
         LOG.info("New Bookie addr : {}", newBkAddr2);
-        ZooKeeper zkc1 = ZooKeeperClient.newBuilder()
-                .connectString(zkUtil.getZooKeeperConnectString())
-                .sessionTimeoutMs(10000)
-                .build();
-        ReplicationWorker rw2 = new ReplicationWorker(zkc1, baseConf);
+        ReplicationWorker rw2 = new ReplicationWorker(baseConf);
         rw1.start();
         rw2.start();
 
@@ -283,7 +278,6 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
         } finally {
             rw1.shutdown();
             rw2.shutdown();
-            zkc1.close();
         }
     }
 
@@ -308,7 +302,7 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
 
         BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress();
         LOG.info("New Bookie addr : {}", newBkAddr);
-        ReplicationWorker rw = new ReplicationWorker(zkc, baseConf);
+        ReplicationWorker rw = new ReplicationWorker(baseConf);
         rw.start();
 
         try {
@@ -363,7 +357,7 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
         BookieSocketAddress newBkAddr = startNewBookieAndReturnAddress();
         LOG.info("New Bookie addr : {}", newBkAddr);
 
-        ReplicationWorker rw = new ReplicationWorker(zkc, baseConf);
+        ReplicationWorker rw = new ReplicationWorker(baseConf);
 
         rw.start();
         try {
@@ -419,7 +413,7 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
 
         // set to 3s instead of default 30s
         baseConf.setOpenLedgerRereplicationGracePeriod("3000");
-        ReplicationWorker rw = new ReplicationWorker(zkc, baseConf);
+        ReplicationWorker rw = new ReplicationWorker(baseConf);
 
         @Cleanup MetadataClientDriver clientDriver = MetadataDrivers.getClientDriver(
             URI.create(baseClientConf.getMetadataServiceUri()));
@@ -489,8 +483,8 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
 
         // create couple of replicationworkers
         baseConf.setLockReleaseOfFailedLedgerGracePeriod("500");
-        ReplicationWorker rw1 = new ReplicationWorker(zkc, baseConf);
-        ReplicationWorker rw2 = new ReplicationWorker(zkc, baseConf);
+        ReplicationWorker rw1 = new ReplicationWorker(baseConf);
+        ReplicationWorker rw2 = new ReplicationWorker(baseConf);
 
         @Cleanup
         MetadataClientDriver clientDriver = MetadataDrivers
@@ -610,7 +604,7 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
             lh.addEntry(data);
         }
 
-        ReplicationWorker rw = new ReplicationWorker(zkc, baseConf);
+        ReplicationWorker rw = new ReplicationWorker(baseConf);
 
         baseClientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
 
@@ -660,7 +654,7 @@ public class TestReplicationWorker extends BookKeeperClusterTestCase {
                 .sessionTimeoutMs(10000)
                 .build()) {
 
-            ReplicationWorker rw = new ReplicationWorker(zk, baseConf);
+            ReplicationWorker rw = new ReplicationWorker(baseConf);
             rw.start();
             for (int i = 0; i < 10; i++) {
                 if (rw.isRunning()) {