You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2015/12/03 22:33:22 UTC

[3/3] hbase git commit: Revert "HBASE-14223 Meta WALs are not cleared if meta region was closed and RS aborts"

Revert "HBASE-14223 Meta WALs are not cleared if meta region was closed and RS aborts"

This reverts commit 5f9e8a879cc4f3ace09ecb65aa6109e65ff00001.


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c4aab43a
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c4aab43a
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c4aab43a

Branch: refs/heads/branch-1.0
Commit: c4aab43a1ac1617890bb4535c9611fa0e31a59c2
Parents: 5f9e8a8
Author: Enis Soztutar <en...@apache.org>
Authored: Thu Dec 3 13:32:52 2015 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Thu Dec 3 13:32:52 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HRegionInfo.java    |   2 +-
 .../hbase/chaos/actions/MoveMetaAction.java     |  36 ----
 .../chaos/factories/NoKillMonkeyFactory.java    |   4 +-
 .../SlowDeterministicMonkeyFactory.java         |   7 +-
 .../StressAssignmentManagerMonkeyFactory.java   |   7 +-
 .../hbase/regionserver/HRegionServer.java       |  20 ---
 .../hadoop/hbase/regionserver/LogRoller.java    |   5 +-
 .../hbase/regionserver/RSRpcServices.java       |   6 +-
 .../regionserver/RegionServerServices.java      |   6 -
 .../regionserver/handler/CloseMetaHandler.java  |   8 -
 .../handler/CloseRegionHandler.java             |  15 +-
 .../org/apache/hadoop/hbase/wal/WALFactory.java |  15 --
 .../hadoop/hbase/HBaseTestingUtility.java       |  46 -----
 .../hadoop/hbase/MockRegionServerServices.java  |   4 -
 .../hadoop/hbase/master/MockRegionServer.java   |   4 -
 .../hadoop/hbase/wal/TestMetaWALsAreClosed.java | 166 -------------------
 16 files changed, 13 insertions(+), 338 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index dbc25af..c53ecce 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -141,7 +141,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
   public static final byte REPLICA_ID_DELIMITER = (byte)'_';
 
   private static final int MAX_REPLICA_ID = 0xFFFF;
-  public static final int DEFAULT_REPLICA_ID = 0;
+  static final int DEFAULT_REPLICA_ID = 0;
   /**
    * Does region name contain its encoded name?
    * @param regionName region name

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveMetaAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveMetaAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveMetaAction.java
deleted file mode 100644
index e38ed21..0000000
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveMetaAction.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.chaos.actions;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
-
-/**
- * Action that moves the meta table region(s).
- */
-public class MoveMetaAction extends MoveRegionsOfTableAction {
-
-  public MoveMetaAction() {
-    this(-1, MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME);
-  }
-
-  public MoveMetaAction(long sleepTime, long maxSleepTime) {
-    super(sleepTime, maxSleepTime, TableName.META_TABLE_NAME);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/NoKillMonkeyFactory.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/NoKillMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/NoKillMonkeyFactory.java
index 12b2163..def9df0 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/NoKillMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/NoKillMonkeyFactory.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.chaos.actions.DumpClusterStatusAction;
 import org.apache.hadoop.hbase.chaos.actions.FlushRandomRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.FlushTableAction;
 import org.apache.hadoop.hbase.chaos.actions.MergeRandomAdjacentRegionsOfTableAction;
-import org.apache.hadoop.hbase.chaos.actions.MoveMetaAction;
 import org.apache.hadoop.hbase.chaos.actions.MoveRandomRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.MoveRegionsOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.RemoveColumnAction;
@@ -53,8 +52,7 @@ public class NoKillMonkeyFactory extends MonkeyFactory {
             MonkeyConstants.DEFAULT_COMPACT_RANDOM_REGION_RATIO),
         new FlushTableAction(tableName),
         new FlushRandomRegionOfTableAction(tableName),
-        new MoveRandomRegionOfTableAction(tableName),
-        new MoveMetaAction()
+        new MoveRandomRegionOfTableAction(tableName)
     };
 
     Action[] actions2 = new Action[] {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/SlowDeterministicMonkeyFactory.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/SlowDeterministicMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/SlowDeterministicMonkeyFactory.java
index b8eec67..6195737 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/SlowDeterministicMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/SlowDeterministicMonkeyFactory.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.chaos.actions.DumpClusterStatusAction;
 import org.apache.hadoop.hbase.chaos.actions.FlushRandomRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.FlushTableAction;
 import org.apache.hadoop.hbase.chaos.actions.MergeRandomAdjacentRegionsOfTableAction;
-import org.apache.hadoop.hbase.chaos.actions.MoveMetaAction;
 import org.apache.hadoop.hbase.chaos.actions.MoveRandomRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.MoveRegionsOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.RemoveColumnAction;
@@ -78,8 +77,7 @@ public class SlowDeterministicMonkeyFactory extends MonkeyFactory {
         new CompactRandomRegionOfTableAction(tableName, compactRandomRegionRatio),
         new FlushTableAction(tableName),
         new FlushRandomRegionOfTableAction(tableName),
-        new MoveRandomRegionOfTableAction(tableName),
-        new MoveMetaAction()
+        new MoveRandomRegionOfTableAction(tableName)
     };
 
     // Actions such as split/merge/snapshot.
@@ -107,8 +105,7 @@ public class SlowDeterministicMonkeyFactory extends MonkeyFactory {
         new RestartActiveMasterAction(restartActiveMasterSleepTime),
         new RollingBatchRestartRsAction(rollingBatchRestartRSSleepTime,
             rollingBatchRestartRSRatio),
-        new RestartRsHoldingMetaAction(restartRsHoldingMetaSleepTime),
-        new MoveMetaAction()
+        new RestartRsHoldingMetaAction(restartRsHoldingMetaSleepTime)
     };
 
     // Action to log more info for debugging

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/StressAssignmentManagerMonkeyFactory.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/StressAssignmentManagerMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/StressAssignmentManagerMonkeyFactory.java
index 61ec32a..befb2fa 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/StressAssignmentManagerMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/StressAssignmentManagerMonkeyFactory.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.chaos.actions.DumpClusterStatusAction;
 import org.apache.hadoop.hbase.chaos.actions.FlushRandomRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.FlushTableAction;
 import org.apache.hadoop.hbase.chaos.actions.MergeRandomAdjacentRegionsOfTableAction;
-import org.apache.hadoop.hbase.chaos.actions.MoveMetaAction;
 import org.apache.hadoop.hbase.chaos.actions.MoveRandomRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.MoveRegionsOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.RemoveColumnAction;
@@ -50,8 +49,7 @@ public class StressAssignmentManagerMonkeyFactory extends MonkeyFactory {
         new CompactTableAction(tableName, 0.5f),
         new CompactRandomRegionOfTableAction(tableName, 0.6f),
         new FlushTableAction(tableName),
-        new FlushRandomRegionOfTableAction(tableName),
-        new MoveMetaAction()
+        new FlushRandomRegionOfTableAction(tableName)
     };
 
     Action[] actions2 = new Action[] {
@@ -64,8 +62,7 @@ public class StressAssignmentManagerMonkeyFactory extends MonkeyFactory {
         new RestartRandomRsAction(60000),
         new BatchRestartRsAction(5000, 0.5f),
         new RollingBatchRestartRsAction(5000, 1.0f),
-        new RestartRsHoldingMetaAction(35000),
-        new MoveMetaAction()
+        new RestartRsHoldingMetaAction(35000)
     };
 
     // Action to log more info for debugging

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 9472d34..7a01934 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1771,26 +1771,6 @@ public class HRegionServer extends HasThread implements
   }
 
   @Override
-  public void releaseWAL(HRegionInfo regionInfo, WAL wal) throws IOException {
-    if (regionInfo != null && regionInfo.isMetaTable() &&
-        regionInfo.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
-
-      walFactory.closeMetaWAL(regionInfo.getEncodedNameAsBytes());
-
-      LogRoller roller;
-      if (regionInfo != null && regionInfo.isMetaTable() &&
-          regionInfo.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
-        roller = metawalRoller.get();
-        if (roller != null) {
-          roller.removeWAL(wal); // only do this for meta WAL
-        }
-
-        // TODO: meta wal roller is left running. Should be fine.
-      }
-    }
-  }
-
-  @Override
   public ClusterConnection getConnection() {
     return this.clusterConnection;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
index 9395af8..3dcc69e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
@@ -82,10 +82,6 @@ public class LogRoller extends HasThread {
     }
   }
 
-  public void removeWAL(final WAL wal) {
-    walNeedsRoll.remove(wal);
-  }
-
   public void requestRollAll() {
     for (WAL wal : walNeedsRoll.keySet()) {
       walNeedsRoll.put(wal, Boolean.TRUE);
@@ -187,4 +183,5 @@ public class LogRoller extends HasThread {
         requester);
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
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 bcefd3e..863d4eb 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
@@ -152,6 +152,9 @@ import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -160,9 +163,6 @@ import org.apache.hadoop.hbase.util.DNS;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Strings;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
 import org.apache.zookeeper.KeeperException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
index 43063cc..68ee3e0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
@@ -50,12 +50,6 @@ public interface RegionServerServices
   WAL getWAL(HRegionInfo regionInfo) throws IOException;
 
   /**
-   * Releases the dependency of this region to the WAL previously obtained from
-   * {@link #getWAL(HRegionInfo)}.
-   */
-  void releaseWAL(HRegionInfo regionInfo, WAL wal) throws IOException;
-
-  /**
    * @return Implementation of {@link CompactionRequestor} or null.
    */
   CompactionRequestor getCompactionRequester();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
index 72d06d2..dba9240 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java
@@ -19,9 +19,6 @@
 package org.apache.hadoop.hbase.regionserver.handler;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-import java.io.IOException;
-
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.executor.EventType;
@@ -43,9 +40,4 @@ public class CloseMetaHandler extends CloseRegionHandler {
     super(server, rsServices, regionInfo, abort, closeRegionCoordination,
       crd, EventType.M_RS_CLOSE_META);
   }
-
-  @Override
-  protected void releaseWALIfNeeded() throws IOException {
-    rsServices.releaseWAL(regionInfo, region.getWAL());
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
index 36b54fd..9e7786f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
@@ -46,9 +46,8 @@ public class CloseRegionHandler extends EventHandler {
   // have a running queue of user regions to close?
   private static final Log LOG = LogFactory.getLog(CloseRegionHandler.class);
 
-  protected final RegionServerServices rsServices;
-  protected final HRegionInfo regionInfo;
-  protected HRegion region;
+  private final RegionServerServices rsServices;
+  private final HRegionInfo regionInfo;
 
   // If true, the hosting server is aborting.  Region close process is different
   // when we are aborting.
@@ -120,7 +119,7 @@ public class CloseRegionHandler extends EventHandler {
       LOG.debug("Processing close of " + name);
       String encodedRegionName = regionInfo.getEncodedName();
       // Check that this region is being served here
-      region = this.rsServices.getFromOnlineRegions(encodedRegionName);
+      HRegion region = this.rsServices.getFromOnlineRegions(encodedRegionName);
       if (region == null) {
         LOG.warn("Received CLOSE for region " + name + " but currently not serving - ignoring");
         // TODO: do better than a simple warning
@@ -144,10 +143,6 @@ public class CloseRegionHandler extends EventHandler {
             regionInfo.getRegionNameAsString());
           return;
         }
-
-        if (!abort) {
-          releaseWALIfNeeded();
-        }
       } catch (IOException ioe) {
         // An IOException here indicates that we couldn't successfully flush the
         // memstore before closing. So, we need to abort the server and allow
@@ -172,8 +167,4 @@ public class CloseRegionHandler extends EventHandler {
           remove(this.regionInfo.getEncodedNameAsBytes());
     }
   }
-
-  protected void releaseWALIfNeeded() throws IOException {
-    // release the WAL if needed. Only meta does this for now.
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
----------------------------------------------------------------------
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 aa244ba..4ef320a 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
@@ -250,21 +250,6 @@ public class WALFactory {
     return metaProvider.getWAL(identifier);
   }
 
-  /**
-   * Closes the meta WAL and meta WALProvider
-   */
-  public void closeMetaWAL(final byte[] identifier) throws IOException {
-    // NOTE: this assumes single META region. The close of WAL does not do ref-counting for the
-    // number of regions depending on the meta WAL
-    WALProvider metaProvider = this.metaProvider.get();
-    if (metaProvider != null) {
-      if (this.metaProvider.compareAndSet(metaProvider, null)) {
-        // close the metaProvider
-        metaProvider.close();
-      }
-    }
-  }
-
   public Reader createReader(final FileSystem fs, final Path path) throws IOException {
     return createReader(fs, path, (CancelableProgressable)null);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 729548c..6521739 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -59,7 +58,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
@@ -2137,50 +2135,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     }
   }
 
-  public void verifyNumericRows(Table table, final byte[] f, int startRow, int endRow,
-      int replicaId)
-      throws IOException {
-    for (int i = startRow; i < endRow; i++) {
-      String failMsg = "Failed verification of row :" + i;
-      byte[] data = Bytes.toBytes(String.valueOf(i));
-      Get get = new Get(data);
-      get.setReplicaId(replicaId);
-      get.setConsistency(Consistency.TIMELINE);
-      Result result = table.get(get);
-      assertTrue(failMsg, result.containsColumn(f, null));
-      assertEquals(failMsg, result.getColumnCells(f, null).size(), 1);
-      Cell cell = result.getColumnLatestCell(f, null);
-      assertTrue(failMsg,
-        Bytes.equals(data, 0, data.length, cell.getValueArray(), cell.getValueOffset(),
-          cell.getValueLength()));
-    }
-  }
-
-  public void verifyNumericRows(HRegion region, final byte[] f, int startRow, int endRow)
-      throws IOException {
-    verifyNumericRows(region, f, startRow, endRow, true);
-  }
-
-  public void verifyNumericRows(HRegion region, final byte[] f, int startRow, int endRow,
-      final boolean present) throws IOException {
-    for (int i = startRow; i < endRow; i++) {
-      String failMsg = "Failed verification of row :" + i;
-      byte[] data = Bytes.toBytes(String.valueOf(i));
-      Result result = region.get(new Get(data));
-
-      boolean hasResult = result != null && !result.isEmpty();
-      assertEquals(failMsg + result, present, hasResult);
-      if (!present) continue;
-
-      assertTrue(failMsg, result.containsColumn(f, null));
-      assertEquals(failMsg, result.getColumnCells(f, null).size(), 1);
-      Cell cell = result.getColumnLatestCell(f, null);
-      assertTrue(failMsg,
-        Bytes.equals(data, 0, data.length, cell.getValueArray(), cell.getValueOffset(),
-          cell.getValueLength()));
-    }
-  }
-
   public void deleteNumericRows(final Table t, final byte[] f, int startRow, int endRow) throws IOException {
     for (int i = startRow; i < endRow; i++) {
       byte[] data = Bytes.toBytes(String.valueOf(i));

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index 0813479..0d0e780 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -283,8 +283,4 @@ public class MockRegionServerServices implements RegionServerServices {
   public HeapMemoryManager getHeapMemoryManager() {
     return null;
   }
-
-  @Override
-  public void releaseWAL(HRegionInfo regionInfo, WAL wal) throws IOException {
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index d1bb072..607f1e1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -619,8 +619,4 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   public HeapMemoryManager getHeapMemoryManager() {
     return null;
   }
-
-  @Override
-  public void releaseWAL(HRegionInfo regionInfo, WAL wal) throws IOException {
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4aab43a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestMetaWALsAreClosed.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestMetaWALsAreClosed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestMetaWALsAreClosed.java
deleted file mode 100644
index 040ce66..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestMetaWALsAreClosed.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.wal;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter.Predicate;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Tests the case where a meta region is opened in one regionserver and closed, there should not
- * be any WALs left over.
- */
-@Category({MediumTests.class})
-public class TestMetaWALsAreClosed {
-  protected static final Log LOG = LogFactory.getLog(TestMetaWALsAreClosed.class);
-
-  protected static final int NUM_RS = 2;
-
-  protected static final HBaseTestingUtility TEST_UTIL =
-      new HBaseTestingUtility();
-
-
-  protected final Configuration conf = TEST_UTIL.getConfiguration();
-
-  @Before
-  public void setUp() throws Exception {
-    TEST_UTIL.startMiniCluster(1, NUM_RS);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  private boolean isHostingMeta(FileSystem fs, Path wals, ServerName serverName)
-      throws IOException {
-    for (FileStatus status : fs.listStatus(wals)) {
-      LOG.info(status.getPath());
-      if (DefaultWALProvider.isMetaFile(status.getPath())) {
-        return true; // only 1 meta region for now
-      }
-    }
-    return false;
-  }
-
-  private void moveMetaRegionAndWait(final ServerName target) throws Exception {
-    try (final Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
-        final Admin admin = conn.getAdmin();
-        final RegionLocator rl = conn.getRegionLocator(TableName.META_TABLE_NAME)) {
-
-      LOG.info("Disabling balancer");
-      admin.setBalancerRunning(false, true);
-
-      LOG.info("Moving meta region");
-      admin.move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
-          Bytes.toBytes(target.toString()));
-
-      LOG.info("Waiting for meta region to move");
-      // wait for the move of meta region
-      TEST_UTIL.waitFor(30000, new Predicate<Exception>() {
-        @Override
-        public boolean evaluate() throws Exception {
-          return target.equals(
-              rl.getRegionLocation(HConstants.EMPTY_START_ROW, true).getServerName());
-        }
-      });
-    }
-  }
-
-  @Test (timeout = 60000)
-  public void testMetaWALsAreClosed() throws Exception {
-    MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
-
-    FileSystem fs = TEST_UTIL.getTestFileSystem();
-
-    // find the region server hosting the meta table now.
-    ServerName metaServerName = null;
-    ServerName otherServerName = null;
-    for (RegionServerThread rs : cluster.getRegionServerThreads()) {
-      ServerName serverName = rs.getRegionServer().getServerName();
-
-      Path wals = new Path(FSUtils.getRootDir(TEST_UTIL.getConfiguration()),
-        DefaultWALProvider.getWALDirectoryName(serverName.toString()));
-
-      if (isHostingMeta(fs, wals, serverName)) {
-        metaServerName = serverName; // only 1 meta region for now
-      } else {
-        otherServerName = serverName;
-      }
-    }
-
-    LOG.info(metaServerName);
-    LOG.info(otherServerName);
-    assertNotNull(metaServerName);
-    assertNotNull(otherServerName);
-
-    moveMetaRegionAndWait(otherServerName);
-
-    LOG.info("Checking that old meta server does not have WALs for meta");
-    // the server that used to host meta now should not have any WAL files for the meta region now
-    Path wals = new Path(FSUtils.getRootDir(TEST_UTIL.getConfiguration()),
-      DefaultWALProvider.getWALDirectoryName(metaServerName.toString()));
-    for (FileStatus status : fs.listStatus(wals)) {
-      LOG.info(status.getPath());
-      assertFalse(DefaultWALProvider.isMetaFile(status.getPath()));
-    }
-
-    // assign the meta server back
-    moveMetaRegionAndWait(metaServerName);
-
-    // do some basic operations to ensure that nothing is failing
-    HTableDescriptor htd = TEST_UTIL.createTableDescriptor("foo");
-    TEST_UTIL.getHBaseAdmin().createTable(htd);
-    try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
-        Table table = conn.getTable(htd.getTableName())) {
-
-      TEST_UTIL.loadNumericRows(table, TEST_UTIL.fam1, 0, 100);
-      TEST_UTIL.verifyNumericRows(table, TEST_UTIL.fam1, 0, 100, 0);
-    }
-  }
-}