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

stack pushed a commit to branch HBASE-18070
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 140ae167d9fd9d9d722c93ec3720f5051ec35537
Author: stack <st...@apache.org>
AuthorDate: Sat Oct 3 01:37:45 2020 -0700

    HBASE-25151 warmupRegion frustrates registering WALs on the catalog 
replicationsource
    
    warmupRegion called by Master on Region move will instatiate
    the meta WALProvider as part of its action making it so
    it is already created by the time we go to open the
    hbsae:meta Region. Accommodate meta walProvider
    being already up.
    
    
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
    
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
     Pass regionInfo. Needed internally.
    
    
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
     Add handling if meta wal provider already instantiated when
     addCatalogReplicationSource runs.
    
    
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplicationEndpoint.java
     Add exercising moving meta around between servers. Test replication
     keeps working.
---
 .../apache/hadoop/hbase/regionserver/HRegion.java  | 12 ++---
 .../hadoop/hbase/regionserver/RSRpcServices.java   | 28 +++-------
 .../regionserver/handler/AssignRegionHandler.java  |  3 +-
 .../handler/UnassignRegionHandler.java             |  3 +-
 .../replication/ReplicationSinkServiceImpl.java    |  3 +-
 .../regionserver/ReplicationSource.java            |  1 +
 .../regionserver/ReplicationSourceManager.java     | 61 +++++++++++++---------
 .../regionserver/ReplicationSyncUp.java            |  2 +-
 .../org/apache/hadoop/hbase/wal/WALFactory.java    |  4 +-
 .../TestMetaRegionReplicaReplicationEndpoint.java  | 27 ++++++++--
 .../regionserver/TestReplicationSource.java        |  4 ++
 .../regionserver/TestReplicationSourceManager.java |  2 +-
 12 files changed, 85 insertions(+), 65 deletions(-)

diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 67764b9..2f0b279 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -1203,11 +1203,11 @@ public class HRegion implements HeapSize, 
PropagatingConfigurationObserver, Regi
   private void initializeWarmup(final CancelableProgressable reporter) throws 
IOException {
     MonitoredTask status = TaskMonitor.get().createStatus("Initializing region 
" + this);
     // Initialize all the HStores
-    status.setStatus("Warming up all the Stores");
+    status.setStatus("Warmup all stores of " + 
this.getRegionInfo().getRegionNameAsString());
     try {
       initializeStores(reporter, status, true);
     } finally {
-      status.markComplete("Done warming up.");
+      status.markComplete("Warmed up " + 
this.getRegionInfo().getRegionNameAsString());
     }
   }
 
@@ -8182,14 +8182,9 @@ public class HRegion implements HeapSize, 
PropagatingConfigurationObserver, Regi
       throws IOException {
 
     Objects.requireNonNull(info, "RegionInfo cannot be null");
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("HRegion.Warming up region: " + info);
-    }
-
+    LOG.debug("Warmup {}", info);
     Path rootDir = CommonFSUtils.getRootDir(conf);
     Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
-
     FileSystem fs = null;
     if (rsServices != null) {
       fs = rsServices.getFileSystem();
@@ -8197,7 +8192,6 @@ public class HRegion implements HeapSize, 
PropagatingConfigurationObserver, Regi
     if (fs == null) {
       fs = rootDir.getFileSystem(conf);
     }
-
     HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, null);
     r.initializeWarmup(reporter);
   }
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 8aaddce..9efdaa9 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -2066,48 +2066,34 @@ public class RSRpcServices implements 
HBaseRPCErrorHandler,
   }
 
   /**
-   *  Wamrmup a region on this server.
-   *
-   * This method should only be called by Master. It synchrnously opens the 
region and
+   * Warmup a region on this server.
+   * This method should only be called by Master. It synchronously opens the 
region and
    * closes the region bringing the most important pages in cache.
-   * <p>
-   *
-   * @param controller the RPC controller
-   * @param request the request
-   * @throws ServiceException
    */
   @Override
   public WarmupRegionResponse warmupRegion(final RpcController controller,
       final WarmupRegionRequest request) throws ServiceException {
-
     final RegionInfo region = 
ProtobufUtil.toRegionInfo(request.getRegionInfo());
-    TableDescriptor htd;
     WarmupRegionResponse response = WarmupRegionResponse.getDefaultInstance();
-
     try {
       checkOpen();
       String encodedName = region.getEncodedName();
       byte[] encodedNameBytes = region.getEncodedNameAsBytes();
       final HRegion onlineRegion = regionServer.getRegion(encodedName);
-
       if (onlineRegion != null) {
-        LOG.info("Region already online. Skipping warming up " + region);
+        LOG.info("{} is online; skipping warmup", region);
         return response;
       }
-
-      htd = regionServer.tableDescriptors.get(region.getTable());
-
+      TableDescriptor htd = 
regionServer.tableDescriptors.get(region.getTable());
       if 
(regionServer.getRegionsInTransitionInRS().containsKey(encodedNameBytes)) {
-        LOG.info("Region is in transition. Skipping warmup " + region);
+        LOG.info("{} is in transition; skipping warmup", region);
         return response;
       }
-
-      LOG.info("Warming up region " + region.getRegionNameAsString());
+      LOG.info("Warmup {}", region.getRegionNameAsString());
       HRegion.warmupHRegion(region, htd, regionServer.getWAL(region),
           regionServer.getConfiguration(), regionServer, null);
-
     } catch (IOException ie) {
-      LOG.error("Failed warming up region " + region.getRegionNameAsString(), 
ie);
+      LOG.error("Failed warmup of {}", region.getRegionNameAsString(), ie);
       throw new ServiceException(ie);
     }
 
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
index 4ee6efc..5d9819c 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
@@ -138,7 +138,8 @@ public class AssignRegionHandler extends EventHandler {
       if (ServerRegionReplicaUtil.isMetaRegionReplicaReplicationEnabled(conf, 
tn)) {
         if 
(RegionReplicaUtil.isDefaultReplica(this.regionInfo.getReplicaId())) {
           // Add the hbase:meta replication source on replica zero/default.
-          
rs.getReplicationSourceService().getReplicationManager().addCatalogReplicationSource();
+          rs.getReplicationSourceService().getReplicationManager().
+            addCatalogReplicationSource(this.regionInfo);
         }
       }
       region = HRegion.openHRegion(regionInfo, htd, rs.getWAL(regionInfo), 
conf, rs, null);
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
index 1ed74bb..0d02f30 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
@@ -126,7 +126,8 @@ public class UnassignRegionHandler extends EventHandler {
       if 
(RegionReplicaUtil.isDefaultReplica(region.getRegionInfo().getReplicaId())) {
         // If hbase:meta read replicas enabled, remove replication source for 
hbase:meta Regions.
         // See assign region handler where we add the replication source on 
open.
-        
rs.getReplicationSourceService().getReplicationManager().removeCatalogReplicationSource();
+        rs.getReplicationSourceService().getReplicationManager().
+          removeCatalogReplicationSource(region.getRegionInfo());
       }
     }
     if (!rs.reportRegionStateTransition(
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationSinkServiceImpl.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationSinkServiceImpl.java
index 91dd8d0..edd5679 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationSinkServiceImpl.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationSinkServiceImpl.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSink;
+import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -64,7 +65,7 @@ public class ReplicationSinkServiceImpl implements 
ReplicationSinkService {
 
   @Override
   public void initialize(Server server, FileSystem fs, Path logdir, Path 
oldLogDir,
-    WALProvider walProvider) throws IOException {
+    WALFactory walFactory) throws IOException {
     this.server = server;
     this.conf = server.getConfiguration();
     this.statsPeriodInSecond =
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 34f9c86..9e5fb94 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -652,6 +652,7 @@ public class ReplicationSource implements 
ReplicationSourceInterface {
             }
           }
         } while ((this.startupOngoing.get() || this.retryStartup.get()) && 
!this.abortOnError);
+      });
     return this;
   }
 
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 8527f96..7b6937d 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -51,6 +51,8 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationListener;
@@ -68,6 +70,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
+import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -369,7 +372,7 @@ public class ReplicationSourceManager implements 
ReplicationListener {
   /**
    * @return a new 'classic' user-space replication source.
    * @param queueId the id of the replication queue to associate the 
ReplicationSource with.
-   * @see #createCatalogReplicationSource() for creating a ReplicationSource 
for hbase:meta.
+   * @see #createCatalogReplicationSource(RegionInfo) for creating a 
ReplicationSource for meta.
    */
   private ReplicationSourceInterface createSource(String queueId, 
ReplicationPeer replicationPeer)
       throws IOException {
@@ -1181,57 +1184,65 @@ public class ReplicationSourceManager implements 
ReplicationListener {
 
   /**
    * Add an hbase:meta Catalog replication source. Called on open of an 
hbase:meta Region.
-   * @see #removeCatalogReplicationSource()
+   * Create it once only. If exists already, use the existing one.
+   * @see #removeCatalogReplicationSource(RegionInfo)
+   * @see #addSource(String) This is specialization on the addSource method.
    */
-  public ReplicationSourceInterface addCatalogReplicationSource() throws 
IOException {
-    // Open/Create the hbase:meta ReplicationSource once only.
+  public ReplicationSourceInterface addCatalogReplicationSource(RegionInfo 
regionInfo)
+      throws IOException {
+    // Poor-man's putIfAbsent
     synchronized (this.catalogReplicationSource) {
       ReplicationSourceInterface rs = this.catalogReplicationSource.get();
       return rs != null ? rs :
-        
this.catalogReplicationSource.getAndSet(createCatalogReplicationSource());
+        
this.catalogReplicationSource.getAndSet(createCatalogReplicationSource(regionInfo));
     }
   }
 
   /**
    * Remove the hbase:meta Catalog replication source.
    * Called when we close hbase:meta.
-   * @see #addCatalogReplicationSource()
+   * @see #addCatalogReplicationSource(RegionInfo regionInfo)
    */
-  public void removeCatalogReplicationSource() {
+  public void removeCatalogReplicationSource(RegionInfo regionInfo) {
     // Nothing to do. Leave any CatalogReplicationSource in place in case an 
hbase:meta Region
     // comes back to this server.
   }
 
   /**
    * Create, initialize, and start the Catalog ReplicationSource.
+   * Presumes called one-time only (caller must ensure one-time only call).
+   * @see #addSource(String) This is a specialization of the addSource call.
    */
-  private ReplicationSourceInterface createCatalogReplicationSource() throws 
IOException {
-    // Has the hbase:meta WALProvider been instantiated?
+  private ReplicationSourceInterface createCatalogReplicationSource(RegionInfo 
regionInfo)
+      throws IOException {
+    // Instantiate meta walProvider. Instantiated here or over in the 
#warmupRegion call made by the
+    // Master on a 'move' operation. Need to do extra work if we did NOT 
instantiate the provider.
     WALProvider walProvider = this.walFactory.getMetaWALProvider();
-    boolean addListener = false;
-    if (walProvider == null) {
-      // The meta walProvider has not been instantiated. Create it.
+    boolean instantiate = walProvider == null;
+    if (instantiate) {
       walProvider = this.walFactory.getMetaProvider();
-      addListener = true;
     }
     CatalogReplicationSourcePeer peer = new 
CatalogReplicationSourcePeer(this.conf,
       this.clusterId.toString(), "meta_" + 
ServerRegionReplicaUtil.REGION_REPLICA_REPLICATION_PEER);
     final ReplicationSourceInterface crs = new CatalogReplicationSource();
     crs.init(conf, fs, this, new NoopReplicationQueueStorage(), peer, server, 
peer.getId(),
       clusterId, walProvider.getWALFileLengthProvider(), new 
MetricsSource(peer.getId()));
-    if (addListener) {
-      walProvider.addWALActionsListener(new WALActionsListener() {
-        @Override
-        public void postLogRoll(Path oldPath, Path newPath) throws IOException 
{
-          crs.enqueueLog(newPath);
-        }
-      });
-    } else {
-      // This is a problem. We'll have a ReplicationSource but no listener on 
hbase:meta WALs
-      // so nothing will be replicated.
-      LOG.error("Did not install WALActionsListener creating 
CatalogReplicationSource!");
+    // Add listener on the provider so we can pick up the WAL to replicate on 
roll.
+    WALActionsListener listener = new WALActionsListener() {
+      @Override public void postLogRoll(Path oldPath, Path newPath) throws 
IOException {
+        crs.enqueueLog(newPath);
+      }
+    };
+    walProvider.addWALActionsListener(listener);
+    if (!instantiate) {
+      // If we did not instantiate provider, need to add our listener on 
already-created WAL
+      // instance too (listeners are passed by provider to WAL instance on 
creation but if provider
+      // created already, our listener add above is missed). And add the 
current WAL file to the
+      // Replication Source so it can start replicating it.
+      WAL wal = walProvider.getWAL(regionInfo);
+      wal.registerWALActionsListener(listener);
+      crs.enqueueLog(((AbstractFSWAL)wal).getCurrentFileName());
     }
-    // Start this ReplicationSource.
     return crs.startup();
   }
 }
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index b04c7eb..a43be29 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -84,7 +84,7 @@ public class ReplicationSyncUp extends Configured implements 
Tool {
       System.out.println("Start Replication Server start");
       Replication replication = new Replication();
       replication.initialize(new DummyServer(zkw), fs, logDir, oldLogDir,
-        new WALFactory(conf, "test", false));
+        new WALFactory(conf, "test", null, false));
       ReplicationSourceManager manager = replication.getReplicationManager();
       manager.init().get();
       while (manager.activeFailoverTaskCount() > 0) {
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index b3663f6..6964bb8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -295,10 +295,10 @@ public class WALFactory {
   }
 
   /**
-   * @param region the region which we want to get a WAL for it. Could be null.
+   * @param region the region which we want to get a WAL for. Could be null.
    */
   public WAL getWAL(RegionInfo region) throws IOException {
-    // use different WAL for hbase:meta
+    // Use different WAL for hbase:meta. Instantiates the meta WALProvider if 
not already up.
     if (region != null && region.isMetaRegion() &&
       region.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
       return getMetaProvider().getWAL(region);
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplicationEndpoint.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplicationEndpoint.java
index 3bf0b9a..694b7a1 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplicationEndpoint.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplicationEndpoint.java
@@ -110,12 +110,13 @@ public class TestMetaRegionReplicaReplicationEndpoint {
    * Assert that the ReplicationSource for hbase:meta gets created when 
hbase:meta is opened.
    */
   @Test
-  public void testHBaseMetaReplicationSourceCreatedOnOpen()
-    throws IOException, InterruptedException {
+  public void testHBaseMetaReplicationSourceCreatedOnOpen() throws Exception {
     MiniHBaseCluster cluster = HTU.getMiniHBaseCluster();
     HRegionServer hrs = 
cluster.getRegionServer(cluster.getServerHoldingMeta());
+    // Replicate a row to prove all working.
+    testHBaseMetaReplicatesOneRow(0);
     assertTrue(isMetaRegionReplicaReplicationSource(hrs));
-    // Now move the hbase:meta and make sure the ReplicationSoruce is in both 
places.
+    // Now move the hbase:meta and make sure the ReplicationSource is in both 
places.
     HRegionServer hrsOther = null;
     for (int i = 0; i < cluster.getNumLiveRegionServers(); i++) {
       hrsOther = cluster.getRegionServer(i);
@@ -139,6 +140,26 @@ public class TestMetaRegionReplicaReplicationEndpoint {
     // Assert that there is a ReplicationSource in both places now.
     assertTrue(isMetaRegionReplicaReplicationSource(hrs));
     assertTrue(isMetaRegionReplicaReplicationSource(hrsOther));
+    // Replicate to show stuff still works.
+    testHBaseMetaReplicatesOneRow(1);
+    // Now pretend a few hours have gone by... roll the meta WAL in original 
location... Move the
+    // meta back and retry replication. See if it works.
+    hrs.getWAL(meta.getRegionInfo()).rollWriter(true);
+    testHBaseMetaReplicatesOneRow(2);
+    hrs.getWAL(meta.getRegionInfo()).rollWriter(true);
+    testHBaseMetaReplicatesOneRow(3);
+  }
+
+  /**
+   * Test meta region replica replication. Create some tables and see if 
replicas pick up the
+   * additions.
+   */
+  private void testHBaseMetaReplicatesOneRow(int i) throws Exception {
+    waitForMetaReplicasToOnline();
+    try (Table table = 
HTU.createTable(TableName.valueOf(this.name.getMethodName() + "_" + i),
+        HConstants.CATALOG_FAMILY)) {
+      verifyReplication(TableName.META_TABLE_NAME, NB_SERVERS, 
getMetaCells(table.getName()));
+    }
   }
 
   /**
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java
index 1455892..99f0ac6 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java
@@ -532,7 +532,11 @@ public class TestReplicationSource {
       FlakyReplicationEndpoint.class.getName());
     try {
       rs.startup();
+      assertTrue(rs.isSourceActive());
+      Waiter.waitFor(conf, 1000, () -> FaultyReplicationEndpoint.count > 0);
       Waiter.waitFor(conf, 1000, () -> rss.isAborted());
+      assertTrue(rss.isAborted());
+      Waiter.waitFor(conf, 1000, () -> !rs.isSourceActive());
       assertFalse(rs.isSourceActive());
     } finally {
       rs.terminate("Done");
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 4abb00f..2e35ef4 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -195,7 +195,7 @@ public abstract class TestReplicationSourceManager {
     remoteLogDir = 
utility.getDataTestDir(ReplicationUtils.REMOTE_WAL_DIR_NAME);
     replication = new Replication();
     replication.initialize(new DummyServer(), fs, logDir, oldLogDir,
-      new WALFactory(conf, "test", false));
+      new WALFactory(conf, "test", null, false));
     managerOfCluster = getManagerFromCluster();
     if (managerOfCluster != null) {
       // After replication procedure, we need to add peer by hand (other than 
by receiving

Reply via email to