You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2018/12/14 00:14:29 UTC

[01/50] [abbrv] hbase git commit: HBASE 17959 Canary timeout should be configurable on a per-table basis

Repository: hbase
Updated Branches:
  refs/heads/branch-1.3 b9adb955c -> d37294174


HBASE 17959 Canary timeout should be configurable on a per-table basis

For branch-1: Added support for configuring read/write timeouts on a per-table basis
when in region mode.
Added unit test for per-table timeout checks.

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 178b675b792b4e9d3ddabd625a79042231a6fc91
Parents: 212e86d
Author: Chinmay Kulkarni <ch...@gmail.com>
Authored: Wed May 31 14:38:41 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:15 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/tool/Canary.java    | 167 ++++++++++++++++---
 .../hadoop/hbase/tool/TestCanaryTool.java       |  75 ++++++++-
 pom.xml                                         |   2 +-
 3 files changed, 211 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/178b675b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index 068e0ad..259690b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -244,6 +244,30 @@ public final class Canary implements Tool {
     }
   }
 
+  public static class RegionStdOutSink extends StdOutSink {
+
+    private Map<String, AtomicLong> perTableReadLatency = new HashMap<>();
+    private AtomicLong writeLatency = new AtomicLong();
+
+    public Map<String, AtomicLong> getReadLatencyMap() {
+      return this.perTableReadLatency;
+    }
+
+    public AtomicLong initializeAndGetReadLatencyForTable(String tableName) {
+      AtomicLong initLatency = new AtomicLong(0L);
+      this.perTableReadLatency.put(tableName, initLatency);
+      return initLatency;
+    }
+
+    public void initializeWriteLatency() {
+      this.writeLatency.set(0L);
+    }
+
+    public AtomicLong getWriteLatency() {
+      return this.writeLatency;
+    }
+  }
+
   static class ZookeeperTask implements Callable<Void> {
     private final Connection connection;
     private final String host;
@@ -291,19 +315,21 @@ public final class Canary implements Tool {
     }
     private Connection connection;
     private HRegionInfo region;
-    private Sink sink;
+    private RegionStdOutSink sink;
     private TaskType taskType;
     private boolean rawScanEnabled;
     private ServerName serverName;
+    private AtomicLong readWriteLatency;
 
-    RegionTask(Connection connection, HRegionInfo region, ServerName serverName, Sink sink,
-        TaskType taskType, boolean rawScanEnabled) {
+    RegionTask(Connection connection, HRegionInfo region, ServerName serverName, RegionStdOutSink sink,
+        TaskType taskType, boolean rawScanEnabled, AtomicLong rwLatency) {
       this.connection = connection;
       this.region = region;
       this.serverName = serverName;
       this.sink = sink;
       this.taskType = taskType;
       this.rawScanEnabled = rawScanEnabled;
+      this.readWriteLatency = rwLatency;
     }
 
     @Override
@@ -384,6 +410,7 @@ public final class Canary implements Tool {
             rs.next();
           }
           stopWatch.stop();
+          this.readWriteLatency.addAndGet(stopWatch.getTime());
           sink.publishReadTiming(serverName, region, column, stopWatch.getTime());
         } catch (Exception e) {
           sink.publishReadFailure(serverName, region, column, e);
@@ -394,7 +421,6 @@ public final class Canary implements Tool {
           }
           scan = null;
           get = null;
-          startKey = null;
         }
       }
       try {
@@ -436,6 +462,7 @@ public final class Canary implements Tool {
             long startTime = System.currentTimeMillis();
             table.put(put);
             long time = System.currentTimeMillis() - startTime;
+            this.readWriteLatency.addAndGet(time);
             sink.publishWriteTiming(serverName, region, column, time);
           } catch (Exception e) {
             sink.publishWriteFailure(serverName, region, column, e);
@@ -569,8 +596,10 @@ public final class Canary implements Tool {
   private boolean zookeeperMode = false;
   private boolean regionServerAllRegions = false;
   private boolean writeSniffing = false;
+  private long configuredWriteTableTimeout = DEFAULT_TIMEOUT;
   private boolean treatFailureAsError = false;
   private TableName writeTableName = DEFAULT_WRITE_TABLE_NAME;
+  private HashMap<String, Long> configuredReadTableTimeouts = new HashMap<>();
 
   private ExecutorService executor; // threads to retrieve data from regionservers
 
@@ -653,6 +682,20 @@ public final class Canary implements Tool {
             System.err.println("-t needs a numeric value argument.");
             printUsageAndExit();
           }
+        } else if(cmd.equals("-writeTableTimeout")) {
+          i++;
+
+          if (i == args.length) {
+            System.err.println("-writeTableTimeout needs a numeric value argument.");
+            printUsageAndExit();
+          }
+
+          try {
+            this.configuredWriteTableTimeout = Long.parseLong(args[i]);
+          } catch (NumberFormatException e) {
+            System.err.println("-writeTableTimeout needs a numeric value argument.");
+            printUsageAndExit();
+          }
         } else if (cmd.equals("-writeTable")) {
           i++;
 
@@ -671,6 +714,29 @@ public final class Canary implements Tool {
           }
 
           this.failOnError = Boolean.parseBoolean(args[i]);
+        } else if (cmd.equals("-readTableTimeouts")) {
+          i++;
+
+          if (i == args.length) {
+            System.err.println("-readTableTimeouts needs a comma-separated list of read timeouts per table (without spaces).");
+            printUsageAndExit();
+          }
+          String [] tableTimeouts = args[i].split(",");
+          for (String tT: tableTimeouts) {
+            String [] nameTimeout = tT.split("=");
+            if (nameTimeout.length < 2) {
+              System.err.println("Each -readTableTimeouts argument must be of the form <tableName>=<read timeout>.");
+              printUsageAndExit();
+            }
+            long timeoutVal = 0L;
+            try {
+              timeoutVal = Long.parseLong(nameTimeout[1]);
+            } catch (NumberFormatException e) {
+              System.err.println("-readTableTimeouts read timeout for each table must be a numeric value argument.");
+              printUsageAndExit();
+            }
+            this.configuredReadTableTimeouts.put(nameTimeout[0], timeoutVal);
+          }
         } else {
           // no options match
           System.err.println(cmd + " options is invalid.");
@@ -692,6 +758,10 @@ public final class Canary implements Tool {
         printUsageAndExit();
       }
     }
+    if (!this.configuredReadTableTimeouts.isEmpty() && (this.regionServerMode || this.zookeeperMode)) {
+      System.err.println("-readTableTimeouts can only be configured in region mode.");
+      printUsageAndExit();
+    }
     return index;
   }
 
@@ -792,7 +862,10 @@ public final class Canary implements Tool {
     System.err.println("      which means the table/regionserver is regular expression pattern");
     System.err.println("   -f <B>         stop whole program if first error occurs," +
         " default is true");
-    System.err.println("   -t <N>         timeout for a check, default is 600000 (milisecs)");
+    System.err.println("   -t <N>         timeout for a check, default is 600000 (millisecs)");
+    System.err.println("   -writeTableTimeout <N>         write timeout for the writeTable, default is 600000 (millisecs)");
+    System.err.println("   -readTableTimeouts <tableName>=<read timeout>,<tableName>=<read timeout>, ...    "
+      + "comma-separated list of read timeouts per table (no spaces), default is 600000 (millisecs)");
     System.err.println("   -writeSniffing enable the write sniffing in canary");
     System.err.println("   -treatFailureAsError treats read / write failure as error");
     System.err.println("   -writeTable    The table used for write sniffing."
@@ -832,8 +905,10 @@ public final class Canary implements Tool {
               (ZookeeperStdOutSink) this.sink, this.executor, this.treatFailureAsError);
     } else {
       monitor =
-          new RegionMonitor(connection, monitorTargets, this.useRegExp, this.sink, this.executor,
-              this.writeSniffing, this.writeTableName, this.treatFailureAsError);
+        new RegionMonitor(connection, monitorTargets, this.useRegExp,
+          (RegionStdOutSink) this.sink, this.executor, this.writeSniffing,
+          this.writeTableName, this.treatFailureAsError, this.configuredReadTableTimeouts,
+          this.configuredWriteTableTimeout);
     }
     return monitor;
   }
@@ -924,10 +999,12 @@ public final class Canary implements Tool {
     private float regionsUpperLimit;
     private int checkPeriod;
     private boolean rawScanEnabled;
+    private HashMap<String, Long> configuredReadTableTimeouts;
+    private long configuredWriteTableTimeout;
 
     public RegionMonitor(Connection connection, String[] monitorTargets, boolean useRegExp,
-        Sink sink, ExecutorService executor, boolean writeSniffing, TableName writeTableName,
-        boolean treatFailureAsError) {
+      RegionStdOutSink sink, ExecutorService executor, boolean writeSniffing, TableName writeTableName,
+      boolean treatFailureAsError, HashMap<String, Long> configuredReadTableTimeouts, long configuredWriteTableTimeout) {
       super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError);
       Configuration conf = connection.getConfiguration();
       this.writeSniffing = writeSniffing;
@@ -942,22 +1019,38 @@ public final class Canary implements Tool {
           conf.getInt(HConstants.HBASE_CANARY_WRITE_TABLE_CHECK_PERIOD_KEY,
             DEFAULT_WRITE_TABLE_CHECK_PERIOD);
       this.rawScanEnabled = conf.getBoolean(HConstants.HBASE_CANARY_READ_RAW_SCAN_KEY, false);
+      this.configuredReadTableTimeouts = new HashMap<>(configuredReadTableTimeouts);
+      this.configuredWriteTableTimeout = configuredWriteTableTimeout;
+    }
+
+    private RegionStdOutSink getSink() {
+      if (!(sink instanceof RegionStdOutSink)) {
+        throw new RuntimeException("Can only write to Region sink");
+      }
+      return ((RegionStdOutSink) sink);
     }
 
     @Override
     public void run() {
       if (this.initAdmin()) {
         try {
-          List<Future<Void>> taskFutures = new LinkedList<Future<Void>>();
+          List<Future<Void>> taskFutures = new LinkedList<>();
+          RegionStdOutSink regionSink = this.getSink();
           if (this.targets != null && this.targets.length > 0) {
             String[] tables = generateMonitorTables(this.targets);
+            // Check to see that each table name passed in the -readTableTimeouts argument is also passed as a monitor target.
+            if (! new HashSet<>(Arrays.asList(tables)).containsAll(this.configuredReadTableTimeouts.keySet())) {
+              LOG.error("-readTableTimeouts can only specify read timeouts for monitor targets passed via command line.");
+              this.errorCode = USAGE_EXIT_CODE;
+            }
             this.initialized = true;
             for (String table : tables) {
-              taskFutures.addAll(Canary.sniff(admin, sink, table, executor, TaskType.READ,
-                this.rawScanEnabled));
+              AtomicLong readLatency = regionSink.initializeAndGetReadLatencyForTable(table);
+              taskFutures.addAll(Canary.sniff(admin, regionSink, table, executor, TaskType.READ,
+                this.rawScanEnabled, readLatency));
             }
           } else {
-            taskFutures.addAll(sniff(TaskType.READ));
+            taskFutures.addAll(sniff(TaskType.READ, regionSink));
           }
 
           if (writeSniffing) {
@@ -970,8 +1063,10 @@ public final class Canary implements Tool {
               lastCheckTime = EnvironmentEdgeManager.currentTime();
             }
             // sniff canary table with write operation
-            taskFutures.addAll(Canary.sniff(admin, sink, admin.getTableDescriptor(writeTableName),
-              executor, TaskType.WRITE, this.rawScanEnabled));
+            regionSink.initializeWriteLatency();
+            AtomicLong writeTableLatency = regionSink.getWriteLatency();
+            taskFutures.addAll(Canary.sniff(admin, regionSink, admin.getTableDescriptor(writeTableName),
+              executor, TaskType.WRITE, this.rawScanEnabled, writeTableLatency));
           }
 
           for (Future<Void> future : taskFutures) {
@@ -981,6 +1076,30 @@ public final class Canary implements Tool {
               LOG.error("Sniff region failed!", e);
             }
           }
+          Map<String, AtomicLong> actualReadTableLatency = regionSink.getReadLatencyMap();
+          for (String tableName : this.configuredReadTableTimeouts.keySet()) {
+            if (actualReadTableLatency.containsKey(tableName)) {
+              Long actual = actualReadTableLatency.get(tableName).longValue();
+              Long configured = this.configuredReadTableTimeouts.get(tableName);
+              LOG.info("Read operation for " + tableName + " took " + actual +
+                " ms. The configured read timeout was " + configured + " ms.");
+              if (actual > configured) {
+                LOG.error("Read operation for " + tableName + " exceeded the configured read timeout.");
+              }
+            } else {
+              LOG.error("Read operation for " + tableName + " failed!");
+            }
+          }
+          if (this.writeSniffing) {
+            String writeTableStringName = this.writeTableName.getNameAsString();
+            long actualWriteLatency = regionSink.getWriteLatency().longValue();
+            LOG.info("Write operation for " + writeTableStringName + " took " + actualWriteLatency + " ms. The configured write timeout was " +
+              this.configuredWriteTableTimeout + " ms.");
+            // Check that the writeTable write operation latency does not exceed the configured timeout.
+            if (actualWriteLatency > this.configuredWriteTableTimeout) {
+              LOG.error("Write operation for " + writeTableStringName + " exceeded the configured write timeout.");
+            }
+          }
         } catch (Exception e) {
           LOG.error("Run regionMonitor failed", e);
           this.errorCode = ERROR_EXIT_CODE;
@@ -1035,15 +1154,16 @@ public final class Canary implements Tool {
     /*
      * canary entry point to monitor all the tables.
      */
-    private List<Future<Void>> sniff(TaskType taskType) throws Exception {
+    private List<Future<Void>> sniff(TaskType taskType, RegionStdOutSink regionSink) throws Exception {
       if (LOG.isDebugEnabled()) {
         LOG.debug(String.format("reading list of tables"));
       }
-      List<Future<Void>> taskFutures = new LinkedList<Future<Void>>();
+      List<Future<Void>> taskFutures = new LinkedList<>();
       for (HTableDescriptor table : admin.listTables()) {
         if (admin.isTableEnabled(table.getTableName())
             && (!table.getTableName().equals(writeTableName))) {
-          taskFutures.addAll(Canary.sniff(admin, sink, table, executor, taskType, this.rawScanEnabled));
+          AtomicLong readLatency = regionSink.initializeAndGetReadLatencyForTable(table.getNameAsString());
+          taskFutures.addAll(Canary.sniff(admin, sink, table, executor, taskType, this.rawScanEnabled, readLatency));
         }
       }
       return taskFutures;
@@ -1115,14 +1235,14 @@ public final class Canary implements Tool {
    * @throws Exception
    */
   private static List<Future<Void>> sniff(final Admin admin, final Sink sink, String tableName,
-      ExecutorService executor, TaskType taskType, boolean rawScanEnabled) throws Exception {
+    ExecutorService executor, TaskType taskType, boolean rawScanEnabled, AtomicLong readLatency) throws Exception {
     if (LOG.isDebugEnabled()) {
       LOG.debug(String.format("checking table is enabled and getting table descriptor for table %s",
         tableName));
     }
     if (admin.isTableEnabled(TableName.valueOf(tableName))) {
       return Canary.sniff(admin, sink, admin.getTableDescriptor(TableName.valueOf(tableName)),
-        executor, taskType, rawScanEnabled);
+        executor, taskType, rawScanEnabled, readLatency);
     } else {
       LOG.warn(String.format("Table %s is not enabled", tableName));
     }
@@ -1130,11 +1250,11 @@ public final class Canary implements Tool {
   }
 
   /*
-   * Loops over regions that owns this table, and output some information abouts the state.
+   * Loops over regions that owns this table, and output some information about the state.
    */
   private static List<Future<Void>> sniff(final Admin admin, final Sink sink,
       HTableDescriptor tableDesc, ExecutorService executor, TaskType taskType,
-      boolean rawScanEnabled) throws Exception {
+    boolean rawScanEnabled, AtomicLong rwLatency) throws Exception {
 
     if (LOG.isDebugEnabled()) {
       LOG.debug(String.format("reading list of regions for table %s", tableDesc.getTableName()));
@@ -1158,7 +1278,8 @@ public final class Canary implements Tool {
       for (HRegionLocation location : regionLocator.getAllRegionLocations()) {
         ServerName rs = location.getServerName();
         HRegionInfo region = location.getRegionInfo();
-        tasks.add(new RegionTask(admin.getConnection(), region, rs, sink, taskType, rawScanEnabled));
+        tasks.add(new RegionTask(admin.getConnection(), region, rs, (RegionStdOutSink) sink, taskType, rawScanEnabled,
+          rwLatency));
       }
     } finally {
       if (regionLocator != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/178b675b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
index f3d33e5..b89561f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
@@ -1,5 +1,5 @@
 /**
-q *
+ *
  * 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
@@ -49,15 +49,13 @@ import org.mockito.runners.MockitoJUnitRunner;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 
+import static org.junit.Assert.assertNotEquals;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
 import static org.mockito.Matchers.argThat;
-import static org.mockito.Mockito.never;
 import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.*;
 
 @RunWith(MockitoJUnitRunner.class)
 @Category({MediumTests.class})
@@ -113,7 +111,7 @@ public class TestCanaryTool {
       table.put(p);
     }
     ExecutorService executor = new ScheduledThreadPoolExecutor(1);
-    Canary.RegionServerStdOutSink sink = spy(new Canary.RegionServerStdOutSink());
+    Canary.RegionStdOutSink sink = spy(new Canary.RegionStdOutSink());
     Canary canary = new Canary(executor, sink);
     String[] args = { "-writeSniffing", "-t", "10000", "testTable" };
     ToolRunner.run(testingUtility.getConfiguration(), canary, args);
@@ -122,6 +120,66 @@ public class TestCanaryTool {
     verify(sink, atLeastOnce()).publishReadTiming(isA(ServerName.class), isA(HRegionInfo.class), isA(HColumnDescriptor.class), anyLong());
   }
 
+  @Test
+  public void testReadTableTimeouts() throws Exception {
+    final TableName [] tableNames = new TableName[2];
+    tableNames[0] = TableName.valueOf("testReadTableTimeouts1");
+    tableNames[1] = TableName.valueOf("testReadTableTimeouts2");
+    // Create 2 test tables.
+    for (int j = 0; j<2; j++) {
+      Table table = testingUtility.createTable(tableNames[j], new byte[][] { FAMILY });
+      // insert some test rows
+      for (int i=0; i<1000; i++) {
+        byte[] iBytes = Bytes.toBytes(i + j);
+        Put p = new Put(iBytes);
+        p.addColumn(FAMILY, COLUMN, iBytes);
+        table.put(p);
+      }
+    }
+    ExecutorService executor = new ScheduledThreadPoolExecutor(1);
+    Canary.RegionStdOutSink sink = spy(new Canary.RegionStdOutSink());
+    Canary canary = new Canary(executor, sink);
+    String configuredTimeoutStr = tableNames[0].getNameAsString() + "=" + Long.MAX_VALUE + "," +
+      tableNames[1].getNameAsString() + "=0";
+    String[] args = { "-readTableTimeouts", configuredTimeoutStr, tableNames[0].getNameAsString(), tableNames[1].getNameAsString()};
+    ToolRunner.run(testingUtility.getConfiguration(), canary, args);
+    verify(sink, times(tableNames.length)).initializeAndGetReadLatencyForTable(isA(String.class));
+    for (int i=0; i<2; i++) {
+      assertNotEquals("verify non-null read latency", null, sink.getReadLatencyMap().get(tableNames[i].getNameAsString()));
+      assertNotEquals("verify non-zero read latency", 0L, sink.getReadLatencyMap().get(tableNames[i].getNameAsString()));
+    }
+    // One table's timeout is set for 0 ms and thus, should lead to an error.
+    verify(mockAppender, times(1)).doAppend(argThat(new ArgumentMatcher<LoggingEvent>() {
+      @Override
+      public boolean matches(Object argument) {
+        return ((LoggingEvent) argument).getRenderedMessage().contains("exceeded the configured read timeout.");
+      }
+    }));
+    verify(mockAppender, times(2)).doAppend(argThat(new ArgumentMatcher<LoggingEvent>() {
+      @Override
+      public boolean matches(Object argument) {
+        return ((LoggingEvent) argument).getRenderedMessage().contains("The configured read timeout was");
+      }
+    }));
+  }
+
+  @Test
+  public void testWriteTableTimeout() throws Exception {
+    ExecutorService executor = new ScheduledThreadPoolExecutor(1);
+    Canary.RegionStdOutSink sink = spy(new Canary.RegionStdOutSink());
+    Canary canary = new Canary(executor, sink);
+    String[] args = { "-writeSniffing", "-writeTableTimeout", String.valueOf(Long.MAX_VALUE)};
+    ToolRunner.run(testingUtility.getConfiguration(), canary, args);
+    assertNotEquals("verify non-null write latency", null, sink.getWriteLatency());
+    assertNotEquals("verify non-zero write latency", 0L, sink.getWriteLatency());
+    verify(mockAppender, times(1)).doAppend(argThat(new ArgumentMatcher<LoggingEvent>() {
+      @Override
+      public boolean matches(Object argument) {
+        return ((LoggingEvent) argument).getRenderedMessage().contains("The configured write timeout was");
+      }
+    }));
+  }
+
   //no table created, so there should be no regions
   @Test
   public void testRegionserverNoRegions() throws Exception {
@@ -160,7 +218,7 @@ public class TestCanaryTool {
       table.put(p);
     }
     ExecutorService executor = new ScheduledThreadPoolExecutor(1);
-    Canary.RegionServerStdOutSink sink = spy(new Canary.RegionServerStdOutSink());
+    Canary.RegionStdOutSink sink = spy(new Canary.RegionStdOutSink());
     Canary canary = new Canary(executor, sink);
     String[] args = { "-t", "10000", "testTableRawScan" };
     org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration(testingUtility.getConfiguration());
@@ -179,5 +237,4 @@ public class TestCanaryTool {
     assertEquals("verify no read error count", 0, canary.getReadFailures().size());
   }
 
-}
-
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/178b675b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 30841b0..893b355 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1269,7 +1269,7 @@
     <hamcrest.version>1.3</hamcrest.version>
     <htrace.version>3.1.0-incubating</htrace.version>
     <log4j.version>1.2.17</log4j.version>
-    <mockito-all.version>1.10.8</mockito-all.version>
+    <mockito-all.version>1.10.19</mockito-all.version>
     <protobuf.version>2.5.0</protobuf.version>
     <thrift.path>thrift</thrift.path>
     <thrift.version>0.9.3</thrift.version>


[28/50] [abbrv] hbase git commit: HBASE-20554 "WALs outstanding" message from CleanerChore is noisy

Posted by ap...@apache.org.
HBASE-20554 "WALs outstanding" message from CleanerChore is noisy


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

Branch: refs/heads/branch-1.3
Commit: 511a4c49ceb223b289462121c788994fbfa2713b
Parents: 017cb75
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed May 9 10:16:44 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:19 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/cleaner/CleanerChore.java       | 12 +++++++++---
 1 file changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/511a4c49/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
index 5a4c407..28023c6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
@@ -273,9 +273,13 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
       try {
         POOL.latchCountUp();
         if (runCleaner()) {
-          LOG.debug("Cleaned all WALs under " + oldFileDir);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Cleaned all WALs under " + oldFileDir);
+          }
         } else {
-          LOG.warn("WALs outstanding under " + oldFileDir);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("WALs outstanding under " + oldFileDir);
+          }
         }
       } finally {
         POOL.latchCountDown();
@@ -288,7 +292,9 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
         POOL.updatePool((long) (0.8 * getTimeUnit().toMillis(getPeriod())));
       }
     } else {
-      LOG.debug("Cleaner chore disabled! Not cleaning.");
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Cleaner chore disabled! Not cleaning.");
+      }
     }
   }
 


[08/50] [abbrv] hbase git commit: Amend HBASE-18830 TestCanaryTool does not check Canary monitor's error code

Posted by ap...@apache.org.
Amend HBASE-18830 TestCanaryTool does not check Canary monitor's error code

Adjust exception control flow to fix findbugs warning
NP_NULL_ON_SOME_PATH_EXCEPTION, Possible null pointer dereference of
regionSink in org.apache.hadoop.hbase.tool.Canary$RegionMonitor.run()
on exception path


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

Branch: refs/heads/branch-1.3
Commit: 5d0d3aa2b962a27a7eb33b20ef91476893380b89
Parents: e72ed4f
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Sep 26 08:33:19 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:16 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/tool/Canary.java | 13 +++++--------
 1 file changed, 5 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5d0d3aa2/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index c7323fe..dcaa057 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -1022,19 +1022,14 @@ public final class Canary implements Tool {
       if (this.initAdmin()) {
         try {
           List<Future<Void>> taskFutures = new LinkedList<>();
-          RegionStdOutSink regionSink = null;
-          try {
-            regionSink = this.getSink();
-          } catch (RuntimeException e) {
-            LOG.error("Run RegionMonitor failed!", e);
-            this.errorCode = ERROR_EXIT_CODE;
-          }
+          RegionStdOutSink regionSink = this.getSink();
           if (this.targets != null && this.targets.length > 0) {
             String[] tables = generateMonitorTables(this.targets);
             // Check to see that each table name passed in the -readTableTimeouts argument is also passed as a monitor target.
             if (! new HashSet<>(Arrays.asList(tables)).containsAll(this.configuredReadTableTimeouts.keySet())) {
               LOG.error("-readTableTimeouts can only specify read timeouts for monitor targets passed via command line.");
               this.errorCode = USAGE_EXIT_CODE;
+              return;
             }
             this.initialized = true;
             for (String table : tables) {
@@ -1096,7 +1091,9 @@ public final class Canary implements Tool {
         } catch (Exception e) {
           LOG.error("Run regionMonitor failed", e);
           this.errorCode = ERROR_EXIT_CODE;
-        }
+        } finally {
+          this.done = true;
+	}
       }
       this.done = true;
     }


[37/50] [abbrv] hbase git commit: HBASE-20858 Port HBASE-20695 (Implement table level RegionServer replication metrics) to branch-1

Posted by ap...@apache.org.
HBASE-20858 Port HBASE-20695 (Implement table level RegionServer replication metrics) to branch-1

Signed-off-by: Andrew Purtell <ap...@apache.org>

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java


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

Branch: refs/heads/branch-1.3
Commit: 9e39a2009648ba21d55676ea90b328941bb9b7db
Parents: 614b5f6
Author: Xu Cang <xc...@salesforce.com>
Authored: Fri Jul 6 16:36:05 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:20 2018 -0800

----------------------------------------------------------------------
 .../replication/regionserver/MetricsSource.java | 43 +++++++++++++---
 .../regionserver/ReplicationSource.java         |  5 ++
 .../replication/TestReplicationEndpoint.java    | 52 ++++++++++++++++++++
 3 files changed, 94 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9e39a200/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
index 9b99f2a..56baa05 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -35,8 +33,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
 public class MetricsSource {
 
-  private static final Log LOG = LogFactory.getLog(MetricsSource.class);
-
   // tracks last shipped timestamp for each wal group
   private Map<String, Long> lastTimeStamps = new HashMap<String, Long>();
   private long lastHFileRefsQueueSize = 0;
@@ -44,7 +40,7 @@ public class MetricsSource {
 
   private final MetricsReplicationSourceSource singleSourceSource;
   private final MetricsReplicationSourceSource globalSourceSource;
-
+  private Map<String, MetricsReplicationSourceSource> singleSourceSourceByTable;
 
   /**
    * Constructor used to register the metrics
@@ -56,7 +52,24 @@ public class MetricsSource {
     singleSourceSource =
         CompatibilitySingletonFactory.getInstance(MetricsReplicationSourceFactory.class)
             .getSource(id);
-    globalSourceSource = CompatibilitySingletonFactory.getInstance(MetricsReplicationSourceFactory.class).getGlobalSource();
+    globalSourceSource = CompatibilitySingletonFactory
+        .getInstance(MetricsReplicationSourceFactory.class).getGlobalSource();
+    singleSourceSourceByTable = new HashMap<>();
+  }
+
+  /**
+   * Constructor for injecting custom (or test) MetricsReplicationSourceSources
+   * @param id Name of the source this class is monitoring
+   * @param singleSourceSource Class to monitor id-scoped metrics
+   * @param globalSourceSource Class to monitor global-scoped metrics
+   */
+  public MetricsSource(String id, MetricsReplicationSourceSource singleSourceSource,
+                       MetricsReplicationSourceSource globalSourceSource,
+                       Map<String, MetricsReplicationSourceSource> singleSourceSourceByTable) {
+    this.id = id;
+    this.singleSourceSource = singleSourceSource;
+    this.globalSourceSource = globalSourceSource;
+    this.singleSourceSourceByTable = singleSourceSourceByTable;
   }
 
   /**
@@ -72,6 +85,20 @@ public class MetricsSource {
   }
 
   /**
+   * Set the age of the last edit that was shipped group by table
+   * @param timestamp write time of the edit
+   * @param tableName String as group and tableName
+   */
+  public void setAgeOfLastShippedOpByTable(long timestamp, String tableName) {
+    long age = EnvironmentEdgeManager.currentTime() - timestamp;
+    if (!this.getSingleSourceSourceByTable().containsKey(tableName)) {
+      this.getSingleSourceSourceByTable().put(tableName,
+          CompatibilitySingletonFactory.getInstance(MetricsReplicationSourceFactory.class)
+          .getSource(tableName));
+    }
+    this.singleSourceSourceByTable.get(tableName).setLastShippedAge(age);
+  }
+  /**
    * Convenience method to use the last given timestamp to refresh the age of the last edit. Used
    * when replication fails and need to keep that metric accurate.
    * @param walGroupId id of the group to update
@@ -262,4 +289,8 @@ public class MetricsSource {
     singleSourceSource.incrCompletedRecoveryQueue();
     globalSourceSource.incrCompletedRecoveryQueue();
   }
+
+  public Map<String, MetricsReplicationSourceSource> getSingleSourceSourceByTable() {
+    return singleSourceSourceByTable;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9e39a200/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
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 78b465c..8112553 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
@@ -1066,6 +1066,11 @@ public class ReplicationSource extends Thread
             int size = entries.size();
             for (int i = 0; i < size; i++) {
               cleanUpHFileRefs(entries.get(i).getEdit());
+
+              TableName tableName = entries.get(i).getKey().getTablename();
+              source.getSourceMetrics().setAgeOfLastShippedOpByTable(
+                entries.get(i).getKey().getWriteTime(),
+                tableName.getNameAsString());
             }
             //Log and clean up WAL logs
             manager.logPositionAndCleanOldLogs(this.currentPath, peerClusterZnode,

http://git-wip-us.apache.org/repos/asf/hbase/blob/9e39a200/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
index d570549..c3822c1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
@@ -18,9 +18,15 @@
 
 package org.apache.hadoop.hbase.replication;
 
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.spy;
+
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -39,10 +45,16 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationGlobalSourceSource;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceImpl;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceSource;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceSourceImpl;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
+import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
@@ -253,6 +265,46 @@ public class TestReplicationEndpoint extends TestReplicationBase {
     admin.removePeer("testWALEntryFilterFromReplicationEndpoint");
   }
 
+  @Test
+  public void testMetricsSourceBaseSourcePassthrough(){
+    /*
+    The replication MetricsSource wraps a MetricsReplicationSourceSourceImpl
+    and a MetricsReplicationGlobalSourceSource, so that metrics get written to both namespaces.
+    Both of those classes wrap a MetricsReplicationSourceImpl that implements BaseSource, which
+    allows for custom JMX metrics.
+    This test checks to make sure the BaseSource decorator logic on MetricsSource actually calls down through
+    the two layers of wrapping to the actual BaseSource.
+    */
+    String id = "id";
+    DynamicMetricsRegistry mockRegistry = new DynamicMetricsRegistry(id);
+    MetricsReplicationSourceImpl singleRms = mock(MetricsReplicationSourceImpl.class);
+    when(singleRms.getMetricsRegistry()).thenReturn(mockRegistry);
+    MetricsReplicationSourceImpl globalRms = mock(MetricsReplicationSourceImpl.class);
+    when(globalRms.getMetricsRegistry()).thenReturn(mockRegistry);
+
+    MetricsReplicationSourceSource singleSourceSource = new MetricsReplicationSourceSourceImpl(singleRms, id);
+    MetricsReplicationSourceSource globalSourceSource = new MetricsReplicationGlobalSourceSource(globalRms);
+    MetricsReplicationSourceSource spyglobalSourceSource = spy(globalSourceSource);
+    Map<String, MetricsReplicationSourceSource> singleSourceSourceByTable = new HashMap<>();
+    MetricsSource source = new MetricsSource(id, singleSourceSource, spyglobalSourceSource,
+        singleSourceSourceByTable);
+
+    // check singleSourceSourceByTable metrics.
+    // singleSourceSourceByTable map entry will be created only
+    // after calling #setAgeOfLastShippedOpByTable
+    boolean containsRandomNewTable = source.getSingleSourceSourceByTable()
+        .containsKey("RandomNewTable");
+    Assert.assertEquals(false, containsRandomNewTable);
+    source.setAgeOfLastShippedOpByTable(123L, "RandomNewTable");
+    containsRandomNewTable = source.getSingleSourceSourceByTable()
+      .containsKey("RandomNewTable");
+    Assert.assertEquals(true, containsRandomNewTable);
+    MetricsReplicationSourceSource msr = source.getSingleSourceSourceByTable()
+        .get("RandomNewTable");
+    // cannot put more concreate value here to verify because the age is arbitrary.
+    // as long as it's greater than 0, we see it as correct answer.
+    Assert.assertTrue(msr.getLastShippedAge() > 0);
+  }
 
   private void doPut(byte[] row) throws IOException {
     try (Connection connection = ConnectionFactory.createConnection(conf1)) {


[12/50] [abbrv] hbase git commit: HBASE-19358 Improve the stability of splitting log when do fail over

Posted by ap...@apache.org.
HBASE-19358 Improve the stability of splitting log when do fail over

Signed-off-by: Yu Li <li...@apache.org>

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java


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

Branch: refs/heads/branch-1.3
Commit: e4f46f53b737d0a5ea0c4c079b6182a5d918e91d
Parents: cd1726f
Author: Jingyun Tian <ti...@gmail.com>
Authored: Tue Jan 2 17:21:32 2018 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:17 2018 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/wal/WALSplitter.java    | 358 +++++++++++++------
 .../TestWALReplayBoundedLogWriterCreation.java  |  33 ++
 .../TestWALSplitBoundedLogWriterCreation.java   |  44 +++
 3 files changed, 330 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e4f46f53/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index cc065e5..1927eb3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -25,6 +25,7 @@ import java.io.InterruptedIOException;
 import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
@@ -161,6 +162,9 @@ public class WALSplitter {
 
   protected boolean distributedLogReplay;
 
+  private final boolean splitWriterCreationBounded;
+
+
   // Map encodedRegionName -> lastFlushedSequenceId
   protected Map<String, Long> lastFlushedSequenceIds = new ConcurrentHashMap<String, Long>();
 
@@ -180,6 +184,8 @@ public class WALSplitter {
   // the file being split currently
   private FileStatus fileBeingSplit;
 
+  public final static String SPLIT_WRITER_CREATION_BOUNDED = "hbase.split.writer.creation.bounded";
+
   @VisibleForTesting
   WALSplitter(final WALFactory factory, Configuration conf, Path rootDir,
       FileSystem fs, LastSequenceId idChecker,
@@ -194,10 +200,10 @@ public class WALSplitter {
     this.csm = (BaseCoordinatedStateManager)csm;
     this.walFactory = factory;
     this.controller = new PipelineController();
-
+    this.splitWriterCreationBounded = conf.getBoolean(SPLIT_WRITER_CREATION_BOUNDED, false);
     entryBuffers = new EntryBuffers(controller,
         this.conf.getInt("hbase.regionserver.hlog.splitlog.buffersize",
-            128*1024*1024));
+            128*1024*1024), splitWriterCreationBounded);
 
     // a larger minBatchSize may slow down recovery because replay writer has to wait for
     // enough edits before replaying them
@@ -212,7 +218,12 @@ public class WALSplitter {
         LOG.info("ZooKeeperWatcher is passed in as NULL so disable distrubitedLogRepaly.");
       }
       this.distributedLogReplay = false;
-      outputSink = new LogRecoveredEditsOutputSink(controller, entryBuffers, numWriterThreads);
+      if(splitWriterCreationBounded){
+        outputSink = new BoundedLogWriterCreationOutputSink(controller,
+            entryBuffers, numWriterThreads);
+      }else {
+        outputSink = new LogRecoveredEditsOutputSink(controller, entryBuffers, numWriterThreads);
+      }
     }
 
   }
@@ -923,11 +934,19 @@ public class WALSplitter {
     Set<byte[]> currentlyWriting = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
 
     long totalBuffered = 0;
-    long maxHeapUsage;
+    final long maxHeapUsage;
+    boolean splitWriterCreationBounded;
+
 
     public EntryBuffers(PipelineController controller, long maxHeapUsage) {
+      this(controller, maxHeapUsage, false);
+    }
+
+    public EntryBuffers(PipelineController controller, long maxHeapUsage,
+        boolean splitWriterCreationBounded) {
       this.controller = controller;
       this.maxHeapUsage = maxHeapUsage;
+      this.splitWriterCreationBounded = splitWriterCreationBounded;
     }
 
     /**
@@ -967,6 +986,14 @@ public class WALSplitter {
      * @return RegionEntryBuffer a buffer of edits to be written or replayed.
      */
     synchronized RegionEntryBuffer getChunkToWrite() {
+      // The core part of limiting opening writers is it doesn't return chunk only if the heap size
+      // is over maxHeapUsage. Thus it doesn't need to create a writer for each region
+      // during splitting. It will flush all the logs in the buffer after splitting through a
+      // threadpool, which means the number of writers it created is under control
+      if(splitWriterCreationBounded && totalBuffered < maxHeapUsage){
+        return null;
+      }
+
       long biggestSize = 0;
       byte[] biggestBufferKey = null;
 
@@ -1145,11 +1172,9 @@ public class WALSplitter {
     protected PipelineController controller;
     protected EntryBuffers entryBuffers;
 
-    protected Map<byte[], SinkWriter> writers = Collections
-        .synchronizedMap(new TreeMap<byte[], SinkWriter>(Bytes.BYTES_COMPARATOR));;
-
-    protected final Map<byte[], Long> regionMaximumEditLogSeqNum = Collections
-        .synchronizedMap(new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR));
+    protected ConcurrentHashMap<String, SinkWriter> writers = new ConcurrentHashMap<>();
+    protected ConcurrentHashMap<String, Long> regionMaximumEditLogSeqNum =
+        new ConcurrentHashMap<>();
 
     protected final List<WriterThread> writerThreads = Lists.newArrayList();
 
@@ -1195,18 +1220,19 @@ public class WALSplitter {
      * Update region's maximum edit log SeqNum.
      */
     void updateRegionMaximumEditLogSeqNum(Entry entry) {
+
       synchronized (regionMaximumEditLogSeqNum) {
-        Long currentMaxSeqNum = regionMaximumEditLogSeqNum.get(entry.getKey()
-            .getEncodedRegionName());
+        String encodedRegionName = Bytes.toString(entry.getKey().getEncodedRegionName());
+        Long currentMaxSeqNum = regionMaximumEditLogSeqNum.get(encodedRegionName);
         if (currentMaxSeqNum == null || entry.getKey().getLogSeqNum() > currentMaxSeqNum) {
-          regionMaximumEditLogSeqNum.put(entry.getKey().getEncodedRegionName(), entry.getKey()
+          regionMaximumEditLogSeqNum.put(encodedRegionName, entry.getKey()
               .getLogSeqNum());
         }
       }
     }
 
     Long getRegionMaximumEditLogSeqNum(byte[] region) {
-      return regionMaximumEditLogSeqNum.get(region);
+      return regionMaximumEditLogSeqNum.get(Bytes.toString(region));
     }
 
     /**
@@ -1328,7 +1354,7 @@ public class WALSplitter {
     }
 
     // delete the one with fewer wal entries
-    private void deleteOneWithFewerEntries(WriterAndPath wap, Path dst) throws IOException {
+    void deleteOneWithFewerEntries(WriterAndPath wap, Path dst) throws IOException {
       long dstMinLogSeqNum = -1L;
       try (WAL.Reader reader = walFactory.createReader(fs, dst)) {
         WAL.Entry entry = reader.next();
@@ -1364,7 +1390,7 @@ public class WALSplitter {
      * Close all of the output streams.
      * @return the list of paths written.
      */
-    private List<Path> close() throws IOException {
+    List<Path> close() throws IOException {
       Preconditions.checkState(!closeAndCleanCompleted);
 
       final List<Path> paths = new ArrayList<Path>();
@@ -1381,71 +1407,9 @@ public class WALSplitter {
         });
       CompletionService<Void> completionService =
         new ExecutorCompletionService<Void>(closeThreadPool);
-      for (final Map.Entry<byte[], SinkWriter> writersEntry : writers.entrySet()) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Submitting close of " + ((WriterAndPath)writersEntry.getValue()).p);
-        }
-        completionService.submit(new Callable<Void>() {
-          @Override
-          public Void call() throws Exception {
-            WriterAndPath wap = (WriterAndPath) writersEntry.getValue();
-            if (LOG.isTraceEnabled()) LOG.trace("Closing " + wap.p);
-            try {
-              wap.w.close();
-            } catch (IOException ioe) {
-              LOG.error("Couldn't close log at " + wap.p, ioe);
-              thrown.add(ioe);
-              return null;
-            }
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Closed wap " + wap.p + " (wrote " + wap.editsWritten
-                + " edits, skipped " + wap.editsSkipped + " edits in "
-                + (wap.nanosSpent / 1000 / 1000) + "ms");
-            }
-            if (wap.editsWritten == 0) {
-              // just remove the empty recovered.edits file
-              if (fs.exists(wap.p) && !fs.delete(wap.p, false)) {
-                LOG.warn("Failed deleting empty " + wap.p);
-                throw new IOException("Failed deleting empty  " + wap.p);
-              }
-              return null;
-            }
-
-            Path dst = getCompletedRecoveredEditsFilePath(wap.p,
-              regionMaximumEditLogSeqNum.get(writersEntry.getKey()));
-            try {
-              if (!dst.equals(wap.p) && fs.exists(dst)) {
-                deleteOneWithFewerEntries(wap, dst);
-              }
-              // Skip the unit tests which create a splitter that reads and
-              // writes the data without touching disk.
-              // TestHLogSplit#testThreading is an example.
-              if (fs.exists(wap.p)) {
-                if (!fs.rename(wap.p, dst)) {
-                  throw new IOException("Failed renaming " + wap.p + " to " + dst);
-                }
-                LOG.info("Rename " + wap.p + " to " + dst);
-              }
-            } catch (IOException ioe) {
-              LOG.error("Couldn't rename " + wap.p + " to " + dst, ioe);
-              thrown.add(ioe);
-              return null;
-            }
-            paths.add(dst);
-            return null;
-          }
-        });
-      }
-
-      boolean progress_failed = false;
-      try {
-        for (int i = 0, n = this.writers.size(); i < n; i++) {
-          Future<Void> future = completionService.take();
-          future.get();
-          if (!progress_failed && reporter != null && !reporter.progress()) {
-            progress_failed = true;
-          }
-        }
+      boolean progress_failed;
+      try{
+        progress_failed = executeCloseTask(completionService, thrown, paths);
       } catch (InterruptedException e) {
         IOException iie = new InterruptedIOException();
         iie.initCause(e);
@@ -1467,6 +1431,83 @@ public class WALSplitter {
       return paths;
     }
 
+    boolean executeCloseTask(CompletionService<Void> completionService,
+        final List<IOException> thrown, final List<Path> paths)
+        throws InterruptedException, ExecutionException {
+      for (final Map.Entry<String, SinkWriter> writersEntry : writers.entrySet()) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Submitting close of " + ((WriterAndPath) writersEntry.getValue()).p);
+        }
+        completionService.submit(new Callable<Void>() {
+          @Override public Void call() throws Exception {
+            WriterAndPath wap = (WriterAndPath) writersEntry.getValue();
+            Path dst = closeWriter(writersEntry.getKey(), wap, thrown);
+            paths.add(dst);
+            return null;
+          }
+        });
+      }
+      boolean progress_failed = false;
+      for (int i = 0, n = this.writers.size(); i < n; i++) {
+        Future<Void> future = completionService.take();
+        future.get();
+        if (!progress_failed && reporter != null && !reporter.progress()) {
+          progress_failed = true;
+        }
+      }
+      return progress_failed;
+    }
+
+    Path closeWriter(String encodedRegionName, WriterAndPath wap, List<IOException> thrown)
+        throws IOException {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Closing " + wap.p);
+      }
+      try {
+        wap.w.close();
+      } catch (IOException ioe) {
+        LOG.error("Couldn't close log at " + wap.p, ioe);
+        thrown.add(ioe);
+        return null;
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Closed wap " + wap.p + " (wrote " + wap.editsWritten
+            + " edits, skipped " + wap.editsSkipped + " edits in "
+            + (wap.nanosSpent / 1000 / 1000) + "ms");
+      }
+      if (wap.editsWritten == 0) {
+        // just remove the empty recovered.edits file
+        if (fs.exists(wap.p) && !fs.delete(wap.p, false)) {
+          LOG.warn("Failed deleting empty " + wap.p);
+          throw new IOException("Failed deleting empty  " + wap.p);
+        }
+        return null;
+      }
+
+      Path dst = getCompletedRecoveredEditsFilePath(wap.p,
+          regionMaximumEditLogSeqNum.get(encodedRegionName));
+      try {
+        if (!dst.equals(wap.p) && fs.exists(dst)) {
+          deleteOneWithFewerEntries(wap, dst);
+        }
+        // Skip the unit tests which create a splitter that reads and
+        // writes the data without touching disk.
+        // TestHLogSplit#testThreading is an example.
+        if (fs.exists(wap.p)) {
+          if (!fs.rename(wap.p, dst)) {
+            throw new IOException("Failed renaming " + wap.p + " to " + dst);
+          }
+          LOG.info("Rename " + wap.p + " to " + dst);
+        }
+      } catch (IOException ioe) {
+        LOG.error("Couldn't rename " + wap.p + " to " + dst, ioe);
+        thrown.add(ioe);
+        return null;
+      }
+      return dst;
+    }
+
+
     private List<IOException> closeLogWriters(List<IOException> thrown) throws IOException {
       if (writersClosed) {
         return thrown;
@@ -1490,20 +1531,19 @@ public class WALSplitter {
           }
         }
       } finally {
-        synchronized (writers) {
-          WriterAndPath wap = null;
-          for (SinkWriter tmpWAP : writers.values()) {
-            try {
-              wap = (WriterAndPath) tmpWAP;
-              wap.w.close();
-            } catch (IOException ioe) {
-              LOG.error("Couldn't close log at " + wap.p, ioe);
-              thrown.add(ioe);
-              continue;
-            }
-            LOG.info("Closed log " + wap.p + " (wrote " + wap.editsWritten + " edits in "
-                + (wap.nanosSpent / 1000 / 1000) + "ms)");
+        WriterAndPath wap = null;
+        for (SinkWriter tmpWAP : writers.values()) {
+          try {
+            wap = (WriterAndPath) tmpWAP;
+            wap.w.close();
+          } catch (IOException ioe) {
+            LOG.error("Couldn't close log at " + wap.p, ioe);
+            thrown.add(ioe);
+            continue;
           }
+          LOG.info(
+              "Closed log " + wap.p + " (wrote " + wap.editsWritten + " edits in " + (wap.nanosSpent
+                  / 1000 / 1000) + "ms)");
         }
         writersClosed = true;
       }
@@ -1516,9 +1556,10 @@ public class WALSplitter {
      * long as multiple threads are always acting on different regions.
      * @return null if this region shouldn't output any logs
      */
-    private WriterAndPath getWriterAndPath(Entry entry) throws IOException {
+    WriterAndPath getWriterAndPath(Entry entry, boolean reusable) throws IOException {
       byte region[] = entry.getKey().getEncodedRegionName();
-      WriterAndPath ret = (WriterAndPath) writers.get(region);
+      String regionName = Bytes.toString(region);
+      WriterAndPath ret = (WriterAndPath) writers.get(regionName);
       if (ret != null) {
         return ret;
       }
@@ -1532,14 +1573,17 @@ public class WALSplitter {
         blacklistedRegions.add(region);
         return null;
       }
-      writers.put(region, ret);
+
+      if(reusable) {
+        writers.put(regionName, ret);
+      }
       return ret;
     }
 
     /**
      * @return a path with a write for that path. caller should close.
      */
-    private WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir) throws IOException {
+    WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir) throws IOException {
       Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir,
           fileBeingSplit.getPath().getName());
       if (regionedits == null) {
@@ -1558,7 +1602,7 @@ public class WALSplitter {
       return new WriterAndPath(regionedits, w, entry.getKey().getLogSeqNum());
     }
 
-    private void filterCellByStore(Entry logEntry) {
+    void filterCellByStore(Entry logEntry) {
       Map<byte[], Long> maxSeqIdInStores =
           regionMaxSeqIdInStores.get(Bytes.toString(logEntry.getKey().getEncodedRegionName()));
       if (maxSeqIdInStores == null || maxSeqIdInStores.isEmpty()) {
@@ -1589,12 +1633,16 @@ public class WALSplitter {
 
     @Override
     public void append(RegionEntryBuffer buffer) throws IOException {
+      appendBuffer(buffer, true);
+    }
+
+
+    WriterAndPath appendBuffer(RegionEntryBuffer buffer, boolean reusable) throws IOException{
       List<Entry> entries = buffer.entryBuffer;
       if (entries.isEmpty()) {
         LOG.warn("got an empty buffer, skipping");
-        return;
+        return null;
       }
-
       WriterAndPath wap = null;
 
       long startTime = System.nanoTime();
@@ -1603,12 +1651,12 @@ public class WALSplitter {
 
         for (Entry logEntry : entries) {
           if (wap == null) {
-            wap = getWriterAndPath(logEntry);
+            wap = getWriterAndPath(logEntry, reusable);
             if (wap == null) {
               if (LOG.isDebugEnabled()) {
                 LOG.debug("getWriterAndPath decided we don't need to write edits for " + logEntry);
               }
-              return;
+              return null;
             }
           }
           filterCellByStore(logEntry);
@@ -1628,6 +1676,7 @@ public class WALSplitter {
         LOG.fatal(" Got while writing log entry to log", e);
         throw e;
       }
+      return wap;
     }
 
     @Override
@@ -1648,8 +1697,8 @@ public class WALSplitter {
     public Map<byte[], Long> getOutputCounts() {
       TreeMap<byte[], Long> ret = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
       synchronized (writers) {
-        for (Map.Entry<byte[], SinkWriter> entry : writers.entrySet()) {
-          ret.put(entry.getKey(), entry.getValue().editsWritten);
+        for (Map.Entry<String, SinkWriter> entry : writers.entrySet()) {
+          ret.put(Bytes.toBytes(entry.getKey()), entry.getValue().editsWritten);
         }
       }
       return ret;
@@ -1662,6 +1711,105 @@ public class WALSplitter {
   }
 
   /**
+   * Class that will limit the number of hdfs writers we create to split the logs
+   */
+  class BoundedLogWriterCreationOutputSink extends LogRecoveredEditsOutputSink{
+
+    ConcurrentHashMap<String, Long> regionRecoverStatMap = new ConcurrentHashMap<>();
+
+
+    public BoundedLogWriterCreationOutputSink(PipelineController controller,
+        EntryBuffers entryBuffers, int numWriters){
+      super(controller, entryBuffers, numWriters);
+    }
+
+    @Override
+    public List<Path> finishWritingAndClose() throws IOException {
+      boolean isSuccessful;
+      List<Path> result;
+      try {
+        isSuccessful = finishWriting(false);
+      } finally {
+        result = close();
+      }
+      if (isSuccessful) {
+        splits = result;
+      }
+      return splits;
+    }
+
+    @Override
+    boolean executeCloseTask(CompletionService<Void> closeCompletionService,
+        final List<IOException> thrown, final List<Path> paths)
+        throws InterruptedException, ExecutionException {
+      for (final Map.Entry<byte[], RegionEntryBuffer> buffer : entryBuffers.buffers.entrySet()) {
+        LOG.info("Submitting write then close of " +
+            Bytes.toString(buffer.getValue().encodedRegionName));
+        closeCompletionService.submit(new Callable<Void>() {
+          public Void call() throws Exception {
+            Path dst = writeThenClose(buffer.getValue());
+            paths.add(dst);
+            return null;
+          }
+        });
+      }
+
+      boolean progress_failed = false;
+      for (int i = 0, n = entryBuffers.buffers.size(); i < n; i++) {
+        Future<Void> future = closeCompletionService.take();
+        future.get();
+        if (!progress_failed && reporter != null && !reporter.progress()) {
+          progress_failed = true;
+        }
+      }
+      return progress_failed;
+    }
+
+    @Override
+    public Map<byte[], Long> getOutputCounts() {
+      Map<byte[], Long> regionRecoverStatMapResult = new HashMap<>();
+      for(Map.Entry<String, Long> entry: regionRecoverStatMap.entrySet()){
+        regionRecoverStatMapResult.put(Bytes.toBytes(entry.getKey()), entry.getValue());
+      }
+      return regionRecoverStatMapResult;
+    }
+
+    @Override
+    public int getNumberOfRecoveredRegions() {
+      return regionRecoverStatMap.size();
+    }
+
+    @Override
+    public void append(RegionEntryBuffer buffer) throws IOException {
+      writeThenClose(buffer);
+    }
+
+    private Path writeThenClose(RegionEntryBuffer buffer) throws IOException {
+      WriterAndPath wap = appendBuffer(buffer, false);
+      Path dst = null;
+      if(wap != null){
+        String encodedRegionName = Bytes.toString(buffer.encodedRegionName);
+        Long value = regionRecoverStatMap.putIfAbsent(encodedRegionName, wap.editsWritten);
+        if(value != null){
+          Long newValue = regionRecoverStatMap.get(encodedRegionName) + wap.editsWritten;
+          regionRecoverStatMap.put(encodedRegionName, newValue);
+        }
+      }
+
+      List<IOException> thrown = new ArrayList<>();
+      if(wap != null) {
+        dst = closeWriter(Bytes.toString(buffer.encodedRegionName), wap, thrown);
+      }
+
+      if(!thrown.isEmpty()){
+        throw MultipleIOException.createIOException(thrown);
+      }
+      return dst;
+    }
+  }
+
+
+  /**
    * Class wraps the actual writer which writes data out and related statistics
    */
   public abstract static class SinkWriter {
@@ -1720,7 +1868,7 @@ public class WALSplitter {
         .synchronizedMap(new TreeMap<TableName, HConnection>());
     /**
      * Map key -> value layout
-     * <servername>:<table name> -> Queue<Row>
+     * servername:table name -> Queue(Row)
      */
     private Map<String, List<Pair<HRegionLocation, Entry>>> serverToBufferQueueMap =
         new ConcurrentHashMap<String, List<Pair<HRegionLocation, Entry>>>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4f46f53/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayBoundedLogWriterCreation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayBoundedLogWriterCreation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayBoundedLogWriterCreation.java
new file mode 100644
index 0000000..36bc63a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayBoundedLogWriterCreation.java
@@ -0,0 +1,33 @@
+/**
+ * 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.regionserver.wal;
+
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+@Category(MediumTests.class)
+public class TestWALReplayBoundedLogWriterCreation extends TestWALReplay {
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TestWALReplay.setUpBeforeClass();
+    TEST_UTIL.getConfiguration().setBoolean(WALSplitter.SPLIT_WRITER_CREATION_BOUNDED, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4f46f53/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitBoundedLogWriterCreation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitBoundedLogWriterCreation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitBoundedLogWriterCreation.java
new file mode 100644
index 0000000..285e8f3
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitBoundedLogWriterCreation.java
@@ -0,0 +1,44 @@
+/**
+ * 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 org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestWALSplitBoundedLogWriterCreation extends TestWALSplit{
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TestWALSplit.setUpBeforeClass();
+    TEST_UTIL.getConfiguration().setBoolean(WALSplitter.SPLIT_WRITER_CREATION_BOUNDED, true);
+  }
+
+  /**
+   * The logic of this test is conflict with the limit writers split logic, skip this test
+   */
+  @Test(timeout=300000)
+  @Ignore
+  public void testThreadingSlowWriterSmallBuffer() throws Exception {
+    super.testThreadingSlowWriterSmallBuffer();
+  }
+}
+


[11/50] [abbrv] hbase git commit: HBASE-19163 "Maximum lock count exceeded" from region server's batch processing

Posted by ap...@apache.org.
HBASE-19163 "Maximum lock count exceeded" from region server's batch processing

Signed-off-by: Umesh Agashe <ua...@cloudera.com>
Signed-off-by: Michael Stack <st...@apache.org>

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java


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

Branch: refs/heads/branch-1.3
Commit: b3f911c1c6381c801d88b1d0fe8f4620860aada0
Parents: e4f46f5
Author: huaxiangsun <hu...@apache.org>
Authored: Fri Jan 19 11:22:00 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:17 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 43 ++++++++++++++++----
 .../hbase/client/TestFromClientSide3.java       | 27 ++++++++++++
 .../hbase/regionserver/TestAtomicOperation.java |  6 +--
 3 files changed, 66 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b3f911c1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
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 499cfe4..c2ccf83 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
@@ -112,6 +112,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
 import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
+import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@@ -3098,6 +3099,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // We try to set up a batch in the range [firstIndex,lastIndexExclusive)
     int firstIndex = batchOp.nextIndexToProcess;
     int lastIndexExclusive = firstIndex;
+    RowLock prevRowLock = null;
     boolean success = false;
     int noOfPuts = 0, noOfDeletes = 0;
     WALKey walKey = null;
@@ -3160,7 +3162,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         boolean shouldBlock = numReadyToWrite == 0;
         RowLock rowLock = null;
         try {
-          rowLock = getRowLockInternal(mutation.getRow(), true, shouldBlock);
+          rowLock = getRowLockInternal(mutation.getRow(), true, shouldBlock, prevRowLock);
+        } catch (TimeoutIOException e) {
+          // We will retry when other exceptions, but we should stop if we timeout .
+          throw e;
         } catch (IOException ioe) {
           LOG.warn("Failed getting lock in batch put, row="
             + Bytes.toStringBinary(mutation.getRow()), ioe);
@@ -3171,7 +3176,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           break;
 
         } else {
-          acquiredRowLocks.add(rowLock);
+          if (rowLock != prevRowLock) {
+            // It is a different row now, add this to the acquiredRowLocks and
+            // set prevRowLock to the new returned rowLock
+            acquiredRowLocks.add(rowLock);
+            prevRowLock = rowLock;
+          }
         }
 
         lastIndexExclusive++;
@@ -3265,7 +3275,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
               checkAndPrepareMutation(cpMutation, isInReplay, cpFamilyMap, now);
 
               // Acquire row locks. If not, the whole batch will fail.
-              acquiredRowLocks.add(getRowLockInternal(cpMutation.getRow(), true, true));
+              acquiredRowLocks.add(getRowLockInternal(cpMutation.getRow(), true, true, null));
 
               if (cpMutation.getDurability() == Durability.SKIP_WAL) {
                 recordMutationWithoutWal(cpFamilyMap);
@@ -5360,17 +5370,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public RowLock getRowLock(byte[] row, boolean readLock, boolean waitForLock) throws IOException {
     // Make sure the row is inside of this region before getting the lock for it.
     checkRow(row, "row lock");
-    return getRowLockInternal(row, readLock, waitForLock);
+    return getRowLockInternal(row, readLock, waitForLock, null);
   }
 
   // getRowLock calls checkRow. Call this to skip checkRow.
   protected RowLock getRowLockInternal(byte[] row)
   throws IOException {
-    return getRowLockInternal(row, false, true);
+    return getRowLockInternal(row, false, true, null);
   }
 
-  protected RowLock getRowLockInternal(byte[] row, boolean readLock, boolean waitForLock)
-  throws IOException {
+  protected RowLock getRowLockInternal(byte[] row, boolean readLock, boolean waitForLock,
+      final RowLock prevRowLock) throws IOException {
     // create an object to use a a key in the row lock map
     HashedBytes rowKey = new HashedBytes(row);
 
@@ -5404,6 +5414,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         //
         // This can fail as
         if (readLock) {
+          // For read lock, if the caller has locked the same row previously, it will not try
+          // to acquire the same read lock. It simply returns the previous row lock.
+          RowLockImpl prevRowLockImpl = (RowLockImpl)prevRowLock;
+          if ((prevRowLockImpl != null) && (prevRowLockImpl.getLock() ==
+              rowLockContext.readWriteLock.readLock())) {
+            success = true;
+            return prevRowLock;
+          }
           result = rowLockContext.newReadLock();
         } else {
           result = rowLockContext.newWriteLock();
@@ -5441,6 +5459,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       }
       Thread.currentThread().interrupt();
       throw iie;
+    } catch (Error error) {
+      // The maximum lock count for read lock is 64K (hardcoded), when this maximum count
+      // is reached, it will throw out an Error. This Error needs to be caught so it can
+      // go ahead to process the minibatch with lock acquired.
+      LOG.warn("Error to get row lock for " + Bytes.toStringBinary(row) + ", cause: " + error);
+      IOException ioe = new IOException();
+      ioe.initCause(error);
+      if (traceScope != null) {
+        traceScope.getSpan().addTimelineAnnotation("Error getting row lock");
+      }
+      throw ioe;
     } finally {
       // Clean up the counts just in case this was the thing keeping the context alive.
       if (!success && rowLockContext != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b3f911c1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 5baab39..a372644 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -413,6 +413,33 @@ public class TestFromClientSide3 {
     }
   }
 
+  // Test Table.batch with large amount of mutations against the same key.
+  // It used to trigger read lock's "Maximum lock count exceeded" Error.
+  @Test
+  public void testHTableWithLargeBatch() throws Exception {
+    Table table = TEST_UTIL.createTable(TableName.valueOf("testHTableWithLargeBatch"),
+        new byte[][] { FAMILY });
+    int sixtyFourK = 64 * 1024;
+    try {
+      List actions = new ArrayList();
+      Object[] results = new Object[(sixtyFourK + 1) * 2];
+
+      for (int i = 0; i < sixtyFourK + 1; i ++) {
+        Put put1 = new Put(ROW);
+        put1.addColumn(FAMILY, QUALIFIER, VALUE);
+        actions.add(put1);
+
+        Put put2 = new Put(ANOTHERROW);
+        put2.addColumn(FAMILY, QUALIFIER, VALUE);
+        actions.add(put2);
+      }
+
+      table.batch(actions, results);
+    } finally {
+      table.close();
+    }
+  }
+
   @Test
   public void testHTableExistsMethodSingleRegionSingleGet() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b3f911c1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
index d9415df..b9f2290 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
@@ -663,12 +663,12 @@ public class TestAtomicOperation {
     }
 
     @Override
-    public RowLock getRowLockInternal(final byte[] row, boolean readLock, boolean waitForLock)
-      throws IOException {
+    public RowLock getRowLockInternal(final byte[] row, boolean readLock, boolean waitForLock,
+        final RowLock prevRowLock) throws IOException {
       if (testStep == TestStep.CHECKANDPUT_STARTED) {
         latch.countDown();
       }
-      return new WrappedRowLock(super.getRowLockInternal(row, readLock, waitForLock));
+      return new WrappedRowLock(super.getRowLockInternal(row, readLock, waitForLock, prevRowLock));
     }
 
     public class WrappedRowLock implements RowLock {


[17/50] [abbrv] hbase git commit: HBASE-18786 FileNotFoundException should not be silently handled for primary region replicas

Posted by ap...@apache.org.
HBASE-18786 FileNotFoundException should not be silently handled for primary region replicas

Amend HBASE-18786 Remove now invalid unit test TestCorruptedRegionStoreFile

Amend HBASE-18786 Remove now invalid unit test TestRegionServerAbort

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: b442a7fc92763fb63ed1c01ef4072c9823836d79
Parents: ac241dc
Author: Andrew Purtell <ap...@apache.org>
Authored: Thu Sep 21 13:49:07 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:17 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      |   2 -
 .../TestCorruptedRegionStoreFile.java           | 249 -------------------
 .../regionserver/TestRegionServerAbort.java     | 210 ----------------
 3 files changed, 461 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b442a7fc/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
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 74148ea..499cfe4 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
@@ -6410,8 +6410,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         if (this.joinedHeap != null) {
           result = this.joinedHeap.requestSeek(kv, true, true) || result;
         }
-      } catch (FileNotFoundException e) {
-        throw handleFileNotFound(e);
       } finally {
         closeRegionOperation();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b442a7fc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
deleted file mode 100644
index 969ef34..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
+++ /dev/null
@@ -1,249 +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.regionserver;
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.io.HFileLink;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.FSVisitor;
-import org.apache.hadoop.hbase.util.TestTableName;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-@Category(LargeTests.class)
-public class TestCorruptedRegionStoreFile {
-  private static final Log LOG = LogFactory.getLog(TestCorruptedRegionStoreFile.class);
-
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private static final String FAMILY_NAME_STR = "f";
-  private static final byte[] FAMILY_NAME = Bytes.toBytes(FAMILY_NAME_STR);
-
-  private static final int NUM_FILES = 10;
-  private static final int ROW_PER_FILE = 2000;
-  private static final int NUM_ROWS = NUM_FILES * ROW_PER_FILE;
-
-  @Rule public TestTableName TEST_TABLE = new TestTableName();
-
-  private final ArrayList<Path> storeFiles = new ArrayList<Path>();
-  private Path tableDir;
-  private int rowCount;
-
-  private static void setupConf(Configuration conf) {
-    // Disable compaction so the store file count stays constant
-    conf.setLong("hbase.hstore.compactionThreshold", NUM_FILES + 1);
-    conf.setLong("hbase.hstore.blockingStoreFiles", NUM_FILES * 2);
-  }
-
-  private void setupTable(final TableName tableName) throws IOException {
-    // load the table
-    Table table = UTIL.createTable(tableName, FAMILY_NAME);
-    try {
-      rowCount = 0;
-      byte[] value = new byte[1024];
-      byte[] q = Bytes.toBytes("q");
-      while (rowCount < NUM_ROWS) {
-        Put put = new Put(Bytes.toBytes(String.format("%010d", rowCount)));
-        put.setDurability(Durability.SKIP_WAL);
-        put.add(FAMILY_NAME, q, value);
-        table.put(put);
-
-        if ((rowCount++ % ROW_PER_FILE) == 0) {
-          // flush it
-          ((HTable)table).flushCommits();
-          UTIL.getHBaseAdmin().flush(tableName);
-        }
-      }
-    } finally {
-      UTIL.getHBaseAdmin().flush(tableName);
-      table.close();
-    }
-
-    assertEquals(NUM_ROWS, rowCount);
-
-    // get the store file paths
-    storeFiles.clear();
-    tableDir = FSUtils.getTableDir(getRootDir(), tableName);
-    FSVisitor.visitTableStoreFiles(getFileSystem(), tableDir, new FSVisitor.StoreFileVisitor() {
-      @Override
-      public void storeFile(final String region, final String family, final String hfile)
-          throws IOException {
-        HFileLink link = HFileLink.build(UTIL.getConfiguration(), tableName, region, family, hfile);
-        storeFiles.add(link.getOriginPath());
-      }
-    });
-    assertTrue("Expected at least " + NUM_FILES + " store files", storeFiles.size() >= NUM_FILES);
-    LOG.info("Store files: " + storeFiles);
-  }
-
-  @Before
-  public void setup() throws Exception {
-    setupConf(UTIL.getConfiguration());
-    UTIL.startMiniCluster(2, 3);
-
-    setupTable(TEST_TABLE.getTableName());
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    try {
-      UTIL.shutdownMiniCluster();
-    } catch (Exception e) {
-      LOG.warn("failure shutting down cluster", e);
-    }
-  }
-
-  @Test(timeout=180000)
-  public void testLosingFileDuringScan() throws Exception {
-    assertEquals(rowCount, fullScanAndCount(TEST_TABLE.getTableName()));
-
-    final FileSystem fs = getFileSystem();
-    final Path tmpStoreFilePath = new Path(UTIL.getDataTestDir(), "corruptedHFile");
-
-    // try to query with the missing file
-    int count = fullScanAndCount(TEST_TABLE.getTableName(), new ScanInjector() {
-      private boolean hasFile = true;
-
-      @Override
-      public void beforeScanNext(Table table) throws Exception {
-        // move the path away (now the region is corrupted)
-        if (hasFile) {
-          fs.copyToLocalFile(true, storeFiles.get(0), tmpStoreFilePath);
-          LOG.info("Move file to local");
-          evictHFileCache(storeFiles.get(0));
-          hasFile = false;
-        }
-      }
-    });
-    assertTrue("expected one file lost: rowCount=" + count + " lostRows=" + (NUM_ROWS - count),
-               count >= (NUM_ROWS - ROW_PER_FILE));
-  }
-
-  @Test(timeout=180000)
-  public void testLosingFileAfterScannerInit() throws Exception {
-    assertEquals(rowCount, fullScanAndCount(TEST_TABLE.getTableName()));
-
-    final FileSystem fs = getFileSystem();
-    final Path tmpStoreFilePath = new Path(UTIL.getDataTestDir(), "corruptedHFile");
-
-    // try to query with the missing file
-    int count = fullScanAndCount(TEST_TABLE.getTableName(), new ScanInjector() {
-      private boolean hasFile = true;
-
-      @Override
-      public void beforeScan(Table table, Scan scan) throws Exception {
-        // move the path away (now the region is corrupted)
-        if (hasFile) {
-          fs.copyToLocalFile(true, storeFiles.get(0), tmpStoreFilePath);
-          LOG.info("Move file to local");
-          evictHFileCache(storeFiles.get(0));
-          hasFile = false;
-        }
-      }
-    });
-    assertTrue("expected one file lost: rowCount=" + count + " lostRows=" + (NUM_ROWS - count),
-               count >= (NUM_ROWS - ROW_PER_FILE));
-  }
-
-  // ==========================================================================
-  //  Helpers
-  // ==========================================================================
-  private FileSystem getFileSystem() {
-    return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
-  }
-
-  private Path getRootDir() {
-    return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
-  }
-
-  private void evictHFileCache(final Path hfile) throws Exception {
-    for (RegionServerThread rst: UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
-      HRegionServer rs = rst.getRegionServer();
-      rs.getCacheConfig().getBlockCache().evictBlocksByHfileName(hfile.getName());
-    }
-    Thread.sleep(6000);
-  }
-
-  private int fullScanAndCount(final TableName tableName) throws Exception {
-    return fullScanAndCount(tableName, new ScanInjector());
-  }
-
-  private int fullScanAndCount(final TableName tableName, final ScanInjector injector)
-      throws Exception {
-    Table table = UTIL.getConnection().getTable(tableName);
-    int count = 0;
-    try {
-      Scan scan = new Scan();
-      scan.setCaching(1);
-      scan.setCacheBlocks(false);
-      injector.beforeScan(table, scan);
-      ResultScanner scanner = table.getScanner(scan);
-      try {
-        while (true) {
-          injector.beforeScanNext(table);
-          Result result = scanner.next();
-          injector.afterScanNext(table, result);
-          if (result == null) break;
-          if ((count++ % (ROW_PER_FILE / 2)) == 0) {
-            LOG.debug("scan next " + count);
-          }
-        }
-      } finally {
-        scanner.close();
-        injector.afterScan(table);
-      }
-    } finally {
-      table.close();
-    }
-    return count;
-  }
-
-  private class ScanInjector {
-    protected void beforeScan(Table table, Scan scan) throws Exception {}
-    protected void beforeScanNext(Table table) throws Exception {}
-    protected void afterScanNext(Table table, Result result) throws Exception {}
-    protected void afterScan(Table table) throws Exception {}
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b442a7fc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
deleted file mode 100644
index 3d66c5f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
+++ /dev/null
@@ -1,210 +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.regionserver;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-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.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.coprocessor.BaseRegionServerObserver;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests around regionserver shutdown and abort
- */
-@Category({RegionServerTests.class, MediumTests.class})
-public class TestRegionServerAbort {
-  private static final byte[] FAMILY_BYTES = Bytes.toBytes("f");
-
-  private static final Log LOG = LogFactory.getLog(TestRegionServerAbort.class);
-
-  private HBaseTestingUtility testUtil;
-  private Configuration conf;
-  private MiniDFSCluster dfsCluster;
-  private MiniHBaseCluster cluster;
-
-  @Before
-  public void setup() throws Exception {
-    testUtil = new HBaseTestingUtility();
-    conf = testUtil.getConfiguration();
-    conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
-        StopBlockingRegionObserver.class.getName());
-    // make sure we have multiple blocks so that the client does not prefetch all block locations
-    conf.set("dfs.blocksize", Long.toString(100 * 1024));
-    // prefetch the first block
-    conf.set(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY, Long.toString(100 * 1024));
-    conf.set(HConstants.REGION_IMPL, ErrorThrowingHRegion.class.getName());
-
-    testUtil.startMiniZKCluster();
-    dfsCluster = testUtil.startMiniDFSCluster(2);
-    cluster = testUtil.startMiniHBaseCluster(1, 2);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    for (JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) {
-      HRegionServer rs = t.getRegionServer();
-      RegionServerCoprocessorHost cpHost = rs.getRegionServerCoprocessorHost();
-      StopBlockingRegionObserver cp = (StopBlockingRegionObserver)
-          cpHost.findCoprocessor(StopBlockingRegionObserver.class.getName());
-      cp.setStopAllowed(true);
-    }
-    testUtil.shutdownMiniCluster();
-  }
-
-  /**
-   * Test that a regionserver is able to abort properly, even when a coprocessor
-   * throws an exception in preStopRegionServer().
-   */
-  @Test
-  public void testAbortFromRPC() throws Exception {
-    TableName tableName = TableName.valueOf("testAbortFromRPC");
-    // create a test table
-    HTable table = testUtil.createTable(tableName, FAMILY_BYTES);
-
-    // write some edits
-    testUtil.loadTable(table, FAMILY_BYTES);
-    LOG.info("Wrote data");
-    // force a flush
-    cluster.flushcache(tableName);
-    LOG.info("Flushed table");
-
-    // delete a store file from the table region
-    HRegion firstRegion = cluster.findRegionsForTable(tableName).get(0);
-
-    // aborting from region
-    HRegionFileSystem regionFS = firstRegion.getRegionFileSystem();
-    Collection<StoreFileInfo> storeFileInfos = regionFS.getStoreFiles(FAMILY_BYTES);
-    assertFalse(storeFileInfos.isEmpty());
-    StoreFileInfo firstStoreFile = storeFileInfos.iterator().next();
-
-    // move the store file away
-    // we will still be able to read the first block, since the location was pre-fetched on open
-    // but attempts to read subsequent blocks will fail
-    LOG.info("Moving store file " + firstStoreFile.getPath());
-    FileSystem fs = regionFS.getFileSystem();
-    Path tmpdir = new Path("/tmp");
-    fs.mkdirs(tmpdir);
-    assertTrue(fs.rename(firstStoreFile.getPath(),
-        new Path(tmpdir, firstStoreFile.getPath().getName())));
-
-    // start a scan, this should trigger a regionserver abort
-    ResultScanner scanner = table.getScanner(new Scan());
-    int count = 0;
-    for (Result f : scanner) {
-      count++;
-    }
-    LOG.info("Finished scan with " + count + " results");
-    // should have triggered an abort due to FileNotFoundException
-
-    // verify that the regionserver is stopped
-    assertTrue(firstRegion.getRegionServerServices().isAborted());
-    assertTrue(firstRegion.getRegionServerServices().isStopped());
-  }
-
-  /**
-   * Test that a coprocessor is able to override a normal regionserver stop request.
-   */
-  @Test
-  public void testStopOverrideFromCoprocessor() throws Exception {
-    Admin admin = testUtil.getHBaseAdmin();
-    HRegionServer regionserver = cluster.getRegionServer(0);
-    admin.stopRegionServer(regionserver.getServerName().getHostAndPort());
-
-    // regionserver should have failed to stop due to coprocessor
-    assertFalse(cluster.getRegionServer(0).isAborted());
-    assertFalse(cluster.getRegionServer(0).isStopped());
-  }
-
-  public static class StopBlockingRegionObserver extends BaseRegionServerObserver {
-    private boolean stopAllowed;
-
-    @Override
-    public void preStopRegionServer(ObserverContext<RegionServerCoprocessorEnvironment> env)
-        throws IOException {
-      if (!stopAllowed) {
-        throw new IOException("Stop not allowed");
-      }
-    }
-
-    public void setStopAllowed(boolean allowed) {
-      this.stopAllowed = allowed;
-    }
-
-    public boolean isStopAllowed() {
-      return stopAllowed;
-    }
-  }
-
-  /**
-   * Throws an exception during store file refresh in order to trigger a regionserver abort.
-   */
-  public static class ErrorThrowingHRegion extends HRegion {
-    public ErrorThrowingHRegion(Path tableDir, WAL wal, FileSystem fs, Configuration confParam,
-                                HRegionInfo regionInfo, HTableDescriptor htd,
-                                RegionServerServices rsServices) {
-      super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
-    }
-
-    public ErrorThrowingHRegion(HRegionFileSystem fs, WAL wal, Configuration confParam,
-                                HTableDescriptor htd, RegionServerServices rsServices) {
-      super(fs, wal, confParam, htd, rsServices);
-    }
-
-    @Override
-    protected boolean refreshStoreFiles(boolean force) throws IOException {
-      // forced when called through RegionScannerImpl.handleFileNotFound()
-      if (force) {
-        throw new IOException("Failing file refresh for testing");
-      }
-      return super.refreshStoreFiles(force);
-    }
-  }
-}


[02/50] [abbrv] hbase git commit: HBASE-17965 Canary tool should print the regionserver name on failure

Posted by ap...@apache.org.
HBASE-17965 Canary tool should print the regionserver name on failure

Signed-off-by: Andrew Purtell <ap...@apache.org>

Conflicts:
	hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java


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

Branch: refs/heads/branch-1.3
Commit: 212e86dbe3b23947a3560e0f24cf1ea6fdcced09
Parents: 59d36e5
Author: Karan Mehta <ka...@gmail.com>
Authored: Wed Apr 26 17:14:02 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:15 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/tool/Canary.java    | 115 +++++--------------
 .../hadoop/hbase/tool/TestCanaryTool.java       |   7 +-
 2 files changed, 32 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/212e86db/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index c777758..068e0ad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -114,15 +114,15 @@ public final class Canary implements Tool {
   public interface Sink {
     public long getReadFailureCount();
     public long incReadFailureCount();
-    public void publishReadFailure(HRegionInfo region, Exception e);
-    public void publishReadFailure(HRegionInfo region, HColumnDescriptor column, Exception e);
+    public void publishReadFailure(ServerName serverName, HRegionInfo region, Exception e);
+    public void publishReadFailure(ServerName serverName, HRegionInfo region, HColumnDescriptor column, Exception e);
     public void updateReadFailedHostList(HRegionInfo region, String serverName);
     public Map<String,String> getReadFailures();
-    public void publishReadTiming(HRegionInfo region, HColumnDescriptor column, long msTime);
+    public void publishReadTiming(ServerName serverName, HRegionInfo region, HColumnDescriptor column, long msTime);
     public long getWriteFailureCount();
-    public void publishWriteFailure(HRegionInfo region, Exception e);
-    public void publishWriteFailure(HRegionInfo region, HColumnDescriptor column, Exception e);
-    public void publishWriteTiming(HRegionInfo region, HColumnDescriptor column, long msTime);
+    public void publishWriteFailure(ServerName serverName, HRegionInfo region, Exception e);
+    public void publishWriteFailure(ServerName serverName, HRegionInfo region, HColumnDescriptor column, Exception e);
+    public void publishWriteTiming(ServerName serverName, HRegionInfo region, HColumnDescriptor column, long msTime);
     public void updateWriteFailedHostList(HRegionInfo region, String serverName);
     public Map<String,String> getWriteFailures();
   }
@@ -153,16 +153,16 @@ public final class Canary implements Tool {
     }
 
     @Override
-    public void publishReadFailure(HRegionInfo region, Exception e) {
+    public void publishReadFailure(ServerName serverName, HRegionInfo region, Exception e) {
       readFailureCount.incrementAndGet();
-      LOG.error(String.format("read from region %s failed", region.getRegionNameAsString()), e);
+      LOG.error(String.format("read from region %s on regionserver %s failed", region.getRegionNameAsString(), serverName), e);
     }
 
     @Override
-    public void publishReadFailure(HRegionInfo region, HColumnDescriptor column, Exception e) {
+    public void publishReadFailure(ServerName serverName, HRegionInfo region, HColumnDescriptor column, Exception e) {
       readFailureCount.incrementAndGet();
-      LOG.error(String.format("read from region %s column family %s failed",
-                region.getRegionNameAsString(), column.getNameAsString()), e);
+      LOG.error(String.format("read from region %s on regionserver %s column family %s failed",
+                region.getRegionNameAsString(), serverName, column.getNameAsString()), e);
     }
 
     @Override
@@ -171,9 +171,9 @@ public final class Canary implements Tool {
     }
 
     @Override
-    public void publishReadTiming(HRegionInfo region, HColumnDescriptor column, long msTime) {
-      LOG.info(String.format("read from region %s column family %s in %dms",
-        region.getRegionNameAsString(), column.getNameAsString(), msTime));
+    public void publishReadTiming(ServerName serverName, HRegionInfo region, HColumnDescriptor column, long msTime) {
+      LOG.info(String.format("read from region %s on regionserver %s column family %s in %dms",
+        region.getRegionNameAsString(), serverName, column.getNameAsString(), msTime));
     }
 
     @Override
@@ -192,22 +192,22 @@ public final class Canary implements Tool {
     }
 
     @Override
-    public void publishWriteFailure(HRegionInfo region, Exception e) {
+    public void publishWriteFailure(ServerName serverName, HRegionInfo region, Exception e) {
       writeFailureCount.incrementAndGet();
-      LOG.error(String.format("write to region %s failed", region.getRegionNameAsString()), e);
+      LOG.error(String.format("write to region %s on regionserver %s failed", region.getRegionNameAsString(), serverName), e);
     }
 
     @Override
-    public void publishWriteFailure(HRegionInfo region, HColumnDescriptor column, Exception e) {
+    public void publishWriteFailure(ServerName serverName, HRegionInfo region, HColumnDescriptor column, Exception e) {
       writeFailureCount.incrementAndGet();
-      LOG.error(String.format("write to region %s column family %s failed",
-        region.getRegionNameAsString(), column.getNameAsString()), e);
+      LOG.error(String.format("write to region %s on regionserver %s column family %s failed",
+        region.getRegionNameAsString(), serverName, column.getNameAsString()), e);
     }
 
     @Override
-    public void publishWriteTiming(HRegionInfo region, HColumnDescriptor column, long msTime) {
-      LOG.info(String.format("write to region %s column family %s in %dms",
-        region.getRegionNameAsString(), column.getNameAsString(), msTime));
+    public void publishWriteTiming(ServerName serverName, HRegionInfo region, HColumnDescriptor column, long msTime) {
+      LOG.info(String.format("write to region %s on regionserver %s column family %s in %dms",
+        region.getRegionNameAsString(), serverName, column.getNameAsString(), msTime));
     }
 
     @Override
@@ -330,7 +330,7 @@ public final class Canary implements Tool {
         tableDesc = table.getTableDescriptor();
       } catch (IOException e) {
         LOG.debug("sniffRegion failed", e);
-        sink.publishReadFailure(region, e);
+        sink.publishReadFailure(serverName, region, e);
         if (table != null) {
           try {
             table.close();
@@ -384,9 +384,9 @@ public final class Canary implements Tool {
             rs.next();
           }
           stopWatch.stop();
-          sink.publishReadTiming(region, column, stopWatch.getTime());
+          sink.publishReadTiming(serverName, region, column, stopWatch.getTime());
         } catch (Exception e) {
-          sink.publishReadFailure(region, column, e);
+          sink.publishReadFailure(serverName, region, column, e);
           sink.updateReadFailedHostList(region, serverName.getHostname());
         } finally {
           if (rs != null) {
@@ -436,14 +436,14 @@ public final class Canary implements Tool {
             long startTime = System.currentTimeMillis();
             table.put(put);
             long time = System.currentTimeMillis() - startTime;
-            sink.publishWriteTiming(region, column, time);
+            sink.publishWriteTiming(serverName, region, column, time);
           } catch (Exception e) {
-            sink.publishWriteFailure(region, column, e);
+            sink.publishWriteFailure(serverName, region, column, e);
           }
         }
         table.close();
       } catch (IOException e) {
-        sink.publishWriteFailure(region, e);
+        sink.publishWriteFailure(serverName, region, e);
         sink.updateWriteFailedHostList(region, serverName.getHostname());
       }
       return null;
@@ -1168,65 +1168,6 @@ public final class Canary implements Tool {
     return executor.invokeAll(tasks);
   }
 
-  /*
-   * For each column family of the region tries to get one row and outputs the latency, or the
-   * failure.
-   */
-  private static void sniffRegion(
-      final Admin admin,
-      final Sink sink,
-      HRegionInfo region,
-      Table table) throws Exception {
-    HTableDescriptor tableDesc = table.getTableDescriptor();
-    byte[] startKey = null;
-    Get get = null;
-    Scan scan = null;
-    ResultScanner rs = null;
-    StopWatch stopWatch = new StopWatch();
-    for (HColumnDescriptor column : tableDesc.getColumnFamilies()) {
-      stopWatch.reset();
-      startKey = region.getStartKey();
-      // Can't do a get on empty start row so do a Scan of first element if any instead.
-      if (startKey.length > 0) {
-        get = new Get(startKey);
-        get.setCacheBlocks(false);
-        get.setFilter(new FirstKeyOnlyFilter());
-        get.addFamily(column.getName());
-      } else {
-        scan = new Scan();
-        scan.setRaw(true);
-        scan.setCaching(1);
-        scan.setCacheBlocks(false);
-        scan.setFilter(new FirstKeyOnlyFilter());
-        scan.addFamily(column.getName());
-        scan.setMaxResultSize(1L);
-      }
-
-      try {
-        if (startKey.length > 0) {
-          stopWatch.start();
-          table.get(get);
-          stopWatch.stop();
-          sink.publishReadTiming(region, column, stopWatch.getTime());
-        } else {
-          stopWatch.start();
-          rs = table.getScanner(scan);
-          stopWatch.stop();
-          sink.publishReadTiming(region, column, stopWatch.getTime());
-        }
-      } catch (Exception e) {
-        sink.publishReadFailure(region, column, e);
-      } finally {
-        if (rs != null) {
-          rs.close();
-        }
-        scan = null;
-        get = null;
-        startKey = null;
-      }
-    }
-  }
-
   //  monitor for zookeeper mode
   private static class ZookeeperMonitor extends Monitor {
     private List<String> hosts;

http://git-wip-us.apache.org/repos/asf/hbase/blob/212e86db/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
index bbad233..f3d33e5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
@@ -1,5 +1,5 @@
 /**
- *
+q *
  * 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
@@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
@@ -118,7 +119,7 @@ public class TestCanaryTool {
     ToolRunner.run(testingUtility.getConfiguration(), canary, args);
     assertEquals("verify no read error count", 0, canary.getReadFailures().size());
     assertEquals("verify no write error count", 0, canary.getWriteFailures().size());
-    verify(sink, atLeastOnce()).publishReadTiming(isA(HRegionInfo.class), isA(HColumnDescriptor.class), anyLong());
+    verify(sink, atLeastOnce()).publishReadTiming(isA(ServerName.class), isA(HRegionInfo.class), isA(HColumnDescriptor.class), anyLong());
   }
 
   //no table created, so there should be no regions
@@ -166,7 +167,7 @@ public class TestCanaryTool {
     conf.setBoolean(HConstants.HBASE_CANARY_READ_RAW_SCAN_KEY, true);
     ToolRunner.run(conf, canary, args);
     verify(sink, atLeastOnce())
-        .publishReadTiming(isA(HRegionInfo.class), isA(HColumnDescriptor.class), anyLong());
+        .publishReadTiming(isA(ServerName.class), isA(HRegionInfo.class), isA(HColumnDescriptor.class), anyLong());
     assertEquals("verify no read error count", 0, canary.getReadFailures().size());
   }
 


[27/50] [abbrv] hbase git commit: HBASE-20557 Backport HBASE-17215 to branch-1

Posted by ap...@apache.org.
HBASE-20557 Backport HBASE-17215 to branch-1

The second backport of HBASE-20555

Signed-off-by: Zach York <zy...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 30b1dc00b47a0289a721cd0f932d9cdd8422534e
Parents: 8e36761
Author: TAK LON WU <wu...@amazon.com>
Authored: Sat Jun 23 08:43:21 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:19 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java |   1 +
 .../hbase/master/cleaner/HFileCleaner.java      | 316 ++++++++++++++++++-
 .../hbase/regionserver/RSRpcServices.java       |   3 +-
 .../hbase/master/cleaner/TestHFileCleaner.java  | 152 +++++++++
 4 files changed, 468 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/30b1dc00/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index aedb987..315b4c4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -861,6 +861,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     status.markComplete("Initialization successful");
     LOG.info("Master has completed initialization");
     configurationManager.registerObserver(this.balancer);
+    configurationManager.registerObserver(this.hfileCleaner);
     configurationManager.registerObserver(this.logCleaner);
 
     // Set master as 'initialized'.

http://git-wip-us.apache.org/repos/asf/hbase/blob/30b1dc00/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
index 89c316b..defe851 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
@@ -17,22 +17,32 @@
  */
 package org.apache.hadoop.hbase.master.cleaner;
 
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+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.Stoppable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+
 /**
  * This Chore, every time it runs, will clear the HFiles in the hfile archive
  * folder that are deletable for each HFile cleaner in the chain.
  */
 @InterfaceAudience.Private
-public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
+public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> implements
+    ConfigurationObserver {
 
   public static final String MASTER_HFILE_CLEANER_PLUGINS = "hbase.master.hfilecleaner.plugins";
 
@@ -41,6 +51,34 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
     this(period, stopper, conf, fs, directory, null);
   }
 
+  // Configuration key for large/small throttle point
+  public final static String HFILE_DELETE_THROTTLE_THRESHOLD =
+      "hbase.regionserver.thread.hfilecleaner.throttle";
+  public final static int DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD = 64 * 1024 * 1024;// 64M
+
+  // Configuration key for large queue size
+  public final static String LARGE_HFILE_DELETE_QUEUE_SIZE =
+      "hbase.regionserver.hfilecleaner.large.queue.size";
+  public final static int DEFAULT_LARGE_HFILE_DELETE_QUEUE_SIZE = 1048576;
+
+  // Configuration key for small queue size
+  public final static String SMALL_HFILE_DELETE_QUEUE_SIZE =
+      "hbase.regionserver.hfilecleaner.small.queue.size";
+  public final static int DEFAULT_SMALL_HFILE_DELETE_QUEUE_SIZE = 1048576;
+
+  private static final Log LOG = LogFactory.getLog(HFileCleaner.class);
+
+  BlockingQueue<HFileDeleteTask> largeFileQueue;
+  BlockingQueue<HFileDeleteTask> smallFileQueue;
+  private int throttlePoint;
+  private int largeQueueSize;
+  private int smallQueueSize;
+  private List<Thread> threads = new ArrayList<Thread>();
+  private boolean running;
+
+  private long deletedLargeFiles = 0L;
+  private long deletedSmallFiles = 0L;
+
   /**
    * @param period the period of time to sleep between each run
    * @param stopper the stopper
@@ -53,6 +91,15 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
                       Path directory, Map<String, Object> params) {
     super("HFileCleaner", period, stopper, conf, fs,
       directory, MASTER_HFILE_CLEANER_PLUGINS, params);
+    throttlePoint =
+        conf.getInt(HFILE_DELETE_THROTTLE_THRESHOLD, DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD);
+    largeQueueSize =
+        conf.getInt(LARGE_HFILE_DELETE_QUEUE_SIZE, DEFAULT_LARGE_HFILE_DELETE_QUEUE_SIZE);
+    smallQueueSize =
+        conf.getInt(SMALL_HFILE_DELETE_QUEUE_SIZE, DEFAULT_SMALL_HFILE_DELETE_QUEUE_SIZE);
+    largeFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(largeQueueSize);
+    smallFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(smallQueueSize);
+    startHFileDeleteThreads();
   }
 
   @Override
@@ -69,4 +116,267 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
   public List<BaseHFileCleanerDelegate> getDelegatesForTesting() {
     return this.cleanersChain;
   }
+
+  @Override
+  public int deleteFiles(Iterable<FileStatus> filesToDelete) {
+    int deletedFiles = 0;
+    List<HFileDeleteTask> tasks = new ArrayList<HFileDeleteTask>();
+    // construct delete tasks and add into relative queue
+    for (FileStatus file : filesToDelete) {
+      HFileDeleteTask task = deleteFile(file);
+      if (task != null) {
+        tasks.add(task);
+      }
+    }
+    // wait for each submitted task to finish
+    for (HFileDeleteTask task : tasks) {
+      if (task.getResult()) {
+        deletedFiles++;
+      }
+    }
+    return deletedFiles;
+  }
+
+  /**
+   * Construct an {@link HFileDeleteTask} for each file to delete and add into the correct queue
+   * @param file the file to delete
+   * @return HFileDeleteTask to track progress
+   */
+  private HFileDeleteTask deleteFile(FileStatus file) {
+    HFileDeleteTask task = new HFileDeleteTask(file);
+    boolean enqueued = dispatch(task);
+    return enqueued ? task : null;
+  }
+
+  private boolean dispatch(HFileDeleteTask task) {
+    if (task.fileLength >= this.throttlePoint) {
+      if (!this.largeFileQueue.offer(task)) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Large file deletion queue is full");
+        }
+        return false;
+      }
+    } else {
+      if (!this.smallFileQueue.offer(task)) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Small file deletion queue is full");
+        }
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public void cleanup() {
+    super.cleanup();
+    stopHFileDeleteThreads();
+  }
+
+  /**
+   * Start threads for hfile deletion
+   */
+  private void startHFileDeleteThreads() {
+    final String n = Thread.currentThread().getName();
+    running = true;
+    // start thread for large file deletion
+    Thread large = new Thread() {
+      @Override
+      public void run() {
+        consumerLoop(largeFileQueue);
+      }
+    };
+    large.setDaemon(true);
+    large.setName(n + "-HFileCleaner.large-" + System.currentTimeMillis());
+    large.start();
+    LOG.debug("Starting hfile cleaner for large files: " + large.getName());
+    threads.add(large);
+
+    // start thread for small file deletion
+    Thread small = new Thread() {
+      @Override
+      public void run() {
+        consumerLoop(smallFileQueue);
+      }
+    };
+    small.setDaemon(true);
+    small.setName(n + "-HFileCleaner.small-" + System.currentTimeMillis());
+    small.start();
+    LOG.debug("Starting hfile cleaner for small files: " + small.getName());
+    threads.add(small);
+  }
+
+  protected void consumerLoop(BlockingQueue<HFileDeleteTask> queue) {
+    try {
+      while (running) {
+        HFileDeleteTask task = null;
+        try {
+          task = queue.take();
+        } catch (InterruptedException e) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Interrupted while trying to take a task from queue", e);
+          }
+          break;
+        }
+        if (task != null) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Removing: " + task.filePath + " from archive");
+          }
+          boolean succeed;
+          try {
+            succeed = this.fs.delete(task.filePath, false);
+          } catch (IOException e) {
+            LOG.warn("Failed to delete file " + task.filePath, e);
+            succeed = false;
+          }
+          task.setResult(succeed);
+          if (succeed) {
+            countDeletedFiles(queue == largeFileQueue);
+          }
+        }
+      }
+    } finally {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Exit thread: " + Thread.currentThread());
+      }
+    }
+  }
+
+  // Currently only for testing purpose
+  private void countDeletedFiles(boolean isLarge) {
+    if (isLarge) {
+      if (deletedLargeFiles == Long.MAX_VALUE) {
+        LOG.info("Deleted more than Long.MAX_VALUE large files, reset counter to 0");
+        deletedLargeFiles = 0L;
+      }
+      deletedLargeFiles++;
+    } else {
+      if (deletedSmallFiles == Long.MAX_VALUE) {
+        LOG.info("Deleted more than Long.MAX_VALUE small files, reset counter to 0");
+        deletedSmallFiles = 0L;
+      }
+      deletedSmallFiles++;
+    }
+  }
+
+  /**
+   * Stop threads for hfile deletion
+   */
+  private void stopHFileDeleteThreads() {
+    running = false;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Stopping file delete threads");
+    }
+    for(Thread thread: threads){
+      thread.interrupt();
+    }
+  }
+
+  static class HFileDeleteTask {
+    private static final long MAX_WAIT = 60 * 1000L;
+    private static final long WAIT_UNIT = 1000L;
+
+    boolean done = false;
+    boolean result;
+    final Path filePath;
+    final long fileLength;
+
+    public HFileDeleteTask(FileStatus file) {
+      this.filePath = file.getPath();
+      this.fileLength = file.getLen();
+    }
+
+    public synchronized void setResult(boolean result) {
+      this.done = true;
+      this.result = result;
+      notify();
+    }
+
+    public synchronized boolean getResult() {
+      long waitTime = 0;
+      try {
+        while (!done) {
+          wait(WAIT_UNIT);
+          waitTime += WAIT_UNIT;
+          if (done) {
+            return this.result;
+          }
+          if (waitTime > MAX_WAIT) {
+            LOG.warn("Wait more than " + MAX_WAIT + " ms for deleting " + this.filePath
+                + ", exit...");
+            return false;
+          }
+        }
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for result of deleting " + filePath
+            + ", will return false", e);
+        return false;
+      }
+      return this.result;
+    }
+  }
+
+  @VisibleForTesting
+  public List<Thread> getCleanerThreads() {
+    return threads;
+  }
+
+  @VisibleForTesting
+  public long getNumOfDeletedLargeFiles() {
+    return deletedLargeFiles;
+  }
+
+  @VisibleForTesting
+  public long getNumOfDeletedSmallFiles() {
+    return deletedSmallFiles;
+  }
+
+  @VisibleForTesting
+  public long getLargeQueueSize() {
+    return largeQueueSize;
+  }
+
+  @VisibleForTesting
+  public long getSmallQueueSize() {
+    return smallQueueSize;
+  }
+
+  @VisibleForTesting
+  public long getThrottlePoint() {
+    return throttlePoint;
+  }
+
+  @Override
+  public void onConfigurationChange(Configuration conf) {
+    StringBuilder builder = new StringBuilder();
+    builder.append("Updating configuration for HFileCleaner, previous throttle point: ")
+        .append(throttlePoint).append(", largeQueueSize: ").append(largeQueueSize)
+        .append(", smallQueueSize: ").append(smallQueueSize);
+    stopHFileDeleteThreads();
+    this.throttlePoint =
+        conf.getInt(HFILE_DELETE_THROTTLE_THRESHOLD, DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD);
+    this.largeQueueSize =
+        conf.getInt(LARGE_HFILE_DELETE_QUEUE_SIZE, DEFAULT_LARGE_HFILE_DELETE_QUEUE_SIZE);
+    this.smallQueueSize =
+        conf.getInt(SMALL_HFILE_DELETE_QUEUE_SIZE, DEFAULT_SMALL_HFILE_DELETE_QUEUE_SIZE);
+    // record the left over tasks
+    List<HFileDeleteTask> leftOverTasks = new ArrayList<>();
+    for (HFileDeleteTask task : largeFileQueue) {
+      leftOverTasks.add(task);
+    }
+    for (HFileDeleteTask task : smallFileQueue) {
+      leftOverTasks.add(task);
+    }
+    largeFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(largeQueueSize);
+    smallFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(smallQueueSize);
+    threads.clear();
+    builder.append("; new throttle point: ").append(throttlePoint).append(", largeQueueSize: ")
+        .append(largeQueueSize).append(", smallQueueSize: ").append(smallQueueSize);
+    LOG.debug(builder.toString());
+    startHFileDeleteThreads();
+    // re-dispatch the left over tasks
+    for (HFileDeleteTask task : leftOverTasks) {
+      dispatch(task);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/30b1dc00/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 cd7a4c7..75d5e1a 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
@@ -1283,7 +1283,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       throw new RegionServerStoppedException("File system not available");
     }
     if (!regionServer.isOnline()) {
-      throw new ServerNotRunningYetException("Server is not running yet");
+      throw new ServerNotRunningYetException("Server " + regionServer.serverName
+          + " is not running yet");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/30b1dc00/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 32d2afd..5712729 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -22,10 +22,12 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -42,6 +44,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -254,4 +257,153 @@ public class TestHFileCleaner {
       return null;
     }
   }
+
+  @Test
+  public void testThreadCleanup() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+    conf.setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, "");
+    Server server = new DummyServer();
+    Path archivedHfileDir =
+        new Path(UTIL.getDataTestDirOnTestFS(), HConstants.HFILE_ARCHIVE_DIRECTORY);
+
+    // setup the cleaner
+    FileSystem fs = UTIL.getDFSCluster().getFileSystem();
+    HFileCleaner cleaner = new HFileCleaner(1000, server, conf, fs, archivedHfileDir);
+    // clean up archive directory
+    fs.delete(archivedHfileDir, true);
+    fs.mkdirs(archivedHfileDir);
+    // create some file to delete
+    fs.createNewFile(new Path(archivedHfileDir, "dfd-dfd"));
+    // launch the chore
+    cleaner.chore();
+    // call cleanup
+    cleaner.cleanup();
+    // wait awhile for thread to die
+    Thread.sleep(100);
+    for (Thread thread : cleaner.getCleanerThreads()) {
+      Assert.assertFalse(thread.isAlive());
+    }
+  }
+
+  @Test
+  public void testLargeSmallIsolation() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+    // no cleaner policies = delete all files
+    conf.setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, "");
+    conf.setInt(HFileCleaner.HFILE_DELETE_THROTTLE_THRESHOLD, 512 * 1024);
+    Server server = new DummyServer();
+    Path archivedHfileDir =
+        new Path(UTIL.getDataTestDirOnTestFS(), HConstants.HFILE_ARCHIVE_DIRECTORY);
+
+    // setup the cleaner
+    FileSystem fs = UTIL.getDFSCluster().getFileSystem();
+    HFileCleaner cleaner = new HFileCleaner(1000, server, conf, fs, archivedHfileDir);
+    // clean up archive directory
+    fs.delete(archivedHfileDir, true);
+    fs.mkdirs(archivedHfileDir);
+    // necessary set up
+    final int LARGE_FILE_NUM = 5;
+    final int SMALL_FILE_NUM = 20;
+    createFilesForTesting(LARGE_FILE_NUM, SMALL_FILE_NUM, fs, archivedHfileDir);
+    // call cleanup
+    cleaner.chore();
+
+    Assert.assertEquals(LARGE_FILE_NUM, cleaner.getNumOfDeletedLargeFiles());
+    Assert.assertEquals(SMALL_FILE_NUM, cleaner.getNumOfDeletedSmallFiles());
+  }
+
+  @Test(timeout = 60 * 1000)
+  public void testOnConfigurationChange() throws Exception {
+    // constants
+    final int ORIGINAL_THROTTLE_POINT = 512 * 1024;
+    final int ORIGINAL_QUEUE_SIZE = 512;
+    final int UPDATE_THROTTLE_POINT = 1024;// small enough to change large/small check
+    final int UPDATE_QUEUE_SIZE = 1024;
+    final int LARGE_FILE_NUM = 5;
+    final int SMALL_FILE_NUM = 20;
+
+    Configuration conf = UTIL.getConfiguration();
+    // no cleaner policies = delete all files
+    conf.setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, "");
+    conf.setInt(HFileCleaner.HFILE_DELETE_THROTTLE_THRESHOLD, ORIGINAL_THROTTLE_POINT);
+    conf.setInt(HFileCleaner.LARGE_HFILE_DELETE_QUEUE_SIZE, ORIGINAL_QUEUE_SIZE);
+    conf.setInt(HFileCleaner.SMALL_HFILE_DELETE_QUEUE_SIZE, ORIGINAL_QUEUE_SIZE);
+    Server server = new DummyServer();
+    Path archivedHfileDir =
+        new Path(UTIL.getDataTestDirOnTestFS(), HConstants.HFILE_ARCHIVE_DIRECTORY);
+
+    // setup the cleaner
+    FileSystem fs = UTIL.getDFSCluster().getFileSystem();
+    final HFileCleaner cleaner = new HFileCleaner(1000, server, conf, fs, archivedHfileDir);
+    Assert.assertEquals(ORIGINAL_THROTTLE_POINT, cleaner.getThrottlePoint());
+    Assert.assertEquals(ORIGINAL_QUEUE_SIZE, cleaner.getLargeQueueSize());
+    Assert.assertEquals(ORIGINAL_QUEUE_SIZE, cleaner.getSmallQueueSize());
+
+    // clean up archive directory and create files for testing
+    fs.delete(archivedHfileDir, true);
+    fs.mkdirs(archivedHfileDir);
+    createFilesForTesting(LARGE_FILE_NUM, SMALL_FILE_NUM, fs, archivedHfileDir);
+
+    // call cleaner, run as daemon to test the interrupt-at-middle case
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        cleaner.chore();
+      }
+    };
+    t.setDaemon(true);
+    t.start();
+    // let the cleaner run for some while
+    Thread.sleep(20);
+
+    // trigger configuration change
+    Configuration newConf = new Configuration(conf);
+    newConf.setInt(HFileCleaner.HFILE_DELETE_THROTTLE_THRESHOLD, UPDATE_THROTTLE_POINT);
+    newConf.setInt(HFileCleaner.LARGE_HFILE_DELETE_QUEUE_SIZE, UPDATE_QUEUE_SIZE);
+    newConf.setInt(HFileCleaner.SMALL_HFILE_DELETE_QUEUE_SIZE, UPDATE_QUEUE_SIZE);
+    cleaner.onConfigurationChange(newConf);
+    LOG.debug("File deleted from large queue: " + cleaner.getNumOfDeletedLargeFiles()
+        + "; from small queue: " + cleaner.getNumOfDeletedSmallFiles());
+
+    // check values after change
+    Assert.assertEquals(UPDATE_THROTTLE_POINT, cleaner.getThrottlePoint());
+    Assert.assertEquals(UPDATE_QUEUE_SIZE, cleaner.getLargeQueueSize());
+    Assert.assertEquals(UPDATE_QUEUE_SIZE, cleaner.getSmallQueueSize());
+    Assert.assertEquals(2, cleaner.getCleanerThreads().size());
+
+    // wait until clean done and check
+    t.join();
+    LOG.debug("File deleted from large queue: " + cleaner.getNumOfDeletedLargeFiles()
+        + "; from small queue: " + cleaner.getNumOfDeletedSmallFiles());
+    Assert.assertTrue("Should delete more than " + LARGE_FILE_NUM
+        + " files from large queue but actually " + cleaner.getNumOfDeletedLargeFiles(),
+      cleaner.getNumOfDeletedLargeFiles() > LARGE_FILE_NUM);
+    Assert.assertTrue("Should delete less than " + SMALL_FILE_NUM
+        + " files from small queue but actually " + cleaner.getNumOfDeletedSmallFiles(),
+      cleaner.getNumOfDeletedSmallFiles() < SMALL_FILE_NUM);
+  }
+
+  private void createFilesForTesting(int largeFileNum, int smallFileNum, FileSystem fs,
+      Path archivedHfileDir) throws IOException {
+    final Random rand = new Random();
+    final byte[] large = new byte[1024 * 1024];
+    for (int i = 0; i < large.length; i++) {
+      large[i] = (byte) rand.nextInt(128);
+    }
+    final byte[] small = new byte[1024];
+    for (int i = 0; i < small.length; i++) {
+      small[i] = (byte) rand.nextInt(128);
+    }
+    // create large and small files
+    for (int i = 1; i <= largeFileNum; i++) {
+      FSDataOutputStream out = fs.create(new Path(archivedHfileDir, "large-file-" + i));
+      out.write(large);
+      out.close();
+    }
+    for (int i = 1; i <= smallFileNum; i++) {
+      FSDataOutputStream out = fs.create(new Path(archivedHfileDir, "small-file-" + i));
+      out.write(small);
+      out.close();
+    }
+  }
 }


[38/50] [abbrv] hbase git commit: HBASE-17437 Support specifying a WAL directory outside of the root directory (Yishan Yang and Zach York) HBASE-17588 Remove unused imports brought in by HBASE-17437 (Zach York)

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
index 760cdc1..1fcb241 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -88,6 +89,8 @@ public class TestFSHLog {
   protected static Configuration conf;
   protected static FileSystem fs;
   protected static Path dir;
+  protected static Path rootDir;
+  protected static Path walRootDir;
   protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   @Rule
@@ -99,8 +102,10 @@ public class TestFSHLog {
     for (FileStatus dir : entries) {
       fs.delete(dir.getPath(), true);
     }
-    final Path hbaseDir = TEST_UTIL.createRootDir();
-    dir = new Path(hbaseDir, currentTest.getMethodName());
+    rootDir = TEST_UTIL.createRootDir();
+    walRootDir = TEST_UTIL.createWALRootDir();
+    dir = new Path(walRootDir, currentTest.getMethodName());
+    assertNotEquals(rootDir, walRootDir);
   }
 
   @After
@@ -133,6 +138,8 @@ public class TestFSHLog {
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
+    fs.delete(rootDir, true);
+    fs.delete(walRootDir, true);
     TEST_UTIL.shutdownMiniCluster();
   }
 
@@ -144,7 +151,7 @@ public class TestFSHLog {
     // test to see whether the coprocessor is loaded or not.
     FSHLog log = null;
     try {
-      log = new FSHLog(fs, FSUtils.getRootDir(conf), dir.toString(),
+      log = new FSHLog(fs, walRootDir, dir.toString(),
           HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
       WALCoprocessorHost host = log.getCoprocessorHost();
       Coprocessor c = host.findCoprocessor(SampleRegionWALObserver.class.getName());
@@ -195,7 +202,7 @@ public class TestFSHLog {
     FSHLog wal1 = null;
     FSHLog walMeta = null;
     try {
-      wal1 = new FSHLog(fs, FSUtils.getRootDir(conf), dir.toString(),
+      wal1 = new FSHLog(fs, walRootDir, dir.toString(),
           HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
       LOG.debug("Log obtained is: " + wal1);
       Comparator<Path> comp = wal1.LOG_NAME_COMPARATOR;
@@ -205,7 +212,7 @@ public class TestFSHLog {
       assertTrue(comp.compare(p1, p1) == 0);
       // comparing with different filenum.
       assertTrue(comp.compare(p1, p2) < 0);
-      walMeta = new FSHLog(fs, FSUtils.getRootDir(conf), dir.toString(),
+      walMeta = new FSHLog(fs, walRootDir, dir.toString(),
           HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null,
           DefaultWALProvider.META_WAL_PROVIDER_ID);
       Comparator<Path> compMeta = walMeta.LOG_NAME_COMPARATOR;
@@ -253,7 +260,7 @@ public class TestFSHLog {
     LOG.debug("testFindMemStoresEligibleForFlush");
     Configuration conf1 = HBaseConfiguration.create(conf);
     conf1.setInt("hbase.regionserver.maxlogs", 1);
-    FSHLog wal = new FSHLog(fs, FSUtils.getRootDir(conf1), dir.toString(),
+    FSHLog wal = new FSHLog(fs, walRootDir, dir.toString(),
         HConstants.HREGION_OLDLOGDIR_NAME, conf1, null, true, null, null);
     HTableDescriptor t1 =
         new HTableDescriptor(TableName.valueOf("t1")).addFamily(new HColumnDescriptor("row"));
@@ -330,7 +337,7 @@ public class TestFSHLog {
   @Test(expected=IOException.class)
   public void testFailedToCreateWALIfParentRenamed() throws IOException {
     final String name = "testFailedToCreateWALIfParentRenamed";
-    FSHLog log = new FSHLog(fs, FSUtils.getRootDir(conf), name, HConstants.HREGION_OLDLOGDIR_NAME,
+    FSHLog log = new FSHLog(fs, walRootDir, name, HConstants.HREGION_OLDLOGDIR_NAME,
         conf, null, true, null, null);
     long filenum = System.currentTimeMillis();
     Path path = log.computeFilename(filenum);
@@ -359,13 +366,13 @@ public class TestFSHLog {
     final byte[] rowName = tableName.getName();
     final HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(new HColumnDescriptor("f"));
-    HRegion r = HRegion.createHRegion(hri, TEST_UTIL.getDefaultRootDirPath(),
+    HRegion r = HRegion.createHRegion(hri, rootDir,
       TEST_UTIL.getConfiguration(), htd);
     HRegion.closeHRegion(r);
     final int countPerFamily = 10;
     final MutableBoolean goslow = new MutableBoolean(false);
     // subclass and doctor a method.
-    FSHLog wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDefaultRootDirPath(),
+    FSHLog wal = new FSHLog(FileSystem.get(conf), walRootDir,
         testName, conf) {
       @Override
       void atHeadOfRingBufferEventHandlerAppend() {
@@ -377,7 +384,7 @@ public class TestFSHLog {
       }
     };
     HRegion region = HRegion.openHRegion(TEST_UTIL.getConfiguration(),
-      TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath(), hri, htd, wal);
+      TEST_UTIL.getTestFileSystem(), rootDir, hri, htd, wal);
     EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate();
     try {
       List<Put> puts = null;
@@ -430,7 +437,7 @@ public class TestFSHLog {
       SecurityException, IllegalArgumentException, IllegalAccessException {
     final String name = "testSyncRunnerIndexOverflow";
     FSHLog log =
-        new FSHLog(fs, FSUtils.getRootDir(conf), name, HConstants.HREGION_OLDLOGDIR_NAME, conf,
+        new FSHLog(fs, walRootDir, name, HConstants.HREGION_OLDLOGDIR_NAME, conf,
             null, true, null, null);
     try {
       Field ringBufferEventHandlerField = FSHLog.class.getDeclaredField("ringBufferEventHandler");
@@ -468,7 +475,7 @@ public class TestFSHLog {
     final CountDownLatch putFinished = new CountDownLatch(1);
 
     try (FSHLog log =
-        new FSHLog(fs, FSUtils.getRootDir(conf), name, HConstants.HREGION_OLDLOGDIR_NAME, conf,
+        new FSHLog(fs, walRootDir, name, HConstants.HREGION_OLDLOGDIR_NAME, conf,
             null, true, null, null)) {
 
       log.registerWALActionsListener(new WALActionsListener.Base() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
index 2ccf12b..3f15c2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
@@ -71,7 +71,8 @@ public class TestLogRollAbort {
 
   /* For the split-then-roll test */
   private static final Path HBASEDIR = new Path("/hbase");
-  private static final Path OLDLOGDIR = new Path(HBASEDIR, HConstants.HREGION_OLDLOGDIR_NAME);
+  private static final Path HBASELOGDIR = new Path("/hbaselog");
+  private static final Path OLDLOGDIR = new Path(HBASELOGDIR, HConstants.HREGION_OLDLOGDIR_NAME);
 
   // Need to override this setup so we can edit the config before it gets sent
   // to the HDFS & HBase cluster startup.
@@ -112,6 +113,7 @@ public class TestLogRollAbort {
     // disable region rebalancing (interferes with log watching)
     cluster.getMaster().balanceSwitch(false);
     FSUtils.setRootDir(conf, HBASEDIR);
+    FSUtils.setWALRootDir(conf, HBASELOGDIR);
   }
 
   @After
@@ -183,7 +185,7 @@ public class TestLogRollAbort {
   public void testLogRollAfterSplitStart() throws IOException {
     LOG.info("Verify wal roll after split starts will fail.");
     String logName = "testLogRollAfterSplitStart";
-    Path thisTestsDir = new Path(HBASEDIR, DefaultWALProvider.getWALDirectoryName(logName));
+    Path thisTestsDir = new Path(HBASELOGDIR, DefaultWALProvider.getWALDirectoryName(logName));
     final WALFactory wals = new WALFactory(conf, null, logName);
 
     try {
@@ -220,7 +222,7 @@ public class TestLogRollAbort {
       LOG.debug("Renamed region directory: " + rsSplitDir);
 
       LOG.debug("Processing the old log files.");
-      WALSplitter.split(HBASEDIR, rsSplitDir, OLDLOGDIR, fs, conf, wals);
+      WALSplitter.split(HBASELOGDIR, rsSplitDir, OLDLOGDIR, fs, conf, wals);
 
       LOG.debug("Trying to roll the WAL.");
       try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
index 3eba637..bac1b6f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
@@ -20,14 +20,19 @@ package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -53,21 +58,27 @@ public class TestWALActionsListener {
       new HBaseTestingUtility();
 
   private final static byte[] SOME_BYTES =  Bytes.toBytes("t");
-  private static FileSystem fs;
   private static Configuration conf;
+  private static Path rootDir;
+  private static Path walRootDir;
+  private static FileSystem fs;
+  private static FileSystem walFs;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     conf.setInt("hbase.regionserver.maxlogs", 5);
-    fs = FileSystem.get(conf);
-    FSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir());
+    rootDir = TEST_UTIL.createRootDir();
+    walRootDir = TEST_UTIL.createWALRootDir();
+    fs = FSUtils.getRootDirFileSystem(conf);
+    walFs = FSUtils.getWALFileSystem(conf);
   }
 
   @Before
   public void setUp() throws Exception {
-    fs.delete(new Path(TEST_UTIL.getDataTestDir(), HConstants.HREGION_LOGDIR_NAME), true);
-    fs.delete(new Path(TEST_UTIL.getDataTestDir(), HConstants.HREGION_OLDLOGDIR_NAME), true);
+    fs.delete(rootDir, true);
+    walFs.delete(new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME), true);
+    walFs.delete(new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME), true);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
index b478b2d..2622f6d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
@@ -121,6 +122,7 @@ public class TestWALReplay {
   static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private final EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate();
   private Path hbaseRootDir = null;
+  private Path hbaseWALRootDir = null;
   private String logName;
   private Path oldLogDir;
   private Path logDir;
@@ -142,8 +144,12 @@ public class TestWALReplay {
     TEST_UTIL.startMiniCluster(3);
     Path hbaseRootDir =
       TEST_UTIL.getDFSCluster().getFileSystem().makeQualified(new Path("/hbase"));
-    LOG.info("hbase.rootdir=" + hbaseRootDir);
+    Path hbaseWALRootDir =
+        TEST_UTIL.getDFSCluster().getFileSystem().makeQualified(new Path("/hbaselog"));
+    LOG.info(HConstants.HBASE_DIR + "=" + hbaseRootDir);
+    LOG.info(HFileSystem.HBASE_WAL_DIR + "=" + hbaseWALRootDir);
     FSUtils.setRootDir(conf, hbaseRootDir);
+    FSUtils.setWALRootDir(conf, hbaseWALRootDir);
   }
 
   @AfterClass
@@ -156,12 +162,16 @@ public class TestWALReplay {
     this.conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
     this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
     this.hbaseRootDir = FSUtils.getRootDir(this.conf);
-    this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    this.hbaseWALRootDir = FSUtils.getWALRootDir(this.conf);
+    this.oldLogDir = new Path(this.hbaseWALRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
     this.logName = DefaultWALProvider.getWALDirectoryName(currentTest.getMethodName() + "-manual");
-    this.logDir = new Path(this.hbaseRootDir, logName);
+    this.logDir = new Path(this.hbaseWALRootDir, logName);
     if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) {
       TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
     }
+    if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseWALRootDir)) {
+      TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseWALRootDir, true);
+    }
     this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
         RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
     this.wals = new WALFactory(conf, null, currentTest.getMethodName());
@@ -171,6 +181,7 @@ public class TestWALReplay {
   public void tearDown() throws Exception {
     this.wals.close();
     TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
+    TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseWALRootDir, true);
   }
 
   /*
@@ -296,11 +307,11 @@ public class TestWALReplay {
 
     MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
+    Path basedir = FSUtils.getTableDir(hbaseWALRootDir, tableName);
     deleteDir(basedir);
 
     HTableDescriptor htd = createBasic3FamilyHTD(tableName);
-    HRegion region2 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd);
+    HRegion region2 = HRegion.createHRegion(hri, hbaseWALRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
     final byte [] rowName = tableName.getName();
 
@@ -326,7 +337,7 @@ public class TestWALReplay {
 
     WAL wal3 = createWAL(this.conf);
     try {
-      HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal3);
+      HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal3);
       long seqid = region.getOpenSeqNum();
       // The regions opens with sequenceId as 1. With 6k edits, its sequence number reaches 6k + 1.
       // When opened, this region would apply 6k edits, and increment the sequenceId by 1
@@ -358,13 +369,13 @@ public class TestWALReplay {
     final TableName tableName =
         TableName.valueOf("testRegionMadeOfBulkLoadedFilesOnly");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = new Path(this.hbaseRootDir, tableName.getNameAsString());
+    final Path basedir = new Path(this.hbaseWALRootDir, tableName.getNameAsString());
     deleteDir(basedir);
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
-    HRegion region2 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd);
+    HRegion region2 = HRegion.createHRegion(hri, hbaseWALRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
     WAL wal = createWAL(this.conf);
-    HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
 
     byte [] family = htd.getFamilies().iterator().next().getName();
     Path f =  new Path(basedir, "hfile");
@@ -393,7 +404,7 @@ public class TestWALReplay {
         WAL wal2 = createWAL(newConf);
 
         HRegion region2 = HRegion.openHRegion(newConf, FileSystem.get(newConf),
-          hbaseRootDir, hri, htd, wal2);
+          hbaseWALRootDir, hri, htd, wal2);
         long seqid2 = region2.getOpenSeqNum();
         assertTrue(seqid2 > -1);
         assertEquals(rowsInsertedCount, getScannedCount(region2.getScanner(new Scan())));
@@ -424,14 +435,14 @@ public class TestWALReplay {
     final TableName tableName =
         TableName.valueOf("testCompactedBulkLoadedFiles");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = new Path(this.hbaseRootDir, tableName.getNameAsString());
+    final Path basedir = new Path(this.hbaseWALRootDir, tableName.getNameAsString());
     deleteDir(basedir);
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region2 = HRegion.createHRegion(hri,
-        hbaseRootDir, this.conf, htd);
+        hbaseWALRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
     WAL wal = createWAL(this.conf);
-    HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
 
     // Add an edit so something in the WAL
     byte [] row = tableName.getName();
@@ -465,7 +476,7 @@ public class TestWALReplay {
         WAL wal2 = createWAL(newConf);
 
         HRegion region2 = HRegion.openHRegion(newConf, FileSystem.get(newConf),
-            hbaseRootDir, hri, htd, wal2);
+            hbaseWALRootDir, hri, htd, wal2);
         long seqid2 = region2.getOpenSeqNum();
         assertTrue(seqid2 > -1);
         assertEquals(rowsInsertedCount, getScannedCount(region2.getScanner(new Scan())));
@@ -495,19 +506,19 @@ public class TestWALReplay {
     final TableName tableName =
         TableName.valueOf("testReplayEditsWrittenViaHRegion");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseWALRootDir, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region3 = HRegion.createHRegion(hri,
-            hbaseRootDir, this.conf, htd);
+            hbaseWALRootDir, this.conf, htd);
     HRegion.closeHRegion(region3);
     // Write countPerFamily edits into the three families.  Do a flush on one
     // of the families during the load of edits so its seqid is not same as
     // others to test we do right thing when different seqids.
     WAL wal = createWAL(this.conf);
-    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
     long seqid = region.getOpenSeqNum();
     boolean first = true;
     for (HColumnDescriptor hcd: htd.getFamilies()) {
@@ -530,7 +541,7 @@ public class TestWALReplay {
     wal.shutdown();
     runWALSplit(this.conf);
     WAL wal2 = createWAL(this.conf);
-    HRegion region2 = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal2);
+    HRegion region2 = HRegion.openHRegion(conf, this.fs, hbaseWALRootDir, hri, htd, wal2);
     long seqid2 = region2.getOpenSeqNum();
     assertTrue(seqid + result.size() < seqid2);
     final Result result1b = region2.get(g);
@@ -605,19 +616,19 @@ public class TestWALReplay {
     final TableName tableName =
         TableName.valueOf("testReplayEditsWrittenViaHRegion");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseWALRootDir, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region3 = HRegion.createHRegion(hri,
-            hbaseRootDir, this.conf, htd);
+            hbaseWALRootDir, this.conf, htd);
     HRegion.closeHRegion(region3);
     // Write countPerFamily edits into the three families.  Do a flush on one
     // of the families during the load of edits so its seqid is not same as
     // others to test we do right thing when different seqids.
     WAL wal = createWAL(this.conf);
-    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
     long seqid = region.getOpenSeqNum();
     for (HColumnDescriptor hcd: htd.getFamilies()) {
       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
@@ -650,7 +661,7 @@ public class TestWALReplay {
     // Let us try to split and recover
     runWALSplit(this.conf);
     WAL wal2 = createWAL(this.conf);
-    HRegion region2 = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal2);
+    HRegion region2 = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal2);
     long seqid2 = region2.getOpenSeqNum();
     assertTrue(seqid + result.size() < seqid2);
 
@@ -690,10 +701,10 @@ public class TestWALReplay {
     final TableName tableName =
         TableName.valueOf("testReplayEditsAfterAbortingFlush");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseWALRootDir, tableName);
     deleteDir(basedir);
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
-    HRegion region3 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd);
+    HRegion region3 = HRegion.createHRegion(hri, hbaseWALRootDir, this.conf, htd);
     region3.close();
     region3.getWAL().close();
     // Write countPerFamily edits into the three families. Do a flush on one
@@ -707,7 +718,7 @@ public class TestWALReplay {
     customConf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY,
         CustomStoreFlusher.class.getName());
     HRegion region =
-      HRegion.openHRegion(this.hbaseRootDir, hri, htd, wal, customConf, rsServices, null);
+      HRegion.openHRegion(this.hbaseWALRootDir, hri, htd, wal, customConf, rsServices, null);
     int writtenRowCount = 10;
     List<HColumnDescriptor> families = new ArrayList<HColumnDescriptor>(
         htd.getFamilies());
@@ -761,7 +772,7 @@ public class TestWALReplay {
     WAL wal2 = createWAL(this.conf);
     Mockito.doReturn(false).when(rsServices).isAborted();
     HRegion region2 =
-      HRegion.openHRegion(this.hbaseRootDir, hri, htd, wal2, this.conf, rsServices, null);
+      HRegion.openHRegion(this.hbaseWALRootDir, hri, htd, wal2, this.conf, rsServices, null);
     scanner = region2.getScanner(new Scan());
     assertEquals(writtenRowCount, getScannedCount(scanner));
   }
@@ -791,12 +802,12 @@ public class TestWALReplay {
         TableName.valueOf("testReplayEditsWrittenIntoWAL");
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(hbaseWALRootDir, tableName);
     deleteDir(basedir);
 
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region2 = HRegion.createHRegion(hri,
-            hbaseRootDir, this.conf, htd);
+            hbaseWALRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
     final WAL wal = createWAL(this.conf);
     final byte[] rowName = tableName.getName();
@@ -890,7 +901,7 @@ public class TestWALReplay {
     final TableName tableName = TableName.valueOf(currentTest.getMethodName());
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
     final Path basedir =
-        FSUtils.getTableDir(this.hbaseRootDir, tableName);
+        FSUtils.getTableDir(this.hbaseWALRootDir, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
@@ -899,7 +910,7 @@ public class TestWALReplay {
     // Mock the WAL
     MockWAL wal = createMockWAL();
 
-    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
     for (HColumnDescriptor hcd : htd.getFamilies()) {
       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
     }
@@ -920,10 +931,10 @@ public class TestWALReplay {
     FileStatus[] listStatus = wal.getFiles();
     assertNotNull(listStatus);
     assertTrue(listStatus.length > 0);
-    WALSplitter.splitLogFile(hbaseRootDir, listStatus[0],
+    WALSplitter.splitLogFile(hbaseWALRootDir, listStatus[0],
         this.fs, this.conf, null, null, null, mode, wals);
     FileStatus[] listStatus1 = this.fs.listStatus(
-      new Path(FSUtils.getTableDir(hbaseRootDir, tableName), new Path(hri.getEncodedName(),
+      new Path(FSUtils.getTableDir(hbaseWALRootDir, tableName), new Path(hri.getEncodedName(),
           "recovered.edits")), new PathFilter() {
         @Override
         public boolean accept(Path p) {
@@ -951,17 +962,17 @@ public class TestWALReplay {
       IllegalAccessException {
     final TableName tableName = TableName.valueOf("testDatalossWhenInputError");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseWALRootDir, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
     final HTableDescriptor htd = createBasic1FamilyHTD(tableName);
-    HRegion region1 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
+    HRegion region1 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseWALRootDir, this.hbaseWALRootDir, this.conf, htd);
     Path regionDir = region1.getRegionFileSystem().getRegionDir();
     HBaseTestingUtility.closeRegionAndWAL(region1);
 
     WAL wal = createWAL(this.conf);
-    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
     for (HColumnDescriptor hcd : htd.getFamilies()) {
       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
     }
@@ -1031,12 +1042,12 @@ public class TestWALReplay {
     HRegion region2;
     try {
       // log replay should fail due to the IOException, otherwise we may lose data.
-      region2 = HRegion.openHRegion(conf, spyFs, hbaseRootDir, hri, htd, wal2);
+      region2 = HRegion.openHRegion(conf, spyFs, hbaseWALRootDir, hri, htd, wal2);
       assertEquals(result.size(), region2.get(g).size());
     } catch (IOException e) {
       assertEquals("read over limit", e.getMessage());
     }
-    region2 = HRegion.openHRegion(conf, fs, hbaseRootDir, hri, htd, wal2);
+    region2 = HRegion.openHRegion(conf, fs, hbaseWALRootDir, hri, htd, wal2);
     assertEquals(result.size(), region2.get(g).size());
   }
 
@@ -1047,11 +1058,11 @@ public class TestWALReplay {
     final TableName tableName = TableName.valueOf("testReplayEditsWrittenIntoWAL");
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(hbaseWALRootDir, tableName);
     deleteDir(basedir);
 
     final HTableDescriptor htd = createBasic1FamilyHTD(tableName);
-    HRegion region = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
+    HRegion region = HBaseTestingUtility.createRegionAndWAL(hri, hbaseWALRootDir, this.hbaseWALRootDir, this.conf, htd);
     HBaseTestingUtility.closeRegionAndWAL(region);
     final byte[] family = htd.getColumnFamilies()[0].getName();
     final byte[] rowName = tableName.getName();
@@ -1070,12 +1081,12 @@ public class TestWALReplay {
       first = fs.getFileStatus(smallFile);
       second = fs.getFileStatus(largeFile);
     }
-    WALSplitter.splitLogFile(hbaseRootDir, first, fs, conf, null, null, null,
+    WALSplitter.splitLogFile(hbaseWALRootDir, first, fs, conf, null, null, null,
       RecoveryMode.LOG_SPLITTING, wals);
-    WALSplitter.splitLogFile(hbaseRootDir, second, fs, conf, null, null, null,
+    WALSplitter.splitLogFile(hbaseWALRootDir, second, fs, conf, null, null, null,
       RecoveryMode.LOG_SPLITTING, wals);
     WAL wal = createWAL(this.conf);
-    region = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal);
+    region = HRegion.openHRegion(conf, this.fs, hbaseWALRootDir, hri, htd, wal);
     assertTrue(region.getOpenSeqNum() > mvcc.getWritePoint());
     assertEquals(2, region.get(new Get(rowName)).size());
   }
@@ -1093,9 +1104,9 @@ public class TestWALReplay {
   static class MockWAL extends FSHLog {
     boolean doCompleteCacheFlush = false;
 
-    public MockWAL(FileSystem fs, Path rootDir, String logName, Configuration conf)
+    public MockWAL(FileSystem fs, Path walRootDir, String logName, Configuration conf)
         throws IOException {
-      super(fs, rootDir, logName, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
+      super(fs, walRootDir, logName, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
     }
 
     @Override
@@ -1115,7 +1126,7 @@ public class TestWALReplay {
   }
 
   private MockWAL createMockWAL() throws IOException {
-    MockWAL wal = new MockWAL(fs, hbaseRootDir, logName, conf);
+    MockWAL wal = new MockWAL(fs, hbaseWALRootDir, logName, conf);
     // Set down maximum recovery so we dfsclient doesn't linger retrying something
     // long gone.
     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
@@ -1222,7 +1233,7 @@ public class TestWALReplay {
    */
   private Path runWALSplit(final Configuration c) throws IOException {
     List<Path> splits = WALSplitter.split(
-      hbaseRootDir, logDir, oldLogDir, FileSystem.get(c), c, wals);
+      hbaseWALRootDir, logDir, oldLogDir, FSUtils.getWALFileSystem(c), c, wals);
     // Split should generate only 1 file since there's only 1 region
     assertEquals("splits=" + splits, 1, splits.size());
     // Make sure the file exists
@@ -1237,7 +1248,7 @@ public class TestWALReplay {
    * @throws IOException
    */
   private WAL createWAL(final Configuration c) throws IOException {
-    FSHLog wal = new FSHLog(FileSystem.get(c), hbaseRootDir, logName, c);
+    FSHLog wal = new FSHLog(FSUtils.getWALFileSystem(c), hbaseWALRootDir, logName, c);
     // Set down maximum recovery so we dfsclient doesn't linger retrying something
     // long gone.
     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
index 96f4180..5fa9633 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
@@ -39,8 +39,9 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -393,6 +394,54 @@ public class TestFSUtils {
     verifyFileInDirWithStoragePolicy("1772");
   }
 
+  @Test
+  public void testSetWALRootDir() throws Exception {
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    Configuration conf = htu.getConfiguration();
+    Path p = new Path("file:///hbase/root");
+    FSUtils.setWALRootDir(conf, p);
+    assertEquals(p.toString(), conf.get(HFileSystem.HBASE_WAL_DIR));
+  }
+
+  @Test
+  public void testGetWALRootDir() throws IOException {
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    Configuration conf = htu.getConfiguration();
+    Path root = new Path("file:///hbase/root");
+    Path walRoot = new Path("file:///hbase/logroot");
+    FSUtils.setRootDir(conf, root);
+    assertEquals(FSUtils.getRootDir(conf), root);
+    assertEquals(FSUtils.getWALRootDir(conf), root);
+    FSUtils.setWALRootDir(conf, walRoot);
+    assertEquals(FSUtils.getWALRootDir(conf), walRoot);
+  }
+
+  @Test(expected=IllegalStateException.class)
+  public void testGetWALRootDirIllegalWALDir() throws IOException {
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    Configuration conf = htu.getConfiguration();
+    Path root = new Path("file:///hbase/root");
+    Path invalidWALDir = new Path("file:///hbase/root/logroot");
+    FSUtils.setRootDir(conf, root);
+    FSUtils.setWALRootDir(conf, invalidWALDir);
+    FSUtils.getWALRootDir(conf);
+  }
+
+  @Test
+  public void testRemoveWALRootPath() throws Exception {
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    Configuration conf = htu.getConfiguration();
+    FSUtils.setRootDir(conf, new Path("file:///user/hbase"));
+    Path testFile = new Path(FSUtils.getRootDir(conf), "test/testfile");
+    Path tmpFile = new Path("file:///test/testfile");
+    assertEquals(FSUtils.removeWALRootPath(testFile, conf), "test/testfile");
+    assertEquals(FSUtils.removeWALRootPath(tmpFile, conf), tmpFile.toString());
+    FSUtils.setWALRootDir(conf, new Path("file:///user/hbaseLogDir"));
+    assertEquals(FSUtils.removeWALRootPath(testFile, conf), testFile.toString());
+    Path logFile = new Path(FSUtils.getWALRootDir(conf), "test/testlog");
+    assertEquals(FSUtils.removeWALRootPath(logFile, conf), "test/testlog");
+  }
+
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.delete(name, true));

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
index 6b1ca03..8e8e73a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
@@ -101,7 +101,7 @@ public class IOTestProvider implements WALProvider {
       providerId = DEFAULT_PROVIDER_ID;
     }
     final String logPrefix = factory.factoryId + WAL_FILE_NAME_DELIMITER + providerId;
-    log = new IOTestWAL(FileSystem.get(conf), FSUtils.getRootDir(conf),
+    log = new IOTestWAL(FSUtils.getWALFileSystem(conf), FSUtils.getWALRootDir(conf),
         DefaultWALProvider.getWALDirectoryName(factory.factoryId),
         HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
         true, logPrefix, META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java
index e928a4d..fa88294 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java
@@ -66,6 +66,8 @@ public class TestDefaultWALProvider {
 
   protected static Configuration conf;
   protected static FileSystem fs;
+  protected static FileSystem walFs;
+  protected static Path walRootDir;
   protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   protected MultiVersionConcurrencyControl mvcc;
 
@@ -79,6 +81,7 @@ public class TestDefaultWALProvider {
     for (FileStatus dir : entries) {
       fs.delete(dir.getPath(), true);
     }
+    walFs.delete(walRootDir, true);
   }
 
   @After
@@ -104,13 +107,15 @@ public class TestDefaultWALProvider {
     TEST_UTIL.startMiniDFSCluster(3);
 
     // Set up a working space for our tests.
-    TEST_UTIL.createRootDir();
+    walRootDir = TEST_UTIL.createWALRootDir();
     conf = TEST_UTIL.getConfiguration();
-    fs = TEST_UTIL.getDFSCluster().getFileSystem();
+    fs = FSUtils.getRootDirFileSystem(conf);
+    walFs = FSUtils.getWALFileSystem(conf);
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
+    walFs.delete(walRootDir, true);
     TEST_UTIL.shutdownMiniCluster();
   }
 
@@ -121,13 +126,12 @@ public class TestDefaultWALProvider {
   @Test
   public void testGetServerNameFromWALDirectoryName() throws IOException {
     ServerName sn = ServerName.valueOf("hn", 450, 1398);
-    String hl = FSUtils.getRootDir(conf) + "/" +
+    String hl = walRootDir + "/" +
         DefaultWALProvider.getWALDirectoryName(sn.toString());
 
     // Must not throw exception
     assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf, null));
-    assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf,
-        FSUtils.getRootDir(conf).toUri().toString()));
+    assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf, walRootDir.toUri().toString()));
     assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf, ""));
     assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf, "                  "));
     assertNull(DefaultWALProvider.getServerNameFromWALDirectoryName(conf, hl));
@@ -136,7 +140,7 @@ public class TestDefaultWALProvider {
 
     final String wals = "/WALs/";
     ServerName parsed = DefaultWALProvider.getServerNameFromWALDirectoryName(conf,
-      FSUtils.getRootDir(conf).toUri().toString() + wals + sn +
+      walRootDir.toUri().toString() + wals + sn +
       "/localhost%2C32984%2C1343316388997.1343316390417");
     assertEquals("standard",  sn, parsed);
 
@@ -144,7 +148,7 @@ public class TestDefaultWALProvider {
     assertEquals("subdir", sn, parsed);
 
     parsed = DefaultWALProvider.getServerNameFromWALDirectoryName(conf,
-      FSUtils.getRootDir(conf).toUri().toString() + wals + sn +
+      walRootDir.toUri().toString() + wals + sn +
       "-splitting/localhost%3A57020.1340474893931");
     assertEquals("split", sn, parsed);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
index 878bb32..9b4a968 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
@@ -84,6 +84,7 @@ public class TestWALFactory {
   private static MiniDFSCluster cluster;
   protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   protected static Path hbaseDir;
+  protected static Path hbaseWALDir;
 
   protected FileSystem fs;
   protected Path dir;
@@ -142,6 +143,7 @@ public class TestWALFactory {
     cluster = TEST_UTIL.getDFSCluster();
 
     hbaseDir = TEST_UTIL.createRootDir();
+    hbaseWALDir = TEST_UTIL.createWALRootDir();
   }
 
   @AfterClass
@@ -164,12 +166,12 @@ public class TestWALFactory {
     final TableName tableName = TableName.valueOf(currentTest.getMethodName());
     final byte [] rowName = tableName.getName();
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1);
-    final Path logdir = new Path(hbaseDir,
+    final Path logdir = new Path(hbaseWALDir,
         DefaultWALProvider.getWALDirectoryName(currentTest.getMethodName()));
-    Path oldLogDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    Path oldLogDir = new Path(hbaseWALDir, HConstants.HREGION_OLDLOGDIR_NAME);
     final int howmany = 3;
     HRegionInfo[] infos = new HRegionInfo[3];
-    Path tabledir = FSUtils.getTableDir(hbaseDir, tableName);
+    Path tabledir = FSUtils.getTableDir(hbaseWALDir, tableName);
     fs.mkdirs(tabledir);
     for(int i = 0; i < howmany; i++) {
       infos[i] = new HRegionInfo(tableName,
@@ -203,7 +205,7 @@ public class TestWALFactory {
       }
     }
     wals.shutdown();
-    List<Path> splits = WALSplitter.split(hbaseDir, logdir, oldLogDir, fs, conf, wals);
+    List<Path> splits = WALSplitter.split(hbaseWALDir, logdir, oldLogDir, fs, conf, wals);
     verifySplits(splits, howmany);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
new file mode 100644
index 0000000..e4c0ae8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
@@ -0,0 +1,148 @@
+/**
+ * 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 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.TableName;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+@Category(MediumTests.class)
+public class TestWALRootDir {
+  private static final Log LOG = LogFactory.getLog(TestWALRootDir.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf;
+  private static FileSystem fs;
+  private static FileSystem walFs;
+  static final TableName tableName = TableName.valueOf("TestWALWALDir");
+  private static final byte [] rowName = Bytes.toBytes("row");
+  private static final byte [] family = Bytes.toBytes("column");
+  private static HTableDescriptor htd;
+  private static Path walRootDir;
+  private static Path rootDir;
+  private static WALFactory wals;
+
+  @Before
+  public void setUp() throws Exception {
+    cleanup();
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf = TEST_UTIL.getConfiguration();
+    TEST_UTIL.startMiniDFSCluster(1);
+    rootDir = TEST_UTIL.createRootDir();
+    walRootDir = TEST_UTIL.createWALRootDir();
+    fs = FSUtils.getRootDirFileSystem(conf);
+    walFs = FSUtils.getWALFileSystem(conf);
+    htd = new HTableDescriptor(tableName);
+    htd.addFamily(new HColumnDescriptor(family));
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    cleanup();
+    TEST_UTIL.shutdownMiniDFSCluster();
+  }
+
+  @Test
+  public void testWALRootDir() throws Exception {
+    HRegionInfo regionInfo = new HRegionInfo(tableName);
+    wals = new WALFactory(conf, null, "testWALRootDir");
+    WAL log = wals.getWAL(regionInfo.getEncodedNameAsBytes(), regionInfo.getTable().getNamespace());
+
+    assertEquals(1, getWALFiles(walFs, walRootDir).size());
+    byte [] value = Bytes.toBytes("value");
+    WALEdit edit = new WALEdit();
+    edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
+        System.currentTimeMillis(), value));
+    long txid = log.append(htd,regionInfo, getWalKey(System.currentTimeMillis(), regionInfo, 0), edit, true);
+    log.sync(txid);
+    assertEquals("Expect 1 log have been created", 1, getWALFiles(walFs, walRootDir).size());
+    log.rollWriter();
+    //Create 1 more WAL
+    assertEquals(2, getWALFiles(walFs, new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME)).size());
+    edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"),
+        System.currentTimeMillis(), value));
+    txid = log.append(htd, regionInfo, getWalKey(System.currentTimeMillis(), regionInfo, 1), edit, true);
+    log.sync(txid);
+    log.rollWriter();
+    log.shutdown();
+
+    assertEquals("Expect 3 logs in WALs dir", 3, getWALFiles(walFs, new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME)).size());
+  }
+
+  protected WALKey getWalKey(final long time, HRegionInfo hri, final long startPoint) {
+    return new WALKey(hri.getEncodedNameAsBytes(), tableName, time, new MultiVersionConcurrencyControl(startPoint));
+  }
+
+  private List<FileStatus> getWALFiles(FileSystem fs, Path dir)
+      throws IOException {
+    List<FileStatus> result = new ArrayList<FileStatus>();
+    LOG.debug("Scanning " + dir.toString() + " for WAL files");
+
+    FileStatus[] files = fs.listStatus(dir);
+    if (files == null) return Collections.emptyList();
+    for (FileStatus file : files) {
+      if (file.isDirectory()) {
+        // recurse into sub directories
+        result.addAll(getWALFiles(fs, file.getPath()));
+      } else {
+        String name = file.getPath().toString();
+        if (!name.startsWith(".")) {
+          result.add(file);
+        }
+      }
+    }
+    return result;
+  }
+
+  private static void cleanup() throws Exception{
+    walFs.delete(walRootDir, true);
+    fs.delete(rootDir, true);
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
index 4c77d25..3ae155c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -118,6 +118,7 @@ public class TestWALSplit {
   protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private Path HBASEDIR;
+  private Path HBASELOGDIR;
   private Path WALDIR;
   private Path OLDLOGDIR;
   private Path CORRUPTDIR;
@@ -180,8 +181,9 @@ public class TestWALSplit {
     LOG.info("Cleaning up cluster for new test.");
     fs = TEST_UTIL.getDFSCluster().getFileSystem();
     HBASEDIR = TEST_UTIL.createRootDir();
-    OLDLOGDIR = new Path(HBASEDIR, HConstants.HREGION_OLDLOGDIR_NAME);
-    CORRUPTDIR = new Path(HBASEDIR, HConstants.CORRUPT_DIR_NAME);
+    HBASELOGDIR = TEST_UTIL.createWALRootDir();
+    OLDLOGDIR = new Path(HBASELOGDIR, HConstants.HREGION_OLDLOGDIR_NAME);
+    CORRUPTDIR = new Path(HBASELOGDIR, HConstants.CORRUPT_DIR_NAME);
     TABLEDIR = FSUtils.getTableDir(HBASEDIR, TABLE_NAME);
     REGIONS.clear();
     Collections.addAll(REGIONS, "bbb", "ccc");
@@ -189,7 +191,7 @@ public class TestWALSplit {
     this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
         RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
     wals = new WALFactory(conf, null, name.getMethodName());
-    WALDIR = new Path(HBASEDIR, DefaultWALProvider.getWALDirectoryName(name.getMethodName()));
+    WALDIR = new Path(HBASELOGDIR, DefaultWALProvider.getWALDirectoryName(name.getMethodName()));
     //fs.mkdirs(WALDIR);
   }
 
@@ -205,6 +207,7 @@ public class TestWALSplit {
     } finally {
       wals = null;
       fs.delete(HBASEDIR, true);
+      fs.delete(HBASELOGDIR, true);
     }
   }
 
@@ -1111,7 +1114,7 @@ public class TestWALSplit {
     useDifferentDFSClient();
     WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
 
-    final Path corruptDir = new Path(FSUtils.getRootDir(conf), HConstants.CORRUPT_DIR_NAME);
+    final Path corruptDir = new Path(FSUtils.getWALRootDir(conf), HConstants.CORRUPT_DIR_NAME);
     assertEquals(1, fs.listStatus(corruptDir).length);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
index 8af1882..5a3dd87 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
@@ -346,7 +346,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
         }
         if (verify) {
           LOG.info("verifying written log entries.");
-          Path dir = new Path(FSUtils.getRootDir(getConf()),
+          Path dir = new Path(FSUtils.getWALRootDir(getConf()),
               DefaultWALProvider.getWALDirectoryName("wals"));
           long editCount = 0;
           FileStatus [] fsss = fs.listStatus(dir);


[07/50] [abbrv] hbase git commit: HBASE-19551 hbck -boundaries doesn't work correctly (Toshihiro Suzuki)

Posted by ap...@apache.org.
HBASE-19551 hbck -boundaries doesn't work correctly (Toshihiro Suzuki)


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

Branch: refs/heads/branch-1.3
Commit: 217ee60fb4538392d34cccc5e581725e79dc53ed
Parents: 5d0d3aa
Author: tedyu <yu...@gmail.com>
Authored: Sat Dec 30 23:19:13 2017 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:16 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/HBaseFsck.java | 22 +++---
 .../apache/hadoop/hbase/util/TestHBaseFsck.java | 71 ++++++++++++++++++--
 .../hadoop/hbase/util/hbck/HbckTestingUtil.java |  8 +++
 3 files changed, 85 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/217ee60f/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index eae8a64..620a66d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -804,15 +804,17 @@ public class HBaseFsck extends Configured implements Closeable {
             for (FileStatus storeFile : storeFiles) {
               HFile.Reader reader = HFile.createReader(fs, storeFile.getPath(), new CacheConfig(
                   getConf()), getConf());
-              if ((reader.getFirstKey() != null)
-                  && ((storeFirstKey == null) || (comparator.compare(storeFirstKey,
-                      reader.getFirstKey()) > 0))) {
-                storeFirstKey = reader.getFirstKey();
+              if (reader.getFirstKey() != null) {
+                byte[] firstKey = keyOnly(reader.getFirstKey());
+                if (storeFirstKey == null || comparator.compare(storeFirstKey, firstKey) > 0) {
+                  storeFirstKey = firstKey;
+                }
               }
-              if ((reader.getLastKey() != null)
-                  && ((storeLastKey == null) || (comparator.compare(storeLastKey,
-                      reader.getLastKey())) < 0)) {
-                storeLastKey = reader.getLastKey();
+              if (reader.getLastKey() != null) {
+                byte[] lastKey = keyOnly(reader.getLastKey());
+                if (storeLastKey == null || comparator.compare(storeLastKey, lastKey) < 0) {
+                  storeLastKey = lastKey;
+                }
               }
               reader.close();
             }
@@ -820,8 +822,8 @@ public class HBaseFsck extends Configured implements Closeable {
         }
         currentRegionBoundariesInformation.metaFirstKey = regionInfo.getStartKey();
         currentRegionBoundariesInformation.metaLastKey = regionInfo.getEndKey();
-        currentRegionBoundariesInformation.storesFirstKey = keyOnly(storeFirstKey);
-        currentRegionBoundariesInformation.storesLastKey = keyOnly(storeLastKey);
+        currentRegionBoundariesInformation.storesFirstKey = storeFirstKey;
+        currentRegionBoundariesInformation.storesLastKey = storeLastKey;
         if (currentRegionBoundariesInformation.metaFirstKey.length == 0)
           currentRegionBoundariesInformation.metaFirstKey = null;
         if (currentRegionBoundariesInformation.metaLastKey.length == 0)

http://git-wip-us.apache.org/repos/asf/hbase/blob/217ee60f/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
index df06970..c316c98 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.util;
 
 import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
 import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertNoErrors;
+import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.checkRegionBoundaries;
 import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -2998,15 +2999,73 @@ public class TestHBaseFsck {
 
   @Test (timeout = 180000)
   public void testRegionBoundariesCheck() throws Exception {
-    HBaseFsck hbck = doFsck(conf, false);
+    TableName tableName = TableName.valueOf("testRegionBoundariesCheck");
+
+    // setup a table
+    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
+    desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
+    createTable(TEST_UTIL, desc, SPLITS);
+
+    Table table = connection.getTable(tableName, tableExecutorService);
+    List<Put> puts = new ArrayList<>();
+
+    // for the first region
+    puts.add(new Put(Bytes.toBytes("0")).addColumn(FAM, Bytes.toBytes("col"),
+      Bytes.toBytes("val")));
+    puts.add(new Put(Bytes.toBytes("999")).addColumn(FAM, Bytes.toBytes("col"),
+      Bytes.toBytes("val")));
+
+    // for the second region
+    puts.add(new Put(Bytes.toBytes("AA")).addColumn(FAM, Bytes.toBytes("col"),
+      Bytes.toBytes("val")));
+    puts.add(new Put(Bytes.toBytes("AZ")).addColumn(FAM, Bytes.toBytes("col"),
+      Bytes.toBytes("val")));
+
+    table.put(puts);
+
+    // to guarantee all data flushed, disable and enable the table
+    admin.disableTable(tableName);
+    admin.enableTable(tableName);
+
+    // check region boundaries before moving an HFile
+    HBaseFsck hbck = checkRegionBoundaries(conf);
     assertNoErrors(hbck); // no errors
-    try {
-      hbck.checkRegionBoundaries();
-    } catch (IllegalArgumentException e) {
-      if (e.getMessage().endsWith("not a valid DFS filename.")) {
-        fail("Table directory path is not valid." + e.getMessage());
+
+    // move an HFile in the second region to the first region directory
+    admin.disableTable(tableName);
+
+    List<HRegionInfo> tableRegions = admin.getTableRegions(tableName);
+    HRegionInfo firstRegion = tableRegions.get(0);
+    HRegionInfo secondRegion = tableRegions.get(1);
+
+    FileSystem fs = FileSystem.get(conf);
+    Path tableDir= FSUtils.getTableDir(FSUtils.getRootDir(conf), tableName);
+    Path firstRegionFamDir = new Path(new Path(tableDir, firstRegion.getEncodedName()), FAM_STR);
+    Path hfileInFirstRegion = getHFilePath(fs, firstRegionFamDir);
+    Path secondRegionFamDir = new Path(new Path(tableDir, secondRegion.getEncodedName()), FAM_STR);
+    Path hfileInSecondRegion = getHFilePath(fs, secondRegionFamDir);
+
+    // rename HFile names (to "0" and "1") in order to guarantee the same file iteration order of
+    // fs.listStatus()
+    fs.rename(hfileInFirstRegion, new Path(firstRegionFamDir, "0"));
+    fs.rename(hfileInSecondRegion, new Path(firstRegionFamDir, "1"));
+
+    admin.enableTable(tableName);
+
+    // check region boundaries after moving an HFile
+    hbck = checkRegionBoundaries(conf);
+    assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.BOUNDARIES_ERROR });
+  }
+
+  private static Path getHFilePath(FileSystem fs, Path famDir) throws IOException {
+    FileStatus[] hfFss = fs.listStatus(famDir);
+    for (FileStatus hfs : hfFss) {
+      if (!hfs.isDirectory()) {
+        return hfs.getPath();
       }
     }
+    return null;
   }
 
   @org.junit.Rule

http://git-wip-us.apache.org/repos/asf/hbase/blob/217ee60f/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
index 09f6978..a384cd6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
@@ -87,6 +87,14 @@ public class HbckTestingUtil {
     return hbck;
   }
 
+  public static HBaseFsck checkRegionBoundaries(Configuration conf) throws Exception {
+    HBaseFsck hbck = new HBaseFsck(conf, exec);
+    hbck.connect();
+    hbck.checkRegionBoundaries();
+    hbck.close();
+    return hbck;
+  }
+
   public static boolean inconsistencyFound(HBaseFsck fsck) throws Exception {
     List<ERROR_CODE> errs = fsck.getErrors().getErrorList();
     return (errs != null && !errs.isEmpty());


[06/50] [abbrv] hbase git commit: HBASE-17731 Fractional latency reporting in MultiThreadedAction

Posted by ap...@apache.org.
HBASE-17731 Fractional latency reporting in MultiThreadedAction


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

Branch: refs/heads/branch-1.3
Commit: 286ade8155e5c198feef83bbd5fcc4f00a3d7796
Parents: 109219d
Author: Andrew Purtell <ap...@apache.org>
Authored: Thu Mar 9 16:54:23 2017 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:16 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/MultiThreadedAction.java    | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/286ade81/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
index 91b6d3b..0d25a68 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
@@ -218,11 +218,13 @@ public abstract class MultiThreadedAction {
               + ", time="
               + formatTime(time)
               + ((numKeys > 0 && time > 0) ? (" Overall: [" + "keys/s= "
-                  + numKeys * 1000 / time + ", latency=" + totalOpTime
-                  / numKeys + " ms]") : "")
+                  + numKeys * 1000 / time + ", latency="
+                  + String.format("%.2f", (double)totalOpTime / (double)numKeys)
+                  + " ms]") : "")
               + ((numKeysDelta > 0) ? (" Current: [" + "keys/s="
                   + numKeysDelta * 1000 / REPORTING_INTERVAL_MS + ", latency="
-                  + totalOpTimeDelta / numKeysDelta + " ms]") : "")
+                  + String.format("%.2f", (double)totalOpTimeDelta / (double)numKeysDelta)
+                  + " ms]") : "")
               + progressInfo());
 
           if (streamingCounters) {


[05/50] [abbrv] hbase git commit: HBASE-17924 Consider sorting the row order when processing multi() ops before taking rowlocks (Allan Yang)

Posted by ap...@apache.org.
HBASE-17924 Consider sorting the row order when processing multi() ops before taking rowlocks
 (Allan Yang)

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java


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

Branch: refs/heads/branch-1.3
Commit: 4e4756e68887e3a1d77e4483f52f431b441611a5
Parents: 25135dd
Author: Andrew Purtell <ap...@apache.org>
Authored: Mon May 8 16:23:13 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:16 2018 -0800

----------------------------------------------------------------------
 .../hbase/regionserver/RSRpcServices.java       | 21 ++++++++++++++++++--
 .../apache/hadoop/hbase/wal/WALSplitter.java    | 21 +++++++++++++++++++-
 2 files changed, 39 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4e4756e6/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 151a864..cd7a4c7 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
@@ -34,6 +34,7 @@ import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -847,6 +848,14 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     long before = EnvironmentEdgeManager.currentTime();
     boolean batchContainsPuts = false, batchContainsDelete = false;
     try {
+      /** HBASE-17924
+       * mutationActionMap is a map to map the relation between mutations and actions
+       * since mutation array may have been reoredered.In order to return the right
+       * result or exception to the corresponding actions, We need to know which action
+       * is the mutation belong to. We can't sort ClientProtos.Action array, since they
+       * are bonded to cellscanners.
+       */
+      Map<Mutation, ClientProtos.Action> mutationActionMap = new HashMap<Mutation, ClientProtos.Action>();
       int i = 0;
       for (ClientProtos.Action action: mutations) {
         MutationProto m = action.getMutation();
@@ -858,6 +867,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           mutation = ProtobufUtil.toDelete(m, cells);
           batchContainsDelete = true;
         }
+        mutationActionMap.put(mutation, action);
         mArray[i++] = mutation;
         quota.addMutation(mutation);
       }
@@ -865,11 +875,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       if (!region.getRegionInfo().isMetaTable()) {
         regionServer.cacheFlusher.reclaimMemStoreMemory();
       }
-
+      // HBASE-17924
+      // sort to improve lock efficiency
+      Arrays.sort(mArray);
       OperationStatus[] codes = region.batchMutate(mArray, HConstants.NO_NONCE,
         HConstants.NO_NONCE);
       for (i = 0; i < codes.length; i++) {
-        int index = mutations.get(i).getIndex();
+        Mutation currentMutation = mArray[i];
+        ClientProtos.Action currentAction = mutationActionMap.get(currentMutation);
+        int index = currentAction.getIndex();
         Exception e = null;
         switch (codes[i].getOperationStatusCode()) {
           case BAD_FAMILY:
@@ -1927,6 +1941,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           walEntries.add(walEntry);
         }
         if(edits!=null && !edits.isEmpty()) {
+          // HBASE-17924
+          // sort to improve lock efficiency
+          Collections.sort(edits);
           long replaySeqId = (entry.getKey().hasOrigSequenceNumber()) ?
             entry.getKey().getOrigSequenceNumber() : entry.getKey().getLogSequenceNumber();
           OperationStatus[] result = doReplayBatchOp(region, edits, replaySeqId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e4756e6/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index eed9a8b..cc065e5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -2272,7 +2272,7 @@ public class WALSplitter {
   }
 
   /** A struct used by getMutationsFromWALEntry */
-  public static class MutationReplay {
+  public static class MutationReplay implements Comparable<MutationReplay> {
     public MutationReplay(MutationType type, Mutation mutation, long nonceGroup, long nonce) {
       this.type = type;
       this.mutation = mutation;
@@ -2288,6 +2288,25 @@ public class WALSplitter {
     public final Mutation mutation;
     public final long nonceGroup;
     public final long nonce;
+
+    @Override
+    public int compareTo(final MutationReplay d) {
+      return this.mutation.compareTo(d.mutation);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if(!(obj instanceof MutationReplay)) {
+        return false;
+      } else {
+        return this.compareTo((MutationReplay)obj) == 0;
+      }
+    }
+
+    @Override
+    public int hashCode() {
+      return this.mutation.hashCode();
+    }
   }
 
   /**


[48/50] [abbrv] hbase git commit: HBASE-21275 - Disable TRACE HTTP method for thrift http server (branch 1 only)

Posted by ap...@apache.org.
HBASE-21275 - Disable TRACE HTTP method for thrift http server (branch 1 only)

Conflicts:
	hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java


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

Branch: refs/heads/branch-1.3
Commit: 82f187efba6e476cd1b88bc6ae8b238e4c670288
Parents: 10c4f59
Author: Wellington Chevreuil <we...@ChevreuilWellington-MBP15.local>
Authored: Wed Oct 17 12:11:41 2018 +0100
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Dec 13 10:22:39 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/thrift/ThriftServerRunner.java | 10 ++-
 .../hbase/thrift/TestThriftHttpServer.java      | 66 ++++++++++++++++++--
 2 files changed, 69 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/82f187ef/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
index e13b148..76c033d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
@@ -101,6 +101,7 @@ import org.apache.hadoop.hbase.thrift.generated.TScan;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ConnectionCache;
 import org.apache.hadoop.hbase.util.DNS;
+import org.apache.hadoop.hbase.util.HttpServerUtil;
 import org.apache.hadoop.hbase.util.Strings;
 import org.apache.hadoop.security.SaslRpcServer.SaslGssCallbackHandler;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -128,6 +129,7 @@ import org.mortbay.jetty.Server;
 import org.mortbay.jetty.nio.SelectChannelConnector;
 import org.mortbay.jetty.servlet.Context;
 import org.mortbay.jetty.servlet.ServletHolder;
+import org.mortbay.jetty.webapp.WebAppContext;
 import org.mortbay.thread.QueuedThreadPool;
 
 import com.google.common.base.Joiner;
@@ -203,6 +205,9 @@ public class ThriftServerRunner implements Runnable {
   private final boolean securityEnabled;
   private final boolean doAsEnabled;
 
+  static String THRIFT_HTTP_ALLOW_OPTIONS_METHOD = "hbase.thrift.http.allow.options.method";
+  private static boolean THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT = false;
+
   /** An enum of server implementation selections */
   enum ImplType {
     HS_HA("hsha", true, THsHaServer.class, true),
@@ -410,11 +415,14 @@ public class ThriftServerRunner implements Runnable {
 
     httpServer = new Server();
     // Context handler
-    Context context = new Context(httpServer, "/", Context.SESSIONS);
+    Context context = new WebAppContext();
     context.setContextPath("/");
+    context.setResourceBase("hbase-webapps/");
     String httpPath = "/*";
     httpServer.setHandler(context);
     context.addServlet(new ServletHolder(thriftHttpServlet), httpPath);
+    HttpServerUtil.constrainHttpMethods(context,
+        conf.getBoolean(THRIFT_HTTP_ALLOW_OPTIONS_METHOD, THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT));
 
     // set up Jetty and run the embedded server
     Connector connector = new SelectChannelConnector();

http://git-wip-us.apache.org/repos/asf/hbase/blob/82f187ef/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
index cf14e87..b21de38 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
@@ -18,10 +18,13 @@
  */
 package org.apache.hadoop.hbase.thrift;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 
+import java.net.HttpURLConnection;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -149,6 +152,46 @@ public class TestThriftHttpServer {
     runThriftServer(0);
   }
 
+  @Test
+  public void testThriftServerHttpTraceForbiddenWhenOptionsDisabled() throws Exception {
+    // HTTP TRACE method should be disabled for security
+    // See https://www.owasp.org/index.php/Cross_Site_Tracing
+    checkHttpMethods("TRACE", HttpURLConnection.HTTP_FORBIDDEN);
+  }
+
+  @Test
+  public void testThriftServerHttpTraceForbiddenWhenOptionsEnabled() throws Exception {
+    // HTTP TRACE method should be disabled for security
+    // See https://www.owasp.org/index.php/Cross_Site_Tracing
+    TEST_UTIL.getConfiguration().setBoolean(ThriftServerRunner.THRIFT_HTTP_ALLOW_OPTIONS_METHOD,
+        true);
+    checkHttpMethods("TRACE", HttpURLConnection.HTTP_FORBIDDEN);
+  }
+
+  @Test
+  public void testThriftServerHttpOptionsForbiddenWhenOptionsDisabled() throws Exception {
+    // HTTP OPTIONS method should be disabled by default, so we make sure
+    // hbase.thrift.http.allow.options.method is not set anywhere in the config
+    TEST_UTIL.getConfiguration().unset(ThriftServerRunner.THRIFT_HTTP_ALLOW_OPTIONS_METHOD);
+    checkHttpMethods("OPTIONS", HttpURLConnection.HTTP_FORBIDDEN);
+  }
+
+  @Test
+  public void testThriftServerHttpOptionsOkWhenOptionsEnabled() throws Exception {
+    TEST_UTIL.getConfiguration().setBoolean(ThriftServerRunner.THRIFT_HTTP_ALLOW_OPTIONS_METHOD,
+        true);
+    checkHttpMethods("OPTIONS", HttpURLConnection.HTTP_OK);
+  }
+
+  private void waitThriftServerStartup() throws Exception{
+    // wait up to 10s for the server to start
+    for (int i = 0; i < 100
+        && ( thriftServer.serverRunner == null ||  thriftServer.serverRunner.httpServer ==
+        null); i++) {
+      Thread.sleep(100);
+    }
+  }
+
   private void runThriftServer(int customHeaderSize) throws Exception {
     List<String> args = new ArrayList<String>();
     port = HBaseTestingUtility.randomFreePort();
@@ -159,12 +202,7 @@ public class TestThriftHttpServer {
     thriftServer = new ThriftServer(TEST_UTIL.getConfiguration());
     startHttpServerThread(args.toArray(new String[args.size()]));
 
-    // wait up to 10s for the server to start
-    for (int i = 0; i < 100
-        && ( thriftServer.serverRunner == null ||  thriftServer.serverRunner.httpServer ==
-        null); i++) {
-      Thread.sleep(100);
-    }
+    waitThriftServerStartup();
 
     try {
       talkToThriftServer(customHeaderSize);
@@ -223,4 +261,20 @@ public class TestThriftHttpServer {
       throw new Exception(httpServerException);
     }
   }
+
+  private void checkHttpMethods(String httpRequestMethod,
+      int httpExpectedResponse) throws Exception {
+    port = HBaseTestingUtility.randomFreePort();
+    thriftServer = new ThriftServer(TEST_UTIL.getConfiguration());
+    try {
+      startHttpServerThread(new String[] { "-port", String.valueOf(port), "start" });
+      waitThriftServerStartup();
+      final URL url = new URL("http://"+ HConstants.LOCALHOST + ":" + port);
+      final HttpURLConnection httpConn = (HttpURLConnection) url.openConnection();
+      httpConn.setRequestMethod(httpRequestMethod);
+      assertEquals(httpExpectedResponse, httpConn.getResponseCode());
+    } finally {
+      stopHttpServerThread();
+    }
+  }
 }


[46/50] [abbrv] hbase git commit: HBASE-18451 PeriodicMemstoreFlusher should inspect the queue before adding a delayed flush request

Posted by ap...@apache.org.
HBASE-18451 PeriodicMemstoreFlusher should inspect the queue before adding a delayed flush request

Signed-off-by: Andrew Purtell <ap...@apache.org>

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java


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

Branch: refs/heads/branch-1.3
Commit: 785e21fe545da33811a50e0718d7cfeb7dc74df7
Parents: a4baeeb
Author: xcang <xc...@salesforce.com>
Authored: Sun Sep 23 23:42:57 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 19:25:50 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/FlushRequester.java       |  6 ++++--
 .../apache/hadoop/hbase/regionserver/HRegionServer.java | 12 +++++++-----
 .../hadoop/hbase/regionserver/MemStoreFlusher.java      | 12 ++++++++----
 .../hbase/regionserver/TestHeapMemoryManager.java       |  7 ++++---
 .../hadoop/hbase/regionserver/wal/TestWALReplay.java    | 11 +++++------
 5 files changed, 28 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/785e21fe/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java
index c7e155a..243546c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java
@@ -32,8 +32,9 @@ public interface FlushRequester {
    * @param region the Region requesting the cache flush
    * @param forceFlushAllStores whether we want to flush all stores. e.g., when request from log
    *          rolling.
+   * @return true if our region is added into the queue, false otherwise
    */
-  void requestFlush(Region region, boolean forceFlushAllStores);
+  boolean requestFlush(Region region, boolean forceFlushAllStores);
 
   /**
    * Tell the listener the cache needs to be flushed after a delay
@@ -42,8 +43,9 @@ public interface FlushRequester {
    * @param delay after how much time should the flush happen
    * @param forceFlushAllStores whether we want to flush all stores. e.g., when request from log
    *          rolling.
+   * @return true if our region is added into the queue, false otherwise
    */
-  void requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores);
+  boolean requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores);
 
   /**
    * Register a FlushRequestListener

http://git-wip-us.apache.org/repos/asf/hbase/blob/785e21fe/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 21f269e..bbf488a 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
@@ -28,6 +28,7 @@ import java.net.BindException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
+import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -1635,17 +1636,18 @@ public class HRegionServer extends HasThread implements
       final StringBuffer whyFlush = new StringBuffer();
       for (Region r : this.server.onlineRegions.values()) {
         if (r == null) continue;
-        if (((HRegion)r).shouldFlush(whyFlush)) {
+        if (((HRegion) r).shouldFlush(whyFlush)) {
           FlushRequester requester = server.getFlushRequester();
           if (requester != null) {
             long randomDelay = (long) RandomUtils.nextInt(rangeOfDelay) + MIN_DELAY_TIME;
-            LOG.info(getName() + " requesting flush of " +
-              r.getRegionInfo().getRegionNameAsString() + " because " +
-              whyFlush.toString() + " after random delay " + randomDelay + "ms");
             //Throttle the flushes by putting a delay. If we don't throttle, and there
             //is a balanced write-load on the regions in a table, we might end up
             //overwhelming the filesystem with too many flushes at once.
-            requester.requestDelayedFlush(r, randomDelay, false);
+            if (requester.requestDelayedFlush(r, randomDelay, false)) {
+              LOG.info(getName() + " requesting flush of " +
+                  r.getRegionInfo().getRegionNameAsString() + " because " +
+                  whyFlush.toString() + " after random delay " + randomDelay + "ms");
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/785e21fe/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
index b4adea6..80458bc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
@@ -360,8 +360,7 @@ class MemStoreFlusher implements FlushRequester {
   }
 
   @Override
-  public void requestFlush(Region r, boolean forceFlushAllStores) {
-    ((HRegion)r).incrementFlushesQueuedCount();
+  public boolean requestFlush(Region r, boolean forceFlushAllStores) {
     synchronized (regionsInQueue) {
       if (!regionsInQueue.containsKey(r)) {
         // This entry has no delay so it will be added at the top of the flush
@@ -369,13 +368,15 @@ class MemStoreFlusher implements FlushRequester {
         FlushRegionEntry fqe = new FlushRegionEntry(r, forceFlushAllStores);
         this.regionsInQueue.put(r, fqe);
         this.flushQueue.add(fqe);
+        ((HRegion)r).incrementFlushesQueuedCount();
+        return true;
       }
+      return false;
     }
   }
 
   @Override
-  public void requestDelayedFlush(Region r, long delay, boolean forceFlushAllStores) {
-    ((HRegion)r).incrementFlushesQueuedCount();
+  public boolean requestDelayedFlush(Region r, long delay, boolean forceFlushAllStores) {
     synchronized (regionsInQueue) {
       if (!regionsInQueue.containsKey(r)) {
         // This entry has some delay
@@ -383,7 +384,10 @@ class MemStoreFlusher implements FlushRequester {
         fqe.requeue(delay);
         this.regionsInQueue.put(r, fqe);
         this.flushQueue.add(fqe);
+        ((HRegion)r).incrementFlushesQueuedCount();
+        return true;
       }
+      return false;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/785e21fe/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index 0e72d0d..ddc5558 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -633,13 +633,14 @@ public class TestHeapMemoryManager {
     }
 
     @Override
-    public void requestFlush(Region region, boolean forceFlushAllStores) {
+    public boolean requestFlush(Region region, boolean forceFlushAllStores) {
       this.listener.flushRequested(flushType, region);
+      return true;
     }
 
     @Override
-    public void requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores) {
-
+    public boolean requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores) {
+      return true;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/785e21fe/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
index 5240f2a..8758c98 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
@@ -1136,21 +1136,20 @@ public class TestWALReplay {
   // Flusher used in this test.  Keep count of how often we are called and
   // actually run the flush inside here.
   class TestFlusher implements FlushRequester {
-    private HRegion r;
 
     @Override
-    public void requestFlush(Region region, boolean force) {
+    public boolean requestFlush(Region region, boolean force) {
       try {
-        r.flush(force);
+        region.flush(force);
+        return true;
       } catch (IOException e) {
         throw new RuntimeException("Exception flushing", e);
       }
     }
 
     @Override
-    public void requestDelayedFlush(Region region, long when, boolean forceFlushAllStores) {
-      // TODO Auto-generated method stub
-
+    public boolean requestDelayedFlush(Region region, long when, boolean forceFlushAllStores) {
+      return true;
     }
 
     @Override


[04/50] [abbrv] hbase git commit: HBASE-18058 Zookeeper retry sleep time should have an upper limit (Allan Yang)

Posted by ap...@apache.org.
HBASE-18058 Zookeeper retry sleep time should have an upper limit (Allan Yang)


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

Branch: refs/heads/branch-1.3
Commit: ed4f7d1b1b4497caf859da6119c2940fdfaba9a9
Parents: 4e4756e
Author: tedyu <yu...@gmail.com>
Authored: Fri May 19 10:58:38 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:16 2018 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java  | 8 ++++----
 .../main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java  | 3 ++-
 hbase-common/src/main/resources/hbase-default.xml            | 7 +++++++
 3 files changed, 13 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ed4f7d1b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
index e74aeb4..f799148 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
@@ -98,20 +98,20 @@ public class RecoverableZooKeeper {
   private static final int ID_LENGTH_SIZE =  Bytes.SIZEOF_INT;
 
   public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
-      Watcher watcher, int maxRetries, int retryIntervalMillis)
+      Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime)
   throws IOException {
-    this(quorumServers, sessionTimeout, watcher, maxRetries, retryIntervalMillis,
+    this(quorumServers, sessionTimeout, watcher, maxRetries, retryIntervalMillis, maxSleepTime,
         null);
   }
 
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
       justification="None. Its always been this way.")
   public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
-      Watcher watcher, int maxRetries, int retryIntervalMillis, String identifier)
+      Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime, String identifier)
   throws IOException {
     // TODO: Add support for zk 'chroot'; we don't add it to the quorumServers String as we should.
     this.retryCounterFactory =
-      new RetryCounterFactory(maxRetries+1, retryIntervalMillis);
+      new RetryCounterFactory(maxRetries+1, retryIntervalMillis, maxSleepTime);
 
     if (identifier == null || identifier.length() == 0) {
       // the identifier = processID@hostName

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed4f7d1b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 3cbc317..4f4b2eb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -133,10 +133,11 @@ public class ZKUtil {
     int retry = conf.getInt("zookeeper.recovery.retry", 3);
     int retryIntervalMillis =
       conf.getInt("zookeeper.recovery.retry.intervalmill", 1000);
+    int maxSleepTime = conf.getInt("zookeeper.recovery.retry.maxsleeptime", 60000);
     zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout",
         1000);
     return new RecoverableZooKeeper(ensemble, timeout, watcher,
-        retry, retryIntervalMillis, identifier);
+        retry, retryIntervalMillis, maxSleepTime, identifier);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed4f7d1b/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index e28f355..88e57d7 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -103,6 +103,13 @@ possible configurations would overwhelm and obscure the important.
     and running -->
 
   <property>
+    <name>zookeeper.recovery.retry.maxsleeptime</name>
+    <value>60000</value>
+    <description>Max sleep time before retry zookeeper operations in milliseconds,
+    a max time is needed here so that sleep time won't grow unboundedly
+    </description>
+  </property>
+  <property>
     <name>hbase.local.dir</name>
     <value>${hbase.tmp.dir}/local/</value>
     <description>Directory on the local filesystem to be used


[45/50] [abbrv] hbase git commit: HBASE-21212 Wrong flush time when update flush metric

Posted by ap...@apache.org.
HBASE-21212 Wrong flush time when update flush metric


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

Branch: refs/heads/branch-1.3
Commit: a4baeebdfc16878011633aa548b90dd53afddc37
Parents: 09069df
Author: Allan Yang <al...@apache.org>
Authored: Wed Sep 26 19:40:43 2018 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 19:25:46 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/regionserver/HRegion.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a4baeebd/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
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 d41e7cb..71785fd 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
@@ -2639,7 +2639,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     status.setStatus(msg);
 
     if (rsServices != null && rsServices.getMetrics() != null) {
-      rsServices.getMetrics().updateFlush(time - startTime,
+      rsServices.getMetrics().updateFlush(time,
         totalFlushableSizeOfFlushableStores, flushedOutputFileSize);
     }
 


[39/50] [abbrv] hbase git commit: HBASE-17437 Support specifying a WAL directory outside of the root directory (Yishan Yang and Zach York) HBASE-17588 Remove unused imports brought in by HBASE-17437 (Zach York)

Posted by ap...@apache.org.
HBASE-17437 Support specifying a WAL directory outside of the root directory (Yishan Yang and Zach York)
HBASE-17588 Remove unused imports brought in by HBASE-17437 (Zach York)

Signed-off-by: Enis Soztutar <en...@apache.org>

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java


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

Branch: refs/heads/branch-1.3
Commit: 979438d29e49a6a1510d4b82b4c90652bd354942
Parents: 9e39a20
Author: Zach York <zy...@amazon.com>
Authored: Wed Jan 11 12:49:20 2017 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 19:01:12 2018 -0800

----------------------------------------------------------------------
 .../src/main/resources/hbase-default.xml        |   7 +
 .../procedure2/store/wal/WALProcedureStore.java |  14 +-
 .../procedure2/ProcedureTestingUtility.java     |   4 +-
 ...ProcedureWALLoaderPerformanceEvaluation.java |   4 +-
 .../wal/ProcedureWALPerformanceEvaluation.java  |   4 +-
 .../org/apache/hadoop/hbase/fs/HFileSystem.java |  12 +-
 .../org/apache/hadoop/hbase/io/WALLink.java     |  10 +-
 .../hadoop/hbase/master/AssignmentManager.java  |  10 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   6 +-
 .../hadoop/hbase/master/MasterFileSystem.java   | 138 ++++++++++-------
 .../hadoop/hbase/master/SplitLogManager.java    |   2 +-
 .../hadoop/hbase/regionserver/HRegion.java      |   2 +-
 .../hbase/regionserver/HRegionServer.java       |  47 ++++--
 .../hbase/regionserver/SplitLogWorker.java      |   8 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |   8 +-
 .../regionserver/ReplicationSource.java         |   6 +-
 .../regionserver/ReplicationSyncUp.java         |  10 +-
 .../org/apache/hadoop/hbase/util/FSUtils.java   |  62 ++++++--
 .../hadoop/hbase/wal/DefaultWALProvider.java    |  18 +--
 .../hadoop/hbase/wal/DisabledWALProvider.java   |   2 +-
 .../apache/hadoop/hbase/wal/WALSplitter.java    |  14 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |  58 ++++++--
 .../hbase/coprocessor/TestWALObserver.java      |  13 +-
 .../hbase/filter/TestFilterFromRegionSide.java  |   2 +-
 .../hadoop/hbase/fs/TestBlockReorder.java       |  16 +-
 .../encoding/TestSeekBeforeWithReverseScan.java |   2 +-
 .../hadoop/hbase/mapreduce/TestWALPlayer.java   |  16 +-
 .../hbase/mapreduce/TestWALRecordReader.java    |  19 ++-
 .../master/TestMasterFileSystemWithWALDir.java  |  59 ++++++++
 .../procedure/TestWALProcedureStoreOnHDFS.java  |   2 +-
 .../regionserver/TestHRegionServerBulkLoad.java |   2 +-
 .../TestCompactedHFilesDischarger.java          |   2 +-
 .../hbase/regionserver/wal/TestFSHLog.java      |  31 ++--
 .../regionserver/wal/TestLogRollAbort.java      |   8 +-
 .../wal/TestWALActionsListener.java             |  25 +++-
 .../hbase/regionserver/wal/TestWALReplay.java   | 105 +++++++------
 .../apache/hadoop/hbase/util/TestFSUtils.java   |  51 ++++++-
 .../apache/hadoop/hbase/wal/IOTestProvider.java |   2 +-
 .../hbase/wal/TestDefaultWALProvider.java       |  18 ++-
 .../apache/hadoop/hbase/wal/TestWALFactory.java |  10 +-
 .../apache/hadoop/hbase/wal/TestWALRootDir.java | 148 +++++++++++++++++++
 .../apache/hadoop/hbase/wal/TestWALSplit.java   |  11 +-
 .../hbase/wal/WALPerformanceEvaluation.java     |   2 +-
 43 files changed, 729 insertions(+), 261 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 88e57d7..5ee5fc5 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -1172,6 +1172,13 @@ possible configurations would overwhelm and obscure the important.
     if it does not match.</description>
   </property>
   <property>
+    <name>hbase.wal.dir.perms</name>
+    <value>700</value>
+    <description>FS Permissions for the root WAL directory in a secure(kerberos) setup.
+      When master starts, it creates the WAL dir with this permissions or sets the permissions
+      if it does not match.</description>
+  </property>
+  <property>
     <name>hbase.data.umask.enable</name>
     <value>false</value>
     <description>Enable, if true, that file permissions should be assigned

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
index 560072f..817ed46 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
@@ -114,7 +114,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
   private final LeaseRecovery leaseRecovery;
   private final Configuration conf;
   private final FileSystem fs;
-  private final Path logDir;
+  private final Path walDir;
 
   private final AtomicReference<Throwable> syncException = new AtomicReference<Throwable>();
   private final AtomicBoolean loading = new AtomicBoolean(true);
@@ -170,11 +170,11 @@ public class WALProcedureStore extends ProcedureStoreBase {
     }
   }
 
-  public WALProcedureStore(final Configuration conf, final FileSystem fs, final Path logDir,
+  public WALProcedureStore(final Configuration conf, final FileSystem fs, final Path walDir,
       final LeaseRecovery leaseRecovery) {
     this.fs = fs;
     this.conf = conf;
-    this.logDir = logDir;
+    this.walDir = walDir;
     this.leaseRecovery = leaseRecovery;
   }
 
@@ -912,8 +912,8 @@ public class WALProcedureStore extends ProcedureStoreBase {
   // ==========================================================================
   //  FileSystem Log Files helpers
   // ==========================================================================
-  public Path getLogDir() {
-    return this.logDir;
+  public Path getWALDir() {
+    return this.walDir;
   }
 
   public FileSystem getFileSystem() {
@@ -921,7 +921,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
   }
 
   protected Path getLogFilePath(final long logId) throws IOException {
-    return new Path(logDir, String.format("state-%020d.log", logId));
+    return new Path(walDir, String.format("state-%020d.log", logId));
   }
 
   private static long getLogIdFromName(final String name) {
@@ -955,7 +955,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
 
   private FileStatus[] getLogFiles() throws IOException {
     try {
-      FileStatus[] files = fs.listStatus(logDir, WALS_PATH_FILTER);
+      FileStatus[] files = fs.listStatus(walDir, WALS_PATH_FILTER);
       Arrays.sort(files, FILE_STATUS_ID_COMPARATOR);
       return files;
     } catch (FileNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
index 64726ba..2480b9a 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -58,8 +58,8 @@ public class ProcedureTestingUtility {
   }
 
   public static WALProcedureStore createWalStore(final Configuration conf, final FileSystem fs,
-      final Path logDir) throws IOException {
-    return new WALProcedureStore(conf, fs, logDir, new WALProcedureStore.LeaseRecovery() {
+      final Path walDir) throws IOException {
+    return new WALProcedureStore(conf, fs, walDir, new WALProcedureStore.LeaseRecovery() {
       @Override
       public void recoverFileLease(FileSystem fs, Path path) throws IOException {
         // no-op

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java
index 347239d..401b859 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java
@@ -217,10 +217,10 @@ public class ProcedureWALLoaderPerformanceEvaluation extends AbstractHBaseTool {
   public void tearDownProcedureStore() {
     store.stop(false);
     try {
-      store.getFileSystem().delete(store.getLogDir(), true);
+      store.getFileSystem().delete(store.getWALDir(), true);
     } catch (IOException e) {
       System.err.println("Error: Couldn't delete log dir. You can delete it manually to free up "
-          + "disk space. Location: " + store.getLogDir().toString());
+          + "disk space. Location: " + store.getWALDir().toString());
       System.err.println(e.toString());
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALPerformanceEvaluation.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALPerformanceEvaluation.java
index 6f1332c..f657822 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALPerformanceEvaluation.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALPerformanceEvaluation.java
@@ -107,10 +107,10 @@ public class ProcedureWALPerformanceEvaluation extends AbstractHBaseTool {
   private void tearDownProcedureStore() {
     store.stop(false);
     try {
-      store.getFileSystem().delete(store.getLogDir(), true);
+      store.getFileSystem().delete(store.getWALDir(), true);
     } catch (IOException e) {
       System.err.println("Error: Couldn't delete log dir. You can delete it manually to free up "
-          + "disk space. Location: " + store.getLogDir().toString());
+          + "disk space. Location: " + store.getWALDir().toString());
       e.printStackTrace();
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java
index e77409a..daa5580 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java
@@ -31,6 +31,7 @@ import java.lang.reflect.Proxy;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.net.URI;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -51,16 +52,19 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import static org.apache.hadoop.hbase.HConstants.HBASE_DIR;
+
 /**
  * An encapsulation for the FileSystem object that hbase uses to access
- * data. This class allows the flexibility of using  
+ * data. This class allows the flexibility of using
  * separate filesystem objects for reading and writing hfiles and wals.
- * In future, if we want to make wals be in a different filesystem,
- * this is the place to make it happen.
  */
 public class HFileSystem extends FilterFileSystem {
   public static final Log LOG = LogFactory.getLog(HFileSystem.class);
 
+  /** Parameter name for HBase WAL directory */
+  public static final String HBASE_WAL_DIR = "hbase.wal.dir";
+
   private final FileSystem noChecksumFs;   // read hfile data from storage
   private final boolean useHBaseChecksum;
 
@@ -79,7 +83,7 @@ public class HFileSystem extends FilterFileSystem {
     // the underlying filesystem that has checksums switched on.
     this.fs = FileSystem.get(conf);
     this.useHBaseChecksum = useHBaseChecksum;
-    
+
     fs.initialize(getDefaultUri(conf), conf);
     
     // disable checksum verification for local fileSystem, see HBASE-11218

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/io/WALLink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/WALLink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/WALLink.java
index 344d496..f66daa3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/WALLink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/WALLink.java
@@ -45,17 +45,17 @@ public class WALLink extends FileLink {
    */
   public WALLink(final Configuration conf,
       final String serverName, final String logName) throws IOException {
-    this(FSUtils.getRootDir(conf), serverName, logName);
+    this(FSUtils.getWALRootDir(conf), serverName, logName);
   }
 
   /**
-   * @param rootDir Path to the root directory where hbase files are stored
+   * @param walRootDir Path to the root directory where hbase files are stored
    * @param serverName Region Server owner of the log
    * @param logName WAL file name
    */
-  public WALLink(final Path rootDir, final String serverName, final String logName) {
-    final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
-    final Path logDir = new Path(new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), serverName);
+  public WALLink(final Path walRootDir, final String serverName, final String logName) {
+    final Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    final Path logDir = new Path(new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME), serverName);
     setLocations(new Path(logDir, logName), new Path(oldLogDir, logName));
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index 79fe596..b68bd77 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -584,15 +584,15 @@ public class AssignmentManager extends ZooKeeperListener {
       Set<ServerName> queuedDeadServers = serverManager.getRequeuedDeadServers().keySet();
       if (!queuedDeadServers.isEmpty()) {
         Configuration conf = server.getConfiguration();
-        Path rootdir = FSUtils.getRootDir(conf);
-        FileSystem fs = rootdir.getFileSystem(conf);
+        Path walRootDir = FSUtils.getWALRootDir(conf);
+        FileSystem walFs = FSUtils.getWALFileSystem(conf);
         for (ServerName serverName: queuedDeadServers) {
           // In the case of a clean exit, the shutdown handler would have presplit any WALs and
           // removed empty directories.
-          Path logDir = new Path(rootdir,
+          Path walDir = new Path(walRootDir,
               DefaultWALProvider.getWALDirectoryName(serverName.toString()));
-          Path splitDir = logDir.suffix(DefaultWALProvider.SPLITTING_EXT);
-          if (fs.exists(logDir) || fs.exists(splitDir)) {
+          Path splitDir = walDir.suffix(DefaultWALProvider.SPLITTING_EXT);
+          if (walFs.exists(walDir) || walFs.exists(splitDir)) {
             LOG.debug("Found queued dead server " + serverName);
             failover = true;
             break;

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 315b4c4..74e1500 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -1186,7 +1186,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    int cleanerInterval = conf.getInt("hbase.master.cleaner.interval", 60 * 1000);
    this.logCleaner =
       new LogCleaner(cleanerInterval,
-         this, conf, getMasterFileSystem().getFileSystem(),
+         this, conf, getMasterFileSystem().getOldLogDir().getFileSystem(conf),
          getMasterFileSystem().getOldLogDir());
     getChoreService().scheduleChore(logCleaner);
 
@@ -1252,10 +1252,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
   private void startProcedureExecutor() throws IOException {
     final MasterProcedureEnv procEnv = new MasterProcedureEnv(this);
-    final Path logDir = new Path(fileSystemManager.getRootDir(),
+    final Path walDir = new Path(FSUtils.getWALRootDir(this.conf),
         MasterProcedureConstants.MASTER_PROCEDURE_LOGDIR);
 
-    procedureStore = new WALProcedureStore(conf, fileSystemManager.getFileSystem(), logDir,
+    procedureStore = new WALProcedureStore(conf, walDir.getFileSystem(conf), walDir,
         new MasterProcedureEnv.WALStoreLeaseRecovery(this));
     procedureStore.registerListener(new MasterProcedureEnv.MasterProcedureStoreListener(this));
     procedureExecutor = new ProcedureExecutor(conf, procEnv, procedureStore,

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index c15d730..332a726 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -66,7 +66,14 @@ import com.google.common.annotations.VisibleForTesting;
  */
 @InterfaceAudience.Private
 public class MasterFileSystem {
-  private static final Log LOG = LogFactory.getLog(MasterFileSystem.class.getName());
+  private static final Log LOG = LogFactory.getLog(MasterFileSystem.class);
+
+  /** Parameter name for HBase instance root directory permission*/
+  public static final String HBASE_DIR_PERMS = "hbase.rootdir.perms";
+
+  /** Parameter name for HBase WAL directory permission*/
+  public static final String HBASE_WAL_DIR_PERMS = "hbase.wal.dir.perms";
+
   // HBase configuration
   Configuration conf;
   // master status
@@ -77,8 +84,11 @@ public class MasterFileSystem {
   private ClusterId clusterId;
   // Keep around for convenience.
   private final FileSystem fs;
+  private final FileSystem walFs;
+  // root WAL directory
+  private final Path walRootDir;
   // Is the fileystem ok?
-  private volatile boolean fsOk = true;
+  private volatile boolean walFsOk = true;
   // The Path to the old logs dir
   private final Path oldLogDir;
   // root hbase directory on the FS
@@ -119,6 +129,10 @@ public class MasterFileSystem {
     // Cover both bases, the old way of setting default fs and the new.
     // We're supposed to run on 0.20 and 0.21 anyways.
     this.fs = this.rootdir.getFileSystem(conf);
+    this.walRootDir = FSUtils.getWALRootDir(conf);
+    this.walFs = FSUtils.getWALFileSystem(conf);
+    FSUtils.setFsDefault(conf, new Path(this.walFs.getUri()));
+    walFs.setConf(conf);
     FSUtils.setFsDefault(conf, new Path(this.fs.getUri()));
     // make sure the fs has the same conf
     fs.setConf(conf);
@@ -148,17 +162,21 @@ public class MasterFileSystem {
    * Idempotent.
    */
   private Path createInitialFileSystemLayout() throws IOException {
-    // check if the root directory exists
-    checkRootDir(this.rootdir, conf, this.fs);
+
+    checkRootDir(this.rootdir, conf, this.fs, HConstants.HBASE_DIR, HBASE_DIR_PERMS);
+    // if the log directory is different from root, check if it exists
+    if (!this.walRootDir.equals(this.rootdir)) {
+      checkRootDir(this.walRootDir, conf, this.walFs, HFileSystem.HBASE_WAL_DIR, HBASE_WAL_DIR_PERMS);
+    }
 
     // check if temp directory exists and clean it
     checkTempDir(this.tempdir, conf, this.fs);
 
-    Path oldLogDir = new Path(this.rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
+    Path oldLogDir = new Path(this.walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
 
     // Make sure the region servers can archive their old logs
-    if(!this.fs.exists(oldLogDir)) {
-      this.fs.mkdirs(oldLogDir);
+    if(!this.walFs.exists(oldLogDir)) {
+      this.walFs.mkdirs(oldLogDir);
     }
 
     return oldLogDir;
@@ -182,16 +200,24 @@ public class MasterFileSystem {
    * @return false if file system is not available
    */
   public boolean checkFileSystem() {
-    if (this.fsOk) {
+    if (this.walFsOk) {
       try {
-        FSUtils.checkFileSystemAvailable(this.fs);
+        FSUtils.checkFileSystemAvailable(this.walFs);
         FSUtils.checkDfsSafeMode(this.conf);
       } catch (IOException e) {
         master.abort("Shutting down HBase cluster: file system not available", e);
-        this.fsOk = false;
+        this.walFsOk = false;
       }
     }
-    return this.fsOk;
+    return this.walFsOk;
+  }
+
+  protected FileSystem getWALFileSystem() {
+    return this.walFs;
+  }
+
+  public Configuration getConfiguration() {
+    return this.conf;
   }
 
   /**
@@ -202,6 +228,11 @@ public class MasterFileSystem {
   }
 
   /**
+   * @return HBase root log dir.
+   */
+  public Path getWALRootDir() { return this.walRootDir; }
+
+  /**
    * @return HBase temp dir.
    */
   public Path getTempDir() {
@@ -224,7 +255,7 @@ public class MasterFileSystem {
         WALSplitter.SPLIT_SKIP_ERRORS_DEFAULT);
 
     Set<ServerName> serverNames = new HashSet<ServerName>();
-    Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME);
+    Path logsDirPath = new Path(this.walRootDir, HConstants.HREGION_LOGDIR_NAME);
 
     do {
       if (master.isStopped()) {
@@ -232,8 +263,8 @@ public class MasterFileSystem {
         break;
       }
       try {
-        if (!this.fs.exists(logsDirPath)) return serverNames;
-        FileStatus[] logFolders = FSUtils.listStatus(this.fs, logsDirPath, null);
+        if (!this.walFs.exists(logsDirPath)) return serverNames;
+        FileStatus[] logFolders = FSUtils.listStatus(this.walFs, logsDirPath, null);
         // Get online servers after getting log folders to avoid log folder deletion of newly
         // checked in region servers . see HBASE-5916
         Set<ServerName> onlineServers = ((HMaster) master).getServerManager().getOnlineServers()
@@ -244,7 +275,7 @@ public class MasterFileSystem {
           return serverNames;
         }
         for (FileStatus status : logFolders) {
-          FileStatus[] curLogFiles = FSUtils.listStatus(this.fs, status.getPath(), null);
+          FileStatus[] curLogFiles = FSUtils.listStatus(this.walFs, status.getPath(), null);
           if (curLogFiles == null || curLogFiles.length == 0) {
             // Empty log folder. No recovery needed
             continue;
@@ -325,17 +356,17 @@ public class MasterFileSystem {
     }
     try {
       for (ServerName serverName : serverNames) {
-        Path logDir = new Path(this.rootdir,
+        Path logDir = new Path(this.walRootDir,
             DefaultWALProvider.getWALDirectoryName(serverName.toString()));
         Path splitDir = logDir.suffix(DefaultWALProvider.SPLITTING_EXT);
         // Rename the directory so a rogue RS doesn't create more WALs
-        if (fs.exists(logDir)) {
-          if (!this.fs.rename(logDir, splitDir)) {
+        if (walFs.exists(logDir)) {
+          if (!this.walFs.rename(logDir, splitDir)) {
             throw new IOException("Failed fs.rename for log split: " + logDir);
           }
           logDir = splitDir;
           LOG.debug("Renamed region directory: " + splitDir);
-        } else if (!fs.exists(splitDir)) {
+        } else if (!walFs.exists(splitDir)) {
           LOG.info("Log dir for server " + serverName + " does not exist");
           continue;
         }
@@ -422,19 +453,19 @@ public class MasterFileSystem {
    */
   @SuppressWarnings("deprecation")
   private Path checkRootDir(final Path rd, final Configuration c,
-    final FileSystem fs)
+    final FileSystem fs, final String dirConfKey, final String dirPermsConfName)
   throws IOException {
     // If FS is in safe mode wait till out of it.
     FSUtils.waitOnSafeMode(c, c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
 
     boolean isSecurityEnabled = "kerberos".equalsIgnoreCase(c.get("hbase.security.authentication"));
-    FsPermission rootDirPerms = new FsPermission(c.get("hbase.rootdir.perms", "700"));
+    FsPermission dirPerms = new FsPermission(c.get(dirPermsConfName, "700"));
 
-    // Filesystem is good. Go ahead and check for hbase.rootdir.
+    // Filesystem is good. Go ahead and check for rootdir.
     try {
       if (!fs.exists(rd)) {
         if (isSecurityEnabled) {
-          fs.mkdirs(rd, rootDirPerms);
+          fs.mkdirs(rd, dirPerms);
         } else {
           fs.mkdirs(rd);
         }
@@ -452,15 +483,15 @@ public class MasterFileSystem {
         if (!fs.isDirectory(rd)) {
           throw new IllegalArgumentException(rd.toString() + " is not a directory");
         }
-        if (isSecurityEnabled && !rootDirPerms.equals(fs.getFileStatus(rd).getPermission())) {
+        if (isSecurityEnabled && !dirPerms.equals(fs.getFileStatus(rd).getPermission())) {
           // check whether the permission match
-          LOG.warn("Found rootdir permissions NOT matching expected \"hbase.rootdir.perms\" for "
+          LOG.warn("Found rootdir permissions NOT matching expected \"" + dirPermsConfName + "\" for "
               + "rootdir=" + rd.toString() + " permissions=" + fs.getFileStatus(rd).getPermission()
-              + " and  \"hbase.rootdir.perms\" configured as "
-              + c.get("hbase.rootdir.perms", "700") + ". Automatically setting the permissions. You"
-              + " can change the permissions by setting \"hbase.rootdir.perms\" in hbase-site.xml "
+              + " and  \"" + dirPermsConfName + "\" configured as "
+              + c.get(dirPermsConfName, "700") + ". Automatically setting the permissions. You"
+              + " can change the permissions by setting \"" + dirPermsConfName + "\" in hbase-site.xml "
               + "and restarting the master");
-          fs.setPermission(rd, rootDirPerms);
+          fs.setPermission(rd, dirPerms);
         }
         // as above
         FSUtils.checkVersion(fs, rd, true, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
@@ -468,38 +499,41 @@ public class MasterFileSystem {
             HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
       }
     } catch (DeserializationException de) {
-      LOG.fatal("Please fix invalid configuration for " + HConstants.HBASE_DIR, de);
+      LOG.fatal("Please fix invalid configuration for " + dirConfKey, de);
       IOException ioe = new IOException();
       ioe.initCause(de);
       throw ioe;
     } catch (IllegalArgumentException iae) {
       LOG.fatal("Please fix invalid configuration for "
-        + HConstants.HBASE_DIR + " " + rd.toString(), iae);
+        + dirConfKey + " " + rd.toString(), iae);
       throw iae;
     }
-    // Make sure cluster ID exists
-    if (!FSUtils.checkClusterIdExists(fs, rd, c.getInt(
-        HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000))) {
-      FSUtils.setClusterId(fs, rd, new ClusterId(), c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
-    }
-    clusterId = FSUtils.getClusterId(fs, rd);
-
-    // Make sure the meta region directory exists!
-    if (!FSUtils.metaRegionExists(fs, rd)) {
-      bootstrap(rd, c);
-    } else {
-      // Migrate table descriptor files if necessary
-      org.apache.hadoop.hbase.util.FSTableDescriptorMigrationToSubdir
-        .migrateFSTableDescriptorsIfNecessary(fs, rd);
-    }
 
-    // Create tableinfo-s for hbase:meta if not already there.
+    if (dirConfKey.equals(HConstants.HBASE_DIR)) {
+      // Make sure cluster ID exists
+      if (!FSUtils.checkClusterIdExists(fs, rd, c.getInt(
+          HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000))) {
+        FSUtils.setClusterId(fs, rd, new ClusterId(), c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
+      }
+      clusterId = FSUtils.getClusterId(fs, rd);
+
+      // Make sure the meta region directory exists!
+      if (!FSUtils.metaRegionExists(fs, rd)) {
+        bootstrap(rd, c);
+      } else {
+        // Migrate table descriptor files if necessary
+        org.apache.hadoop.hbase.util.FSTableDescriptorMigrationToSubdir
+            .migrateFSTableDescriptorsIfNecessary(fs, rd);
+      }
 
-    // meta table is a system table, so descriptors are predefined,
-    // we should get them from registry.
-    FSTableDescriptors fsd = new FSTableDescriptors(c, fs, rd);
-    fsd.createTableDescriptor(
-      new HTableDescriptor(fsd.get(TableName.META_TABLE_NAME)));
+      // Create tableinfo-s for hbase:meta if not already there.
+
+      // meta table is a system table, so descriptors are predefined,
+      // we should get them from registry.
+      FSTableDescriptors fsd = new FSTableDescriptors(c, fs, rd);
+      fsd.createTableDescriptor(
+          new HTableDescriptor(fsd.get(TableName.META_TABLE_NAME)));
+    }
 
     return rd;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
index 852b6c4..d5620b9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
@@ -266,7 +266,7 @@ public class SplitLogManager {
       // recover-lease is done. totalSize will be under in most cases and the
       // metrics that it drives will also be under-reported.
       totalSize += lf.getLen();
-      String pathToLog = FSUtils.removeRootPath(lf.getPath(), conf);
+      String pathToLog = FSUtils.removeWALRootPath(lf.getPath(), conf);
       if (!enqueueSplitTask(pathToLog, batch)) {
         throw new IOException("duplicate log split scheduled for " + lf.getPath());
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
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 c2ccf83..9d8c0c6 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
@@ -6624,7 +6624,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       // The WAL subsystem will use the default rootDir rather than the passed in rootDir
       // unless I pass along via the conf.
       Configuration confForWAL = new Configuration(conf);
-      confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
+      FSUtils.setRootDir(confForWAL, rootDir);
       effectiveWAL = (new WALFactory(confForWAL,
           Collections.<WALActionsListener>singletonList(new MetricsWAL()),
           "hregion-" + RandomStringUtils.randomNumeric(8))).

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/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 545c926..21f269e 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
@@ -298,6 +298,7 @@ public class HRegionServer extends HasThread implements
   // If false, the file system has become unavailable
   protected volatile boolean fsOk;
   protected HFileSystem fs;
+  protected HFileSystem walFs;
 
   // Set when a report to the master comes back with a message asking us to
   // shutdown. Also set by call to stop when debugging or running unit tests
@@ -319,6 +320,7 @@ public class HRegionServer extends HasThread implements
   protected final Configuration conf;
 
   private Path rootDir;
+  private Path walRootDir;
 
   protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
 
@@ -632,13 +634,16 @@ public class HRegionServer extends HasThread implements
   }
 
   private void initializeFileSystem() throws IOException {
+    // Get fs instance used by this RS.  Do we use checksum verification in the hbase? If hbase
+    // checksum verification enabled, then automatically switch off hdfs checksum verification.
+    boolean useHBaseChecksum = conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, true);
+    FSUtils.setFsDefault(this.conf, FSUtils.getWALRootDir(this.conf));
+    this.walFs = new HFileSystem(this.conf, useHBaseChecksum);
+    this.walRootDir = FSUtils.getWALRootDir(this.conf);
     // Set 'fs.defaultFS' to match the filesystem on hbase.rootdir else
     // underlying hadoop hdfs accessors will be going against wrong filesystem
     // (unless all is set to defaults).
     FSUtils.setFsDefault(this.conf, FSUtils.getRootDir(this.conf));
-    // Get fs instance used by this RS. Do we use checksum verification in the hbase? If hbase
-    // checksum verification enabled, then automatically switch off hdfs checksum verification.
-    boolean useHBaseChecksum = this.conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, true);
     this.fs = new HFileSystem(this.conf, useHBaseChecksum);
     this.rootDir = FSUtils.getRootDir(this.conf);
     this.tableDescriptors = new FSTableDescriptors(
@@ -1666,19 +1671,19 @@ public class HRegionServer extends HasThread implements
    */
   private WALFactory setupWALAndReplication() throws IOException {
     // TODO Replication make assumptions here based on the default filesystem impl
-    final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    final Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
     final String logName = DefaultWALProvider.getWALDirectoryName(this.serverName.toString());
 
-    Path logdir = new Path(rootDir, logName);
-    if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
-    if (this.fs.exists(logdir)) {
+    Path logDir = new Path(walRootDir, logName);
+    if (LOG.isDebugEnabled()) LOG.debug("logDir=" + logDir);
+    if (this.walFs.exists(logDir)) {
       throw new RegionServerRunningException("Region server has already " +
         "created directory at " + this.serverName.toString());
     }
 
     // Instantiate replication manager if replication enabled.  Pass it the
     // log directories.
-    createNewReplicationInstance(conf, this, this.fs, logdir, oldLogDir);
+    createNewReplicationInstance(conf, this, this.walFs, logDir, oldLogDir);
 
     // listeners the wal factory will add to wals it creates.
     final List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
@@ -2619,6 +2624,20 @@ public class HRegionServer extends HasThread implements
     return fs;
   }
 
+  /**
+   * @return Return the walRootDir.
+   */
+  protected Path getWALRootDir() {
+    return walRootDir;
+  }
+
+  /**
+   * @return Return the walFs.
+   */
+  protected FileSystem getWALFileSystem() {
+    return walFs;
+  }
+
   @Override
   public String toString() {
     return getServerName().toString();
@@ -2685,7 +2704,7 @@ public class HRegionServer extends HasThread implements
    * Load the replication service objects, if any
    */
   static private void createNewReplicationInstance(Configuration conf,
-    HRegionServer server, FileSystem fs, Path logDir, Path oldLogDir) throws IOException{
+    HRegionServer server, FileSystem walFs, Path walDir, Path oldWALDir) throws IOException{
 
     // If replication is not enabled, then return immediately.
     if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
@@ -2711,21 +2730,21 @@ public class HRegionServer extends HasThread implements
     if (sourceClassname.equals(sinkClassname)) {
       server.replicationSourceHandler = (ReplicationSourceService)
                                          newReplicationInstance(sourceClassname,
-                                         conf, server, fs, logDir, oldLogDir);
+                                         conf, server, walFs, walDir, oldWALDir);
       server.replicationSinkHandler = (ReplicationSinkService)
                                          server.replicationSourceHandler;
     } else {
       server.replicationSourceHandler = (ReplicationSourceService)
                                          newReplicationInstance(sourceClassname,
-                                         conf, server, fs, logDir, oldLogDir);
+                                         conf, server, walFs, walDir, oldWALDir);
       server.replicationSinkHandler = (ReplicationSinkService)
                                          newReplicationInstance(sinkClassname,
-                                         conf, server, fs, logDir, oldLogDir);
+                                         conf, server, walFs, walDir, oldWALDir);
     }
   }
 
   static private ReplicationService newReplicationInstance(String classname,
-    Configuration conf, HRegionServer server, FileSystem fs, Path logDir,
+    Configuration conf, HRegionServer server, FileSystem walFs, Path walDir,
     Path oldLogDir) throws IOException{
 
     Class<?> clazz = null;
@@ -2739,7 +2758,7 @@ public class HRegionServer extends HasThread implements
     // create an instance of the replication object.
     ReplicationService service = (ReplicationService)
                               ReflectionUtils.newInstance(clazz, conf);
-    service.initialize(server, fs, logDir, oldLogDir);
+    service.initialize(server, walFs, walDir, oldLogDir);
     return service;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
index b3e9f76..fb67736 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
@@ -89,11 +89,11 @@ public class SplitLogWorker implements Runnable {
     this(server, conf, server, new TaskExecutor() {
       @Override
       public Status exec(String filename, RecoveryMode mode, CancelableProgressable p) {
-        Path rootdir;
+        Path walDir;
         FileSystem fs;
         try {
-          rootdir = FSUtils.getRootDir(conf);
-          fs = rootdir.getFileSystem(conf);
+          walDir = FSUtils.getWALRootDir(conf);
+          fs = walDir.getFileSystem(conf);
         } catch (IOException e) {
           LOG.warn("could not find root dir or fs", e);
           return Status.RESIGNED;
@@ -102,7 +102,7 @@ public class SplitLogWorker implements Runnable {
         // interrupted or has encountered a transient error and when it has
         // encountered a bad non-retry-able persistent error.
         try {
-          if (!WALSplitter.splitLogFile(rootdir, fs.getFileStatus(new Path(rootdir, filename)),
+          if (!WALSplitter.splitLogFile(walDir, fs.getFileStatus(new Path(walDir, filename)),
             fs, conf, p, sequenceIdChecker, server.getCoordinatedStateManager(), mode, factory)) {
             return Status.PREEMPTED;
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index eb734bb..1adce03 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -1580,9 +1580,9 @@ public class FSHLog implements WAL {
     ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
     ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
 
-  private static void split(final Configuration conf, final Path p)
-  throws IOException {
-    FileSystem fs = FileSystem.get(conf);
+
+  private static void split(final Configuration conf, final Path p) throws IOException {
+    FileSystem fs = FSUtils.getWALFileSystem(conf);
     if (!fs.exists(p)) {
       throw new FileNotFoundException(p.toString());
     }
@@ -1590,7 +1590,7 @@ public class FSHLog implements WAL {
       throw new IOException(p + " is not a directory");
     }
 
-    final Path baseDir = FSUtils.getRootDir(conf);
+    final Path baseDir = FSUtils.getWALRootDir(conf);
     final Path archiveDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
     WALSplitter.split(baseDir, p, archiveDir, fs, conf, WALFactory.getInstance(conf));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
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 8112553..d5b398c 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
@@ -849,9 +849,9 @@ public class ReplicationSource extends Thread
             // to look at)
             List<String> deadRegionServers = this.replicationQueueInfo.getDeadRegionServers();
             LOG.info("NB dead servers : " + deadRegionServers.size());
-            final Path rootDir = FSUtils.getRootDir(conf);
+            final Path walDir = FSUtils.getWALRootDir(conf);
             for (String curDeadServerName : deadRegionServers) {
-              final Path deadRsDirectory = new Path(rootDir,
+              final Path deadRsDirectory = new Path(walDir,
                   DefaultWALProvider.getWALDirectoryName(curDeadServerName));
               Path[] locs = new Path[] {
                   new Path(deadRsDirectory, currentPath.getName()),
@@ -878,7 +878,7 @@ public class ReplicationSource extends Thread
             // In the case of disaster/recovery, HMaster may be shutdown/crashed before flush data
             // from .logs to .oldlogs. Loop into .logs folders and check whether a match exists
             if (stopper instanceof ReplicationSyncUp.DummyServer) {
-              // N.B. the ReplicationSyncUp tool sets the manager.getLogDir to the root of the wal
+              // N.B. the ReplicationSyncUp tool sets the manager.getWALDir to the root of the wal
               //      area rather than to the wal area for a particular region server.
               FileStatus[] rss = fs.listStatus(manager.getLogDir());
               for (FileStatus rs : rss) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
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 3d28c97..481302f 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
@@ -76,7 +76,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
     Replication replication;
     ReplicationSourceManager manager;
     FileSystem fs;
-    Path oldLogDir, logDir, rootDir;
+    Path oldLogDir, logDir, walRootDir;
     ZooKeeperWatcher zkw;
 
     Abortable abortable = new Abortable() {
@@ -94,10 +94,10 @@ public class ReplicationSyncUp extends Configured implements Tool {
         new ZooKeeperWatcher(conf, "syncupReplication" + System.currentTimeMillis(), abortable,
             true);
 
-    rootDir = FSUtils.getRootDir(conf);
-    fs = FileSystem.get(conf);
-    oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
-    logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
+    walRootDir = FSUtils.getWALRootDir(conf);
+    fs = FSUtils.getWALFileSystem(conf);
+    oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
 
     System.out.println("Start Replication Server start");
     replication = new Replication(new DummyServer(zkw), fs, logDir, oldLogDir);

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index b3be54e..5784b74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -18,6 +18,10 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Iterators;
+import com.google.common.primitives.Ints;
+
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.EOFException;
@@ -83,8 +87,7 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 
-import com.google.common.collect.Iterators;
-import com.google.common.primitives.Ints;
+import static org.apache.hadoop.hbase.HConstants.HBASE_DIR;
 
 /**
  * Utility methods for interacting with the underlying file system.
@@ -935,22 +938,22 @@ public abstract class FSUtils {
       return root;
     } catch (URISyntaxException e) {
       IOException io = new IOException("Root directory path is not a valid " +
-        "URI -- check your " + HConstants.HBASE_DIR + " configuration");
+        "URI -- check your " + HBASE_DIR + " configuration");
       io.initCause(e);
       throw io;
     }
   }
 
   /**
-   * Checks for the presence of the root path (using the provided conf object) in the given path. If
+   * Checks for the presence of the WAL log root path (using the provided conf object) in the given path. If
    * it exists, this method removes it and returns the String representation of remaining relative path.
    * @param path
    * @param conf
    * @return String representation of the remaining relative path
    * @throws IOException
    */
-  public static String removeRootPath(Path path, final Configuration conf) throws IOException {
-    Path root = FSUtils.getRootDir(conf);
+  public static String removeWALRootPath(Path path, final Configuration conf) throws IOException {
+    Path root = getWALRootDir(conf);
     String pathStr = path.toString();
     // check that the path is absolute... it has the root path in it.
     if (!pathStr.startsWith(root.toString())) return pathStr;
@@ -997,24 +1000,65 @@ public abstract class FSUtils {
 
   /**
    * @param c configuration
-   * @return Path to hbase root directory: i.e. <code>hbase.rootdir</code> from
+   * @return {@link Path} to hbase root directory: i.e. {@value org.apache.hadoop.hbase.HConstants#HBASE_DIR} from
    * configuration as a qualified Path.
    * @throws IOException e
    */
   public static Path getRootDir(final Configuration c) throws IOException {
-    Path p = new Path(c.get(HConstants.HBASE_DIR));
+    Path p = new Path(c.get(HBASE_DIR));
     FileSystem fs = p.getFileSystem(c);
     return p.makeQualified(fs);
   }
 
   public static void setRootDir(final Configuration c, final Path root) throws IOException {
-    c.set(HConstants.HBASE_DIR, root.toString());
+    c.set(HBASE_DIR, root.toString());
   }
 
   public static void setFsDefault(final Configuration c, final Path root) throws IOException {
     c.set("fs.defaultFS", root.toString());    // for hadoop 0.21+
   }
 
+  public static FileSystem getRootDirFileSystem(final Configuration c) throws IOException {
+    Path p = getRootDir(c);
+    return p.getFileSystem(c);
+  }
+
+  /**
+   * @param c configuration
+   * @return {@link Path} to hbase log root directory: i.e. {@value org.apache.hadoop.hbase.fs.HFileSystem#HBASE_WAL_DIR} from
+   * configuration as a qualified Path. Defaults to {@value org.apache.hadoop.hbase.HConstants#HBASE_DIR}
+   * @throws IOException e
+   */
+  public static Path getWALRootDir(final Configuration c) throws IOException {
+    Path p = new Path(c.get(HFileSystem.HBASE_WAL_DIR, c.get(HBASE_DIR)));
+    if (!isValidWALRootDir(p, c)) {
+      return FSUtils.getRootDir(c);
+    }
+    FileSystem fs = p.getFileSystem(c);
+    return p.makeQualified(fs);
+  }
+
+  @VisibleForTesting
+  public static void setWALRootDir(final Configuration c, final Path root) throws IOException {
+    c.set(HFileSystem.HBASE_WAL_DIR, root.toString());
+  }
+
+  public static FileSystem getWALFileSystem(final Configuration c) throws IOException {
+    Path p = getWALRootDir(c);
+    return p.getFileSystem(c);
+  }
+
+  private static boolean isValidWALRootDir(Path walDir, final Configuration c) throws IOException {
+    Path rootDir = FSUtils.getRootDir(c);
+    if (walDir != rootDir) {
+      if (walDir.toString().startsWith(rootDir.toString() + "/")) {
+        throw new IllegalStateException("Illegal WAL directory specified. " +
+            "WAL directories are not permitted to be under the root directory if set.");
+      }
+    }
+    return true;
+  }
+
   /**
    * Checks if meta region exists
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
index b618a0f..01234a1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
@@ -45,10 +45,10 @@ import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 
 /**
- * A WAL Provider that returns a single thread safe WAL that writes to HDFS.
- * By default, this implementation picks a directory in HDFS based on a combination of
+ * A WAL Provider that returns a single thread safe WAL that writes to Hadoop FS.
+ * By default, this implementation picks a directory in Hadoop FS based on a combination of
  * <ul>
- *   <li>the HBase root directory
+ *   <li>the HBase root WAL directory
  *   <li>HConstants.HREGION_LOGDIR_NAME
  *   <li>the given factory's factoryId (usually identifying the regionserver by host:port)
  * </ul>
@@ -126,7 +126,7 @@ public class DefaultWALProvider implements WALProvider {
       // creating hlog on fs is time consuming
       synchronized (walCreateLock) {
         if (log == null) {
-          log = new FSHLog(FileSystem.get(conf), FSUtils.getRootDir(conf),
+          log = new FSHLog(FSUtils.getWALFileSystem(conf), FSUtils.getWALRootDir(conf),
               getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf,
               listeners, true, logPrefix,
               META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
@@ -282,14 +282,10 @@ public class DefaultWALProvider implements WALProvider {
       throw new IllegalArgumentException("parameter conf must be set");
     }
 
-    final String rootDir = conf.get(HConstants.HBASE_DIR);
-    if (rootDir == null || rootDir.isEmpty()) {
-      throw new IllegalArgumentException(HConstants.HBASE_DIR
-          + " key not found in conf.");
-    }
+    final String walDir = FSUtils.getWALRootDir(conf).toString();
 
-    final StringBuilder startPathSB = new StringBuilder(rootDir);
-    if (!rootDir.endsWith("/"))
+    final StringBuilder startPathSB = new StringBuilder(walDir);
+    if (!walDir.endsWith("/"))
       startPathSB.append('/');
     startPathSB.append(HConstants.HREGION_LOGDIR_NAME);
     if (!HConstants.HREGION_LOGDIR_NAME.endsWith("/"))

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
index 62df2c7..18720bf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
@@ -61,7 +61,7 @@ class DisabledWALProvider implements WALProvider {
     if (null == providerId) {
       providerId = "defaultDisabled";
     }
-    disabled = new DisabledWAL(new Path(FSUtils.getRootDir(conf), providerId), conf, null);
+    disabled = new DisabledWAL(new Path(FSUtils.getWALRootDir(conf), providerId), conf, null);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 1927eb3..50bb79f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -255,24 +255,24 @@ public class WALSplitter {
   // log splitting. Used by tools and unit tests. It should be package private.
   // It is public only because UpgradeTo96 and TestWALObserver are in different packages,
   // which uses this method to do log splitting.
-  public static List<Path> split(Path rootDir, Path logDir, Path oldLogDir,
-      FileSystem fs, Configuration conf, final WALFactory factory) throws IOException {
+  public static List<Path> split(Path walRootDir, Path logDir, Path oldLogDir,
+      FileSystem walFs, Configuration conf, final WALFactory factory) throws IOException {
     final FileStatus[] logfiles = SplitLogManager.getFileList(conf,
         Collections.singletonList(logDir), null);
     List<Path> splits = new ArrayList<Path>();
     if (logfiles != null && logfiles.length > 0) {
       for (FileStatus logfile: logfiles) {
-        WALSplitter s = new WALSplitter(factory, conf, rootDir, fs, null, null,
+        WALSplitter s = new WALSplitter(factory, conf, walRootDir, walFs, null, null,
             RecoveryMode.LOG_SPLITTING);
         if (s.splitLogFile(logfile, null)) {
-          finishSplitLogFile(rootDir, oldLogDir, logfile.getPath(), conf);
+          finishSplitLogFile(walRootDir, oldLogDir, logfile.getPath(), conf);
           if (s.outputSink.splits != null) {
             splits.addAll(s.outputSink.splits);
           }
         }
       }
     }
-    if (!fs.delete(logDir, true)) {
+    if (!walFs.delete(logDir, true)) {
       throw new IOException("Unable to delete src dir: " + logDir);
     }
     return splits;
@@ -455,7 +455,7 @@ public class WALSplitter {
    */
   public static void finishSplitLogFile(String logfile,
       Configuration conf)  throws IOException {
-    Path rootdir = FSUtils.getRootDir(conf);
+    Path rootdir = FSUtils.getWALRootDir(conf);
     Path oldLogDir = new Path(rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
     Path logPath;
     if (FSUtils.isStartingWithPath(rootdir, logfile)) {
@@ -498,7 +498,7 @@ public class WALSplitter {
       final List<Path> corruptedLogs,
       final List<Path> processedLogs, final Path oldLogDir,
       final FileSystem fs, final Configuration conf) throws IOException {
-    final Path corruptDir = new Path(FSUtils.getRootDir(conf), conf.get(
+    final Path corruptDir = new Path(FSUtils.getWALRootDir(conf), conf.get(
         "hbase.regionserver.hlog.splitlog.corrupt.dir",  HConstants.CORRUPT_DIR_NAME));
 
     if (!fs.mkdirs(corruptDir)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/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 3e42c03..6a33a19 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
@@ -841,6 +841,16 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   }
 
   /**
+   * Start up a minicluster of hbase, dfs, and zookeeper where WAL's walDir is created separately.
+   * @throws Exception
+   * @return Mini hbase cluster instance created.
+   * @see {@link #shutdownMiniDFSCluster()}
+   */
+  public MiniHBaseCluster startMiniCluster(boolean withWALDir) throws Exception {
+    return startMiniCluster(1, 1, 1, null, null, null, false, withWALDir);
+  }
+
+  /**
    * Start up a minicluster of hbase, dfs, and zookeeper.
    * Set the <code>create</code> flag to create root or data directory path or not
    * (will overwrite if dir already exists)
@@ -871,6 +881,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     return startMiniCluster(1, numSlaves, false);
   }
 
+  public MiniHBaseCluster startMiniCluster(final int numSlaves, boolean create, boolean withWALDir)
+          throws Exception {
+    return startMiniCluster(1, numSlaves, numSlaves, null, null, null, create, withWALDir);
+  }
+
   /**
    * Start minicluster. Whether to create a new root or data dir path even if such a path
    * has been created earlier is decided based on flag <code>create</code>
@@ -900,7 +915,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
       final int numSlaves, final String[] dataNodeHosts, boolean create)
       throws Exception {
     return startMiniCluster(numMasters, numSlaves, numSlaves, dataNodeHosts,
-        null, null, create);
+        null, null, create, false);
   }
 
   /**
@@ -983,7 +998,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
       Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
     throws Exception {
     return startMiniCluster(numMasters, numSlaves, numDataNodes, dataNodeHosts,
-        masterClass, regionserverClass, false);
+        masterClass, regionserverClass, false, false);
   }
 
   /**
@@ -997,7 +1012,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     final int numSlaves, int numDataNodes, final String[] dataNodeHosts,
     Class<? extends HMaster> masterClass,
     Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass,
-    boolean create)
+    boolean create, boolean withWALDir)
   throws Exception {
     if (dataNodeHosts != null && dataNodeHosts.length != 0) {
       numDataNodes = dataNodeHosts.length;
@@ -1028,12 +1043,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
 
     // Start the MiniHBaseCluster
     return startMiniHBaseCluster(numMasters, numSlaves, masterClass,
-      regionserverClass, create);
+      regionserverClass, create, withWALDir);
   }
 
   public MiniHBaseCluster startMiniHBaseCluster(final int numMasters, final int numSlaves)
       throws IOException, InterruptedException{
-    return startMiniHBaseCluster(numMasters, numSlaves, null, null, false);
+    return startMiniHBaseCluster(numMasters, numSlaves, null, null, false, false);
   }
 
   /**
@@ -1052,11 +1067,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   public MiniHBaseCluster startMiniHBaseCluster(final int numMasters,
         final int numSlaves, Class<? extends HMaster> masterClass,
         Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass,
-        boolean create)
+        boolean create, boolean withWALDir)
   throws IOException, InterruptedException {
     // Now do the mini hbase cluster.  Set the hbase.rootdir in config.
     createRootDir(create);
 
+    if (withWALDir) {
+      createWALRootDir();
+    }
+
     // These settings will make the server waits until this exact number of
     // regions servers are connected.
     if (conf.getInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1) == -1) {
@@ -1240,6 +1259,22 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     return createRootDir(false);
   }
 
+  /**
+   * Creates a hbase walDir in the user's home directory.
+   * Normally you won't make use of this method. Root hbaseWALDir
+   * is created for you as part of mini cluster startup. You'd only use this
+   * method if you were doing manual operation.
+   *
+   * @return Fully qualified path to hbase WAL root dir
+   * @throws IOException
+  */
+  public Path createWALRootDir() throws IOException {
+    FileSystem fs = FileSystem.get(this.conf);
+    Path walDir = getNewDataTestDirOnTestFS();
+    FSUtils.setWALRootDir(this.conf, walDir);
+    fs.mkdirs(walDir);
+    return walDir;
+  }
 
   private void setHBaseFsTmpDir() throws IOException {
     String hbaseFsTmpDirInString = this.conf.get("hbase.fs.tmp.dir");
@@ -1816,12 +1851,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Create an unmanaged WAL. Be sure to close it when you're through.
    */
-  public static WAL createWal(final Configuration conf, final Path rootDir, final HRegionInfo hri)
+  public static WAL createWal(final Configuration conf, final Path rootDir, final Path walRootDir, final HRegionInfo hri)
       throws IOException {
     // The WAL subsystem will use the default rootDir rather than the passed in rootDir
     // unless I pass along via the conf.
     Configuration confForWAL = new Configuration(conf);
     confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
+    FSUtils.setWALRootDir(confForWAL, walRootDir);
     return (new WALFactory(confForWAL,
         Collections.<WALActionsListener>singletonList(new MetricsWAL()),
         "hregion-" + RandomStringUtils.randomNumeric(8))).
@@ -1833,8 +1869,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
    */
   public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
-      final Configuration conf, final HTableDescriptor htd) throws IOException {
-    return createRegionAndWAL(info, rootDir, conf, htd, true);
+      final Path walRootDir, final Configuration conf, final HTableDescriptor htd) throws IOException {
+    return createRegionAndWAL(info, rootDir, walRootDir, conf, htd, true);
   }
 
   /**
@@ -1842,9 +1878,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
    */
   public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
-      final Configuration conf, final HTableDescriptor htd, boolean initialize)
+      final Path walRootDir, final Configuration conf, final HTableDescriptor htd, boolean initialize)
       throws IOException {
-    WAL wal = createWal(conf, rootDir, info);
+    WAL wal = createWal(conf, rootDir, walRootDir, info);
     return HRegion.createHRegion(info, rootDir, conf, htd, wal, initialize);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
index c7a6a0c..aca2978 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
@@ -97,6 +97,7 @@ public class TestWALObserver {
   private FileSystem fs;
   private Path dir;
   private Path hbaseRootDir;
+  private Path hbaseWALRootDir;
   private String logName;
   private Path oldLogDir;
   private Path logDir;
@@ -115,8 +116,11 @@ public class TestWALObserver {
     TEST_UTIL.startMiniCluster(1);
     Path hbaseRootDir = TEST_UTIL.getDFSCluster().getFileSystem()
         .makeQualified(new Path("/hbase"));
+    Path hbaseWALRootDir = TEST_UTIL.getDFSCluster().getFileSystem()
+            .makeQualified(new Path("/hbaseLogRoot"));
     LOG.info("hbase.rootdir=" + hbaseRootDir);
     FSUtils.setRootDir(conf, hbaseRootDir);
+    FSUtils.setWALRootDir(conf, hbaseWALRootDir);
   }
 
   @AfterClass
@@ -130,16 +134,20 @@ public class TestWALObserver {
     // this.cluster = TEST_UTIL.getDFSCluster();
     this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
     this.hbaseRootDir = FSUtils.getRootDir(conf);
+    this.hbaseWALRootDir = FSUtils.getWALRootDir(conf);
     this.dir = new Path(this.hbaseRootDir, TestWALObserver.class.getName());
-    this.oldLogDir = new Path(this.hbaseRootDir,
+    this.oldLogDir = new Path(this.hbaseWALRootDir,
         HConstants.HREGION_OLDLOGDIR_NAME);
-    this.logDir = new Path(this.hbaseRootDir,
+    this.logDir = new Path(this.hbaseWALRootDir,
         DefaultWALProvider.getWALDirectoryName(currentTest.getMethodName()));
     this.logName = HConstants.HREGION_LOGDIR_NAME;
 
     if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) {
       TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
     }
+    if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseWALRootDir)) {
+      TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseWALRootDir, true);
+    }
     this.wals = new WALFactory(conf, null, currentTest.getMethodName());
   }
 
@@ -153,6 +161,7 @@ public class TestWALObserver {
       LOG.debug("details of failure to close wal factory.", exception);
     }
     TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
+    TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseWALRootDir, true);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
index c574a95..f69f370 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
@@ -84,7 +84,7 @@ public class TestFilterFromRegionSide {
     }
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     REGION = HBaseTestingUtility
-        .createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
+        .createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
     for(Put put:createPuts(ROWS, FAMILIES, QUALIFIERS, VALUE)){
       REGION.put(put);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
index 1aa75a1..91e11dc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
@@ -88,6 +88,8 @@ public class TestBlockReorder {
   private static final String host1 = "host1";
   private static final String host2 = "host2";
   private static final String host3 = "host3";
+  private static Path rootDir;
+  private static Path walRootDir;
 
   @Before
   public void setUp() throws Exception {
@@ -101,10 +103,14 @@ public class TestBlockReorder {
     conf = htu.getConfiguration();
     cluster = htu.getDFSCluster();
     dfs = (DistributedFileSystem) FileSystem.get(conf);
+    rootDir = htu.createRootDir();
+    walRootDir = htu.createWALRootDir();
   }
 
   @After
   public void tearDownAfterClass() throws Exception {
+    dfs.delete(rootDir, true);
+    dfs.delete(walRootDir, true);
     htu.shutdownMiniCluster();
   }
 
@@ -277,7 +283,7 @@ public class TestBlockReorder {
 
     // Now we need to find the log file, its locations, and look at it
 
-    String rootDir = new Path(FSUtils.getRootDir(conf) + "/" + HConstants.HREGION_LOGDIR_NAME +
+    String walDir = new Path(FSUtils.getWALRootDir(conf) + "/" + HConstants.HREGION_LOGDIR_NAME +
             "/" + targetRs.getServerName().toString()).toUri().getPath();
 
     DistributedFileSystem mdfs = (DistributedFileSystem)
@@ -321,7 +327,7 @@ public class TestBlockReorder {
       p.add(sb, sb, sb);
       h.put(p);
 
-      DirectoryListing dl = dfs.getClient().listPaths(rootDir, HdfsFileStatus.EMPTY_NAME);
+      DirectoryListing dl = dfs.getClient().listPaths(walDir, HdfsFileStatus.EMPTY_NAME);
       HdfsFileStatus[] hfs = dl.getPartialListing();
 
       // As we wrote a put, we should have at least one log file.
@@ -329,8 +335,8 @@ public class TestBlockReorder {
       for (HdfsFileStatus hf : hfs) {
         // Because this is a live cluster, log files might get archived while we're processing
         try {
-          LOG.info("Log file found: " + hf.getLocalName() + " in " + rootDir);
-          String logFile = rootDir + "/" + hf.getLocalName();
+          LOG.info("Log file found: " + hf.getLocalName() + " in " + walDir);
+          String logFile = walDir + "/" + hf.getLocalName();
           FileStatus fsLog = rfs.getFileStatus(new Path(logFile));
 
           LOG.info("Checking log file: " + logFile);
@@ -457,7 +463,7 @@ public class TestBlockReorder {
       // Should be reordered, as we pretend to be a file name with a compliant stuff
       Assert.assertNotNull(conf.get(HConstants.HBASE_DIR));
       Assert.assertFalse(conf.get(HConstants.HBASE_DIR).isEmpty());
-      String pseudoLogFile = conf.get(HConstants.HBASE_DIR) + "/" +
+      String pseudoLogFile = conf.get(HFileSystem.HBASE_WAL_DIR) + "/" +
           HConstants.HREGION_LOGDIR_NAME + "/" + host1 + ",6977,6576" + "/mylogfile";
 
       // Check that it will be possible to extract a ServerName from our construction

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java
index 258a11c..37a19a6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java
@@ -59,7 +59,7 @@ public class TestSeekBeforeWithReverseScan {
     htd.addFamily(new HColumnDescriptor(cfName).setDataBlockEncoding(DataBlockEncoding.FAST_DIFF));
     HRegionInfo info = new HRegionInfo(tableName, null, null, false);
     Path path = testUtil.getDataTestDir(getClass().getSimpleName());
-    region = HBaseTestingUtility.createRegionAndWAL(info, path, testUtil.getConfiguration(), htd);
+    region = HBaseTestingUtility.createRegionAndWAL(info, path, path, testUtil.getConfiguration(), htd);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
index 343fc64..4f54732 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
@@ -30,6 +30,7 @@ import java.io.PrintStream;
 import java.util.ArrayList;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -48,6 +49,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.WALPlayer.WALKeyValueMapper;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.LauncherSecurityManager;
@@ -67,16 +69,28 @@ import org.mockito.stubbing.Answer;
 public class TestWALPlayer {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static MiniHBaseCluster cluster;
+  private static Path rootDir;
+  private static Path walRootDir;
+  private static FileSystem fs;
+  private static FileSystem walFs;
+  private static Configuration conf;
 
   @BeforeClass
   public static void beforeClass() throws Exception {
     TEST_UTIL.setJobWithoutMRCluster();
+    conf= TEST_UTIL.getConfiguration();
+    rootDir = TEST_UTIL.createRootDir();
+    walRootDir = TEST_UTIL.createWALRootDir();
+    fs = FSUtils.getRootDirFileSystem(conf);
+    walFs = FSUtils.getWALFileSystem(conf);
     cluster = TEST_UTIL.startMiniCluster();
   }
 
   @AfterClass
   public static void afterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
+    fs.delete(rootDir, true);
+    walFs.delete(walRootDir, true);
   }
 
   /**
@@ -108,7 +122,7 @@ public class TestWALPlayer {
     WAL log = cluster.getRegionServer(0).getWAL(null);
     log.rollWriter();
     String walInputDir = new Path(cluster.getMaster().getMasterFileSystem()
-        .getRootDir(), HConstants.HREGION_LOGDIR_NAME).toString();
+        .getWALRootDir(), HConstants.HREGION_LOGDIR_NAME).toString();
 
     Configuration configuration= TEST_UTIL.getConfiguration();
     WALPlayer player = new WALPlayer(configuration);

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
index 2423d03..34b9570 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
@@ -27,7 +27,6 @@ import java.util.concurrent.atomic.AtomicLong;
 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.Cell;
@@ -43,6 +42,7 @@ import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALKeyRecordReader;
 import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALRecordReader;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
@@ -65,6 +65,8 @@ public class TestWALRecordReader {
   private static Configuration conf;
   private static FileSystem fs;
   private static Path hbaseDir;
+  private static FileSystem walFs;
+  private static Path walRootDir;
   // visible for TestHLogRecordReader
   static final TableName tableName = TableName.valueOf(getName());
   private static final byte [] rowName = tableName.getName();
@@ -83,12 +85,9 @@ public class TestWALRecordReader {
 
   @Before
   public void setUp() throws Exception {
+    fs.delete(hbaseDir, true);
+    walFs.delete(walRootDir, true);
     mvcc = new MultiVersionConcurrencyControl();
-    FileStatus[] entries = fs.listStatus(hbaseDir);
-    for (FileStatus dir : entries) {
-      fs.delete(dir.getPath(), true);
-    }
-
   }
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -102,8 +101,10 @@ public class TestWALRecordReader {
     fs = TEST_UTIL.getDFSCluster().getFileSystem();
 
     hbaseDir = TEST_UTIL.createRootDir();
-    
-    logDir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME);
+
+    walRootDir = TEST_UTIL.createWALRootDir();
+    walFs = FSUtils.getWALFileSystem(conf);
+    logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
 
     htd = new HTableDescriptor(tableName);
     htd.addFamily(new HColumnDescriptor(family));
@@ -111,6 +112,8 @@ public class TestWALRecordReader {
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
+    fs.delete(hbaseDir, true);
+    walFs.delete(walRootDir, true);
     TEST_UTIL.shutdownMiniCluster();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java
new file mode 100644
index 0000000..77e4b3a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java
@@ -0,0 +1,59 @@
+/**
+ * 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.master;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the master filesystem in a local cluster
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestMasterFileSystemWithWALDir {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setupTest() throws Exception {
+    UTIL.startMiniCluster(true);
+  }
+
+  @AfterClass
+  public static void teardownTest() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testFsUriSetProperly() throws Exception {
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    MasterFileSystem fs = master.getMasterFileSystem();
+    Path masterRoot = FSUtils.getRootDir(fs.getConfiguration());
+    Path rootDir = FSUtils.getRootDir(fs.getFileSystem().getConf());
+    assertEquals(masterRoot, rootDir);
+    assertEquals(FSUtils.getWALRootDir(UTIL.getConfiguration()), fs.getWALRootDir());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
index ea45cf6..8a93151 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
@@ -105,7 +105,7 @@ public class TestWALProcedureStoreOnHDFS {
 
   public void tearDown() throws Exception {
     store.stop(false);
-    UTIL.getDFSCluster().getFileSystem().delete(store.getLogDir(), true);
+    UTIL.getDFSCluster().getFileSystem().delete(store.getWALDir(), true);
 
     try {
       UTIL.shutdownMiniCluster();

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index 6e64eb6..c40c684 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -344,7 +344,7 @@ public class TestHRegionServerBulkLoad {
     int millisToRun = 30000;
     int numScanners = 50;
 
-    UTIL.startMiniCluster(1);
+    UTIL.startMiniCluster(1, false, true);
     try {
       WAL log = UTIL.getHBaseCluster().getRegionServer(0).getWAL(null);
       FindBulkHBaseListener listener = new FindBulkHBaseListener();

http://git-wip-us.apache.org/repos/asf/hbase/blob/979438d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
index 816c357..afd71f1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
@@ -76,7 +76,7 @@ public class TestCompactedHFilesDischarger {
     htd.addFamily(new HColumnDescriptor(fam));
     HRegionInfo info = new HRegionInfo(tableName, null, null, false);
     Path path = testUtil.getDataTestDir(getClass().getSimpleName());
-    region = HBaseTestingUtility.createRegionAndWAL(info, path, testUtil.getConfiguration(), htd);
+    region = HBaseTestingUtility.createRegionAndWAL(info, path, path, testUtil.getConfiguration(), htd);
     rss = mock(RegionServerServices.class);
     List<Region> regions = new ArrayList<Region>();
     regions.add(region);


[14/50] [abbrv] hbase git commit: HBASE-19435 Reopen Files for ClosedChannelException in BucketCache

Posted by ap...@apache.org.
HBASE-19435 Reopen Files for ClosedChannelException in BucketCache

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/branch-1.3
Commit: 4b6e589bbbe600ad68c205c758ea7f90ff3c67c7
Parents: b442a7f
Author: Zach York <zy...@amazon.com>
Authored: Mon Dec 4 12:11:21 2017 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:17 2018 -0800

----------------------------------------------------------------------
 .../hbase/io/hfile/bucket/FileIOEngine.java     | 28 +++++++++++++++++++-
 .../hbase/io/hfile/bucket/TestFileIOEngine.java | 15 +++++++++++
 2 files changed, 42 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4b6e589b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
index 3419587..cb454d4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
@@ -19,12 +19,15 @@
 package org.apache.hadoop.hbase.io.hfile.bucket;
 
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
 import java.nio.channels.FileChannel;
 import java.util.Arrays;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -108,6 +111,17 @@ public class FileIOEngine implements IOEngine {
     return 0;
   }
 
+  @VisibleForTesting
+  void closeFileChannels() {
+    for (FileChannel fileChannel: fileChannels) {
+      try {
+        fileChannel.close();
+      } catch (IOException e) {
+        LOG.warn("Failed to close FileChannel", e);
+      }
+    }
+  }
+
   /**
    * Transfers data from the given byte buffer to file
    * @param srcBuffer the given byte buffer from which bytes are to be read
@@ -169,11 +183,18 @@ public class FileIOEngine implements IOEngine {
     int bufLimit = buffer.limit();
     while (true) {
       FileChannel fileChannel = fileChannels[accessFileNum];
+      int accessLen = 0;
       if (endFileNum > accessFileNum) {
         // short the limit;
         buffer.limit((int) (buffer.limit() - remainingAccessDataLen + sizePerFile - accessOffset));
       }
-      int accessLen = accessor.access(fileChannel, buffer, accessOffset);
+      try {
+        accessLen = accessor.access(fileChannel, buffer, accessOffset);
+      } catch (ClosedChannelException e) {
+        LOG.warn("Caught ClosedChannelException accessing BucketCache, reopening file. ", e);
+        refreshFileConnection(accessFileNum);
+        continue;
+      }
       // recover the limit
       buffer.limit(bufLimit);
       if (accessLen < remainingAccessDataLen) {
@@ -213,6 +234,11 @@ public class FileIOEngine implements IOEngine {
     return fileNum;
   }
 
+  private void refreshFileConnection(int accessFileNum) throws FileNotFoundException {
+    rafs[accessFileNum] = new RandomAccessFile(filePaths[accessFileNum], "rw");
+    fileChannels[accessFileNum] = rafs[accessFileNum].getChannel();
+  }
+
   private static interface FileAccessor {
     int access(FileChannel fileChannel, ByteBuffer byteBuffer, long accessOffset)
         throws IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b6e589b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
index a03818b..adf7fd0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
@@ -114,4 +114,19 @@ public class TestFileIOEngine {
     fileIOEngine.read(ByteBuffer.wrap(data2), 0);
     assertArrayEquals(data1, data2);
   }
+
+  @Test
+  public void testClosedChannelException() throws IOException {
+    fileIOEngine.closeFileChannels();
+    int len = 5;
+    long offset = 0L;
+    byte[] data1 = new byte[len];
+    for (int j = 0; j < data1.length; ++j) {
+      data1[j] = (byte) (Math.random() * 255);
+    }
+    byte[] data2 = new byte[len];
+    fileIOEngine.write(ByteBuffer.wrap(data1), offset);
+    fileIOEngine.read(ByteBuffer.wrap(data2), offset);
+    assertArrayEquals(data1, data2);
+  }
 }


[29/50] [abbrv] hbase git commit: HBASE-20647 Backport HBASE-20616 "TruncateTableProcedure is stuck in retry loop in TRUNCATE_TABLE_CREATE_FS_LAYOUT state" to branch-1

Posted by ap...@apache.org.
HBASE-20647 Backport HBASE-20616 "TruncateTableProcedure is stuck in retry loop in TRUNCATE_TABLE_CREATE_FS_LAYOUT state" to branch-1

Signed-off-by: tedyu <yu...@gmail.com>

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java


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

Branch: refs/heads/branch-1.3
Commit: 03885b434f0cfbaaa0ed367d5d30c062eb2a48b7
Parents: 511a4c4
Author: Toshihiro Suzuki <br...@gmail.com>
Authored: Fri May 25 09:37:19 2018 +0900
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:19 2018 -0800

----------------------------------------------------------------------
 .../procedure/TruncateTableProcedure.java       |  32 ++++--
 .../procedure/TestTruncateTableProcedure.java   | 101 +++++++++++++++++--
 2 files changed, 116 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/03885b43/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
index da539a1..8471777 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
@@ -18,29 +18,29 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
-import java.io.InputStream;
+import com.google.common.annotations.VisibleForTesting;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.HBaseException;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -102,15 +102,15 @@ public class TruncateTableProcedure
           break;
         case TRUNCATE_TABLE_CLEAR_FS_LAYOUT:
           DeleteTableProcedure.deleteFromFs(env, getTableName(), regions, true);
+          setNextState(TruncateTableState.TRUNCATE_TABLE_CREATE_FS_LAYOUT);
           if (!preserveSplits) {
-            // if we are not preserving splits, generate a new single region
             regions = Arrays.asList(ModifyRegionUtils.createHRegionInfos(hTableDescriptor, null));
           } else {
             regions = recreateRegionInfo(regions);
           }
-          setNextState(TruncateTableState.TRUNCATE_TABLE_CREATE_FS_LAYOUT);
           break;
         case TRUNCATE_TABLE_CREATE_FS_LAYOUT:
+          DeleteTableProcedure.deleteFromFs(env, getTableName(), regions, true);
           regions = CreateTableProcedure.createFsLayout(env, hTableDescriptor, regions);
           CreateTableProcedure.updateTableDescCache(env, getTableName());
           setNextState(TruncateTableState.TRUNCATE_TABLE_ADD_TO_META);
@@ -183,7 +183,9 @@ public class TruncateTableProcedure
 
   @Override
   protected boolean acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) return false;
+    if (env.waitInitialized(this)) {
+      return false;
+    }
     return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());
   }
 
@@ -295,4 +297,12 @@ public class TruncateTableProcedure
       });
     }
   }
-}
\ No newline at end of file
+
+  @VisibleForTesting
+  HRegionInfo getFirstRegionInfo() {
+    if (regions == null || regions.isEmpty()) {
+      return null;
+    }
+    return regions.get(0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/03885b43/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
index edcfb5b..0ef72f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
@@ -18,21 +18,33 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
 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.ServerRegionReplicaUtil;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -40,11 +52,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-
 @Category(MediumTests.class)
 public class TestTruncateTableProcedure {
   private static final Log LOG = LogFactory.getLog(TestTruncateTableProcedure.class);
@@ -290,4 +297,86 @@ public class TestTruncateTableProcedure {
     UTIL.waitUntilAllRegionsAssigned(tableName);
   }
 
+  @Test
+  public void testOnHDFSFailurePreserveSplits() throws Exception {
+    final TableName tableName = TableName.valueOf("testOnHDFSFailurePreserveSplits");
+    testOnHDFSFailure(tableName, true);
+  }
+
+  @Test
+  public void testOnHDFSFailureNoPreserveSplits() throws Exception {
+    final TableName tableName = TableName.valueOf("testOnHDFSFailureNoPreserveSplits");
+    testOnHDFSFailure(tableName, false);
+  }
+
+  public static class TruncateTableProcedureOnHDFSFailure extends TruncateTableProcedure {
+
+    private boolean failOnce = false;
+
+    public TruncateTableProcedureOnHDFSFailure() {
+      // Required by the Procedure framework to create the procedure on replay
+      super();
+    }
+
+    public TruncateTableProcedureOnHDFSFailure(final MasterProcedureEnv env, TableName tableName,
+      boolean preserveSplits)
+      throws HBaseIOException {
+      super(env, tableName, preserveSplits);
+    }
+
+    @Override
+    protected Flow executeFromState(MasterProcedureEnv env,
+      MasterProcedureProtos.TruncateTableState state) throws InterruptedException {
+
+      if (!failOnce &&
+        state == MasterProcedureProtos.TruncateTableState.TRUNCATE_TABLE_CREATE_FS_LAYOUT) {
+        try {
+          // To emulate an HDFS failure, create only the first region directory
+          HRegionInfo regionInfo = getFirstRegionInfo();
+          Configuration conf = env.getMasterConfiguration();
+          MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+          Path tempdir = mfs.getTempDir();
+          Path tableDir = FSUtils.getTableDir(tempdir, regionInfo.getTable());
+          Path regionDir = new Path(tableDir,
+            ServerRegionReplicaUtil.getRegionInfoForFs(regionInfo).getEncodedName());
+          FileSystem fs = FileSystem.get(conf);
+          fs.mkdirs(regionDir);
+
+          failOnce = true;
+          return Flow.HAS_MORE_STATE;
+        } catch (IOException e) {
+          fail("failed to create a region directory: " + e);
+        }
+      }
+
+      return super.executeFromState(env, state);
+    }
+  }
+
+  private void testOnHDFSFailure(TableName tableName, boolean preserveSplits) throws Exception {
+    String[] families = new String[] { "f1", "f2" };
+    byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+
+    // create a table
+    MasterProcedureTestingUtility.createTable(
+      getMasterProcedureExecutor(), tableName, splitKeys, families);
+
+    // load and verify that there are rows in the table
+    MasterProcedureTestingUtility.loadData(
+      UTIL.getConnection(), tableName, 100, splitKeys, families);
+    assertEquals(100, UTIL.countRows(tableName));
+
+    // disable the table
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    // truncate the table
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    long procId = ProcedureTestingUtility.submitAndWait(procExec,
+      new TruncateTableProcedureOnHDFSFailure(procExec.getEnvironment(), tableName,
+        preserveSplits));
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+  }
+
 }


[47/50] [abbrv] hbase git commit: HBASE-20857 balancer status tag in jmx metrics

Posted by ap...@apache.org.
HBASE-20857 balancer status tag in jmx metrics

Signed-off-by: Andrew Purtell <ap...@apache.org>
Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 10c4f59eca5c8b6e037e5551388e19928cd20fce
Parents: 785e21f
Author: Kiran Kumar Maturi <ma...@gmail.com>
Authored: Tue Sep 18 09:45:26 2018 +0530
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 19:25:56 2018 -0800

----------------------------------------------------------------------
 .../master/balancer/MetricsBalancerSource.java  |   3 +
 .../balancer/MetricsBalancerSourceImpl.java     |   7 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   5 +
 .../hadoop/hbase/master/LoadBalancer.java       |   3 +
 .../hadoop/hbase/master/MasterRpcServices.java  |   1 +
 .../hadoop/hbase/master/MasterServices.java     |   5 +
 .../hbase/master/balancer/BaseLoadBalancer.java |   7 ++
 .../hbase/master/balancer/MetricsBalancer.java  |   4 +
 .../balancer/MetricsStochasticBalancer.java     |   8 ++
 .../hbase/master/MockNoopMasterServices.java    |   5 +
 .../hadoop/hbase/master/TestCatalogJanitor.java |   5 +
 .../TestBalancerStatusTagInJMXMetrics.java      | 108 +++++++++++++++++++
 12 files changed, 160 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSource.java
index 1c9a61e..f9cc62b 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSource.java
@@ -39,6 +39,7 @@ public interface MetricsBalancerSource extends BaseSource  {
 
   String BALANCE_CLUSTER = "balancerCluster";
   String MISC_INVOATION_COUNT = "miscInvocationCount";
+  String BALANCER_STATUS = "isBalancerActive";
 
   /**
    * Description
@@ -48,4 +49,6 @@ public interface MetricsBalancerSource extends BaseSource  {
   void updateBalanceCluster(long time);
 
   void incrMiscInvocations();
+
+  void updateBalancerStatus(boolean status);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
index 0a74630..de904d4 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
@@ -37,6 +37,7 @@ public class MetricsBalancerSourceImpl extends BaseSourceImpl implements Metrics
                                    String metricsDescription,
                                    String metricsContext, String metricsJmxContext) {
     super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+    updateBalancerStatus(true);
   }
 
 
@@ -44,7 +45,6 @@ public class MetricsBalancerSourceImpl extends BaseSourceImpl implements Metrics
   public void init() {
     blanceClusterHisto = metricsRegistry.newTimeHistogram(BALANCE_CLUSTER);
     miscCount = metricsRegistry.newCounter(MISC_INVOATION_COUNT, "", 0L);
-
   }
 
   @Override
@@ -56,4 +56,9 @@ public class MetricsBalancerSourceImpl extends BaseSourceImpl implements Metrics
   public void incrMiscInvocations() {
      miscCount.incr();
   }
+
+  @Override
+  public void updateBalancerStatus(boolean status) {
+    metricsRegistry.tag(BALANCER_STATUS,"", String.valueOf(status), true);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 74e1500..e1f7ad1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -2427,6 +2427,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   }
 
   @Override
+  public LoadBalancer getLoadBalancer() {
+    return balancer;
+  }
+
+  @Override
   public ServerName getServerName() {
     return this.serverName;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
index 1642af1..d5fdeb1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
@@ -154,4 +154,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
    * If balancer needs to do initialization after Master has started up, lets do that here.
    */
   void postMasterStartupInitialize();
+
+  /*Updates balancer status tag reported to JMX*/
+  void updateBalancerStatus(boolean status);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 94554b8..b5b0a78 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -272,6 +272,7 @@ public class MasterRpcServices extends RSRpcServices
       if (master.cpHost != null) {
         master.cpHost.postBalanceSwitch(oldValue, newValue);
       }
+      master.getLoadBalancer().updateBalancerStatus(newValue);
     } catch (IOException ioe) {
       LOG.warn("Error flipping balance switch", ioe);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 368441f..cfc93c4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -97,6 +97,11 @@ public interface MasterServices extends Server {
   ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor();
 
   /**
+   * @return load balancer
+   */
+  LoadBalancer getLoadBalancer();
+
+  /**
    * Check table is modifiable; i.e. exists and is offline.
    * @param tableName Name of table to check.
    * @throws TableNotDisabledException

http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index c6d8e7b..3e26373 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -1535,6 +1535,13 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   }
 
   /**
+  * Updates the balancer status tag reported to JMX
+  */
+  public void updateBalancerStatus(boolean status) {
+    metricsBalancer.balancerStatus(status);
+  }
+
+  /**
    * Used to assign a single region to a random server.
    */
   private ServerName randomAssignment(Cluster cluster, HRegionInfo regionInfo,

http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancer.java
index ac2a517..40fb2e0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancer.java
@@ -46,4 +46,8 @@ public class MetricsBalancer {
   public void incrMiscInvocations() {
     source.incrMiscInvocations();
   }
+
+  public void balancerStatus(boolean status) {
+    source.updateBalancerStatus(status);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancer.java
index 850a9f5..0a1f4ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsStochasticBalancer.java
@@ -55,6 +55,14 @@ public class MetricsStochasticBalancer extends MetricsBalancer {
   }
 
   /**
+   * Updates the balancer status tag reported to JMX
+   */
+  @Override
+  public void balancerStatus(boolean status) {
+    stochasticSource.updateBalancerStatus(status);
+  }
+
+  /**
    * Updates the number of metrics reported to JMX
    */
   public void updateMetricsSize(int size) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 530359a..94e9e20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -85,6 +85,11 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
+  public LoadBalancer getLoadBalancer() {
+    return null;
+  }
+
+  @Override
   public void checkTableModifiable(TableName tableName) throws IOException, TableNotFoundException, TableNotDisabledException {
 
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 59c7a10..c53ab01 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -284,6 +284,11 @@ public class TestCatalogJanitor {
     }
 
     @Override
+    public LoadBalancer getLoadBalancer() {
+      return null;
+    }
+
+    @Override
     public ServerManager getServerManager() {
       return sm;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/10c4f59e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java
new file mode 100644
index 0000000..03488ea
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java
@@ -0,0 +1,108 @@
+/**
+ * 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.master.balancer;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsTag;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class })
+public class TestBalancerStatusTagInJMXMetrics extends BalancerTestBase {
+  private static final Log LOG = LogFactory.getLog(TestBalancerStatusTagInJMXMetrics.class);
+  private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static int connectorPort = 61120;
+  private static HMaster master;
+  private static MiniHBaseCluster cluster;
+  private static Configuration conf = null;
+
+  /**
+   * Setup the environment for the test.
+   */
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+
+    conf = UTIL.getConfiguration();
+    Random rand = new Random();
+    for (int i = 0; i < 10; i++) {
+      do {
+        int sign = i % 2 == 0 ? 1 : -1;
+        connectorPort += sign * rand.nextInt(100);
+      } while (!HBaseTestingUtility.available(connectorPort));
+      try {
+        conf.setInt("regionserver.rmi.registry.port", connectorPort);
+        cluster = UTIL.startMiniCluster();
+        LOG.info("Waiting for active/ready master");
+        cluster.waitForActiveAndReadyMaster();
+        master = cluster.getMaster();
+        break;
+      } catch (Exception e) {
+        LOG.debug("Encountered exception when starting mini cluster. Trying port " + connectorPort,
+          e);
+        try {
+          // this is to avoid "IllegalStateException: A mini-cluster is already running"
+          UTIL.shutdownMiniCluster();
+        } catch (Exception ex) {
+          LOG.debug("Encountered exception shutting down cluster", ex);
+        }
+      }
+    }
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Tests the status change using the Default Metrics System
+   */
+  @Test
+  public void testJmxMetrics() throws Exception {
+
+    assertEquals(getStatus(), "true");
+    master.getLoadBalancer().updateBalancerStatus(false);
+    assertEquals(getStatus(), "false");
+
+  }
+
+  /**
+   * Gets the balancer status tag from the Metrics registry
+   */
+  public String getStatus() throws Exception {
+    MetricsSource source =
+        DefaultMetricsSystem.instance().getSource(MetricsBalancerSource.METRICS_JMX_CONTEXT);
+    if (source instanceof MetricsBalancerSourceImpl) {
+      MetricsTag status = ((MetricsBalancerSourceImpl) source).getMetricsRegistry()
+          .getTag(MetricsBalancerSource.BALANCER_STATUS);
+      return status.value();
+    } else {
+      LOG.warn("Balancer JMX Metrics not registered");
+      throw new Exception("MetricsBalancer JMX Context not found");
+    }
+  }
+
+}


[26/50] [abbrv] hbase git commit: HBASE-20404 Fixes to CleanChore correctness and operability.

Posted by ap...@apache.org.
HBASE-20404 Fixes to CleanChore correctness and operability.

* Make CleanerChore less chatty: move WARN message to DEBUG when we expect non-empty dirs
* Make CleanerChore less chatty: move IOE we'll retry to INFO
* CleanerChore should treat IOE for FileStatus as a failure
* Add tests asserting assumptions in above

Signed-off-by: Reid Chan <re...@outlook.com>
Signed-off-by: Mike Drob <md...@apache.org>

 Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java


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

Branch: refs/heads/branch-1.3
Commit: be4915eb4f6604edd570b2bb937800e663f76043
Parents: 193d1dc
Author: Sean Busbey <bu...@apache.org>
Authored: Fri Apr 13 00:57:35 2018 -0500
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:19 2018 -0800

----------------------------------------------------------------------
 .../hbase/master/cleaner/CleanerChore.java      | 24 +++++++--
 .../hbase/master/cleaner/TestCleanerChore.java  | 54 ++++++++++++++++++--
 .../apache/hadoop/hbase/util/TestFSUtils.java   | 12 +++++
 3 files changed, 84 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/be4915eb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
index dc614fb..5a4c407 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
@@ -33,6 +33,7 @@ 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.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -406,6 +407,10 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
     T act() throws IOException;
   }
 
+  /**
+   * Attemps to clean up a directory, its subdirectories, and files.
+   * Return value is true if everything was deleted. false on partial / total failures.
+   */
   private class CleanerTask extends RecursiveTask<Boolean> {
     private final Path dir;
     private final boolean root;
@@ -425,6 +430,8 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
       List<FileStatus> subDirs;
       final List<FileStatus> files;
       try {
+        // if dir doesn't exist, we'll get null back for both of these
+        // which will fall through to succeeding.
         subDirs = FSUtils.listStatusWithStatusFilter(fs, dir, new FileStatusFilter() {
           @Override
           public boolean accept(FileStatus f) {
@@ -438,8 +445,8 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
           }
         });
       } catch (IOException ioe) {
-        LOG.warn(dir + " doesn't exist, just skip it. ", ioe);
-        return true;
+        LOG.warn("failed to get FileStatus for contents of '" + dir + "'", ioe);
+        return false;
       }
 
       boolean nullSubDirs = subDirs == null;
@@ -497,8 +504,19 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
       try {
         LOG.trace("Start deleting " + type + " under " + dir);
         deleted = deletion.act();
+      } catch (PathIsNotEmptyDirectoryException exception) {
+        // N.B. HDFS throws this exception when we try to delete a non-empty directory, but
+        // LocalFileSystem throws a bare IOException. So some test code will get the verbose
+        // message below.
+        LOG.debug("Couldn't delete '" + dir + "' yet because it isn't empty. Probably transient. " +
+            "exception details at TRACE.");
+        LOG.trace("Couldn't delete '" + dir + "' yet because it isn't empty w/exception.",
+            exception);
+        deleted = false;
       } catch (IOException ioe) {
-        LOG.warn("Could not delete " + type + " under " + dir, ioe);
+        LOG.info("Could not delete " + type + " under " + dir + ". might be transient; we'll " +
+            "retry. if it keeps happening, use following exception when asking on mailing list.",
+            ioe);
         deleted = false;
       }
       LOG.trace("Finish deleting " + type + " under " + dir + " deleted=" + deleted);

http://git-wip-us.apache.org/repos/asf/hbase/blob/be4915eb/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
index 505fd2c..7711354 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -30,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -84,9 +86,55 @@ public class TestCleanerChore {
     // run the chore
     chore.chore();
 
-    // verify all the files got deleted
-    assertTrue("File didn't get deleted", fs.exists(file));
-    assertTrue("Empty directory didn't get deleted", fs.exists(parent));
+    // verify all the files were preserved
+    assertTrue("File shouldn't have been deleted", fs.exists(file));
+    assertTrue("directory shouldn't have been deleted", fs.exists(parent));
+  }
+
+  @Test
+  public void retriesIOExceptionInStatus() throws Exception {
+    Stoppable stop = new StoppableImplementation();
+    Configuration conf = UTIL.getConfiguration();
+    Path testDir = UTIL.getDataTestDir();
+    FileSystem fs = UTIL.getTestFileSystem();
+    String confKey = "hbase.test.cleaner.delegates";
+
+    Path child = new Path(testDir, "child");
+    Path file = new Path(child, "file");
+    fs.mkdirs(child);
+    fs.create(file).close();
+    assertTrue("test file didn't get created.", fs.exists(file));
+    final AtomicBoolean fails = new AtomicBoolean(true);
+
+    FilterFileSystem filtered = new FilterFileSystem(fs) {
+      public FileStatus[] listStatus(Path f) throws IOException {
+        if (fails.get()) {
+          throw new IOException("whomp whomp.");
+        }
+        return fs.listStatus(f);
+      }
+    };
+
+    AllValidPaths chore = new AllValidPaths("test-retry-ioe", stop, conf, filtered, testDir, confKey);
+
+    // trouble talking to the filesystem
+    Boolean result = chore.runCleaner();
+
+    // verify that it couldn't clean the files.
+    assertTrue("test rig failed to inject failure.", fs.exists(file));
+    assertTrue("test rig failed to inject failure.", fs.exists(child));
+    // and verify that it accurately reported the failure.
+    assertFalse("chore should report that it failed.", result);
+
+    // filesystem is back
+    fails.set(false);
+    result = chore.runCleaner();
+
+    // verify everything is gone.
+    assertFalse("file should have been destroyed.", fs.exists(file));
+    assertFalse("directory should have been destroyed.", fs.exists(child));
+    // and verify that it accurately reported success.
+    assertTrue("chore should claim it succeeded.", result);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/be4915eb/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
index 2699292..96f4180 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
@@ -300,6 +300,18 @@ public class TestFSUtils {
   }
 
   @Test
+  public void testFilteredStatusDoesNotThrowOnNotFound() throws Exception {
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    MiniDFSCluster cluster = htu.startMiniDFSCluster(1);
+    try {
+      assertNull(FSUtils.listStatusWithStatusFilter(cluster.getFileSystem(), new Path("definitely/doesn't/exist"), null));
+    } finally {
+      cluster.shutdown();
+    }
+
+  }
+
+  @Test
   public void testRenameAndSetModifyTime() throws Exception {
     HBaseTestingUtility htu = new HBaseTestingUtility();
     Configuration conf = htu.getConfiguration();


[13/50] [abbrv] hbase git commit: HBASE-19364 Truncate_preserve fails with table when replica region > 1

Posted by ap...@apache.org.
HBASE-19364 Truncate_preserve fails with table when replica region > 1

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java


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

Branch: refs/heads/branch-1.3
Commit: cd1726f53181e655246bda16b63a7f1fb990955a
Parents: 4b6e589
Author: Pankaj Kumar <pa...@huawei.com>
Authored: Thu Dec 7 22:51:01 2017 +0530
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:17 2018 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 45 ++++++++++++++++--
 .../master/procedure/ProcedureSyncWait.java     | 25 ++++++----
 .../procedure/TruncateTableProcedure.java       |  2 +-
 .../MasterProcedureTestingUtility.java          |  5 ++
 .../procedure/TestTruncateTableProcedure.java   | 50 ++++++++++++++++++++
 5 files changed, 112 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cd1726f5/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 3f11558..440f8c6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -392,12 +392,27 @@ public class MetaTableAccessor {
    * @return Ordered list of {@link HRegionInfo}.
    * @throws IOException
    */
-  public static List<HRegionInfo> getTableRegions(ZooKeeperWatcher zkw,
-      Connection connection, TableName tableName, final boolean excludeOfflinedSplitParents)
-        throws IOException {
+  public static List<HRegionInfo> getTableRegions(ZooKeeperWatcher zkw, Connection connection,
+      TableName tableName, final boolean excludeOfflinedSplitParents) throws IOException {
+    return getTableRegions(zkw, connection, tableName, excludeOfflinedSplitParents, false);
+  }
+
+  /**
+   * Gets all of the regions of the specified table.
+   * @param zkw zookeeper connection to access meta table
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @param excludeOfflinedSplitParents If true, do not include offlined split parents in the
+   *          return.
+   * @param excludeReplicaRegions If true, do not include replica regions in the result.
+   * @return Ordered list of {@link HRegionInfo}.
+   */
+  public static List<HRegionInfo> getTableRegions(ZooKeeperWatcher zkw, Connection connection,
+      TableName tableName, final boolean excludeOfflinedSplitParents,
+      final boolean excludeReplicaRegions) throws IOException {
     List<Pair<HRegionInfo, ServerName>> result = null;
-      result = getTableRegionsAndLocations(zkw, connection, tableName,
-        excludeOfflinedSplitParents);
+    result = getTableRegionsAndLocations(zkw, connection, tableName, excludeOfflinedSplitParents,
+      excludeReplicaRegions);
     return getListOfHRegionInfos(result);
   }
 
@@ -478,7 +493,22 @@ public class MetaTableAccessor {
   public static List<Pair<HRegionInfo, ServerName>> getTableRegionsAndLocations(
       ZooKeeperWatcher zkw, Connection connection, final TableName tableName,
       final boolean excludeOfflinedSplitParents) throws IOException {
+    return getTableRegionsAndLocations(zkw, connection, tableName, excludeOfflinedSplitParents,
+      false);
+  }
 
+  /**
+   * @param zkw ZooKeeperWatcher instance we're using to get hbase:meta location
+   * @param connection connection we're using
+   * @param tableName table to work with
+   * @param excludeOfflinedSplitParents Exclude offline regions
+   * @param excludeReplicaRegions If true, do not include replica regions in the result.
+   * @return List of regioninfos and server addresses.
+   */
+  public static List<Pair<HRegionInfo, ServerName>> getTableRegionsAndLocations(
+      ZooKeeperWatcher zkw, Connection connection, final TableName tableName,
+      final boolean excludeOfflinedSplitParents, final boolean excludeReplicaRegions)
+          throws IOException {
     if (tableName.equals(TableName.META_TABLE_NAME)) {
       // If meta, do a bit of special handling.
       ServerName serverName = new MetaTableLocator().getMetaRegionLocation(zkw);
@@ -514,6 +544,11 @@ public class MetaTableAccessor {
           }
           for (HRegionLocation loc : current.getRegionLocations()) {
             if (loc != null) {
+              // Exclude replica region from the result
+              if (excludeReplicaRegions
+                  && !RegionReplicaUtil.isDefaultReplica(loc.getRegionInfo())) {
+                continue;
+              }
               this.results.add(new Pair<HRegionInfo, ServerName>(
                 loc.getRegionInfo(), loc.getServerName()));
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cd1726f5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
index 5c03a4a..1846cef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
@@ -149,17 +149,24 @@ public final class ProcedureSyncWait {
 
   protected static List<HRegionInfo> getRegionsFromMeta(final MasterProcedureEnv env,
       final TableName tableName) throws IOException {
+    return getRegionsFromMeta(env, tableName, false, false);
+  }
+
+  protected static List<HRegionInfo> getRegionsFromMeta(final MasterProcedureEnv env,
+      final TableName tableName, final boolean excludeOfflinedSplitParents,
+      final boolean excludeReplicaRegions) throws IOException {
     return ProcedureSyncWait.waitFor(env, "regions of table=" + tableName + " from meta",
-        new ProcedureSyncWait.Predicate<List<HRegionInfo>>() {
-      @Override
-      public List<HRegionInfo> evaluate() throws IOException {
-        if (TableName.META_TABLE_NAME.equals(tableName)) {
-          return new MetaTableLocator().getMetaRegions(env.getMasterServices().getZooKeeper());
+      new ProcedureSyncWait.Predicate<List<HRegionInfo>>() {
+        @Override
+        public List<HRegionInfo> evaluate() throws IOException {
+          if (TableName.META_TABLE_NAME.equals(tableName)) {
+            return new MetaTableLocator().getMetaRegions(env.getMasterServices().getZooKeeper());
+          }
+          return MetaTableAccessor.getTableRegions(env.getMasterServices().getZooKeeper(),
+            env.getMasterServices().getConnection(), tableName, excludeOfflinedSplitParents,
+            excludeReplicaRegions);
         }
-        return MetaTableAccessor.getTableRegions(env.getMasterServices().getZooKeeper(),
-            env.getMasterServices().getConnection(), tableName);
-      }
-    });
+      });
   }
 
   protected static void waitRegionInTransition(final MasterProcedureEnv env,

http://git-wip-us.apache.org/repos/asf/hbase/blob/cd1726f5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
index 9dccef6..da539a1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
@@ -85,7 +85,7 @@ public class TruncateTableProcedure
 
           // TODO: Move out... in the acquireLock()
           LOG.debug("waiting for '" + getTableName() + "' regions in transition");
-          regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+          regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName(), true, true);
           assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
           ProcedureSyncWait.waitRegionInTransition(env, regions);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/cd1726f5/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index fa57b12..ff479d4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -74,6 +74,11 @@ public class MasterProcedureTestingUtility {
   public static HRegionInfo[] createTable(final ProcedureExecutor<MasterProcedureEnv> procExec,
       final TableName tableName, final byte[][] splitKeys, String... family) throws IOException {
     HTableDescriptor htd = createHTD(tableName, family);
+    return createTable(procExec, htd, splitKeys);
+  }
+
+  public static HRegionInfo[] createTable(final ProcedureExecutor<MasterProcedureEnv> procExec,
+      HTableDescriptor htd, final byte[][] splitKeys) throws IOException {
     HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
     long procId = ProcedureTestingUtility.submitAndWait(procExec,
       new CreateTableProcedure(procExec.getEnvironment(), htd, regions));

http://git-wip-us.apache.org/repos/asf/hbase/blob/cd1726f5/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
index 59126c5..edcfb5b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
@@ -43,6 +43,8 @@ import org.junit.experimental.categories.Category;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
+
 @Category(MediumTests.class)
 public class TestTruncateTableProcedure {
   private static final Log LOG = LogFactory.getLog(TestTruncateTableProcedure.class);
@@ -240,4 +242,52 @@ public class TestTruncateTableProcedure {
   private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
     return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
   }
+
+  @Test(timeout = 60000)
+  public void testTruncateWithPreserveAfterSplit() throws Exception {
+    final String[] families = new String[] { "f1", "f2" };
+    final byte[][] splitKeys =
+        new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c") };
+    TableName tableName = TableName.valueOf("testTruncateWithPreserveAfterSplit");
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(getMasterProcedureExecutor(),
+      tableName, splitKeys, families);
+    splitAndTruncate(families, splitKeys, tableName, regions);
+  }
+
+  @Test(timeout = 60000)
+  public void testTruncatePreserveWithReplicaRegionAfterSplit() throws Exception {
+    final String[] families = new String[] { "f1", "f2" };
+    final byte[][] splitKeys =
+        new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c") };
+    TableName tableName = TableName.valueOf("testTruncateWithPreserveAfterSplit");
+    HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, families);
+    htd.setRegionReplication(3);
+    HRegionInfo[] regions =
+        MasterProcedureTestingUtility.createTable(getMasterProcedureExecutor(), htd, splitKeys);
+    splitAndTruncate(families, splitKeys, tableName, regions);
+  }
+
+  private void splitAndTruncate(final String[] families, final byte[][] splitKeys,
+      TableName tableName, HRegionInfo[] regions) throws IOException, InterruptedException {
+    // load enough data so the table can split
+    MasterProcedureTestingUtility.loadData(UTIL.getConnection(), tableName, 5000, splitKeys,
+      families);
+    assertEquals(5000, UTIL.countRows(tableName));
+    UTIL.getHBaseAdmin().split(tableName);
+    UTIL.waitUntilAllRegionsAssigned(tableName);
+    // wait until split really happens
+    while (UTIL.getHBaseAdmin().getTableRegions(tableName).size() <= regions.length) {
+      Thread.sleep(50);
+    }
+    // disable the table
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    // truncate the table
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    long procId = ProcedureTestingUtility.submitAndWait(procExec,
+      new TruncateTableProcedure(procExec.getEnvironment(), tableName, true));
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+
+    UTIL.waitUntilAllRegionsAssigned(tableName);
+  }
+
 }


[50/50] [abbrv] hbase git commit: HBASE-21546 ConnectException in TestThriftHttpServer

Posted by ap...@apache.org.
HBASE-21546 ConnectException in TestThriftHttpServer

Signed-off-by: Sean Busbey <bu...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: e063aa8db09d8e8224a923d01aa027742f253aff
Parents: 82f187e
Author: Peter Somogyi <ps...@apache.org>
Authored: Tue Dec 4 14:57:50 2018 +0100
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Dec 13 10:25:10 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java | 8 ++------
 1 file changed, 2 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e063aa8d/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
index b21de38..8ed2259 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
@@ -185,11 +185,7 @@ public class TestThriftHttpServer {
 
   private void waitThriftServerStartup() throws Exception{
     // wait up to 10s for the server to start
-    for (int i = 0; i < 100
-        && ( thriftServer.serverRunner == null ||  thriftServer.serverRunner.httpServer ==
-        null); i++) {
-      Thread.sleep(100);
-    }
+    HBaseTestingUtility.waitForHostPort(HConstants.LOCALHOST, port);
   }
 
   private void runThriftServer(int customHeaderSize) throws Exception {
@@ -252,7 +248,7 @@ public class TestThriftHttpServer {
   }
 
   private void stopHttpServerThread() throws Exception {
-    LOG.debug("Stopping " + " Thrift HTTP server");
+    LOG.debug("Stopping Thrift HTTP server");
     thriftServer.stop();
     httpServerThread.join();
     if (httpServerException != null) {


[40/50] [abbrv] hbase git commit: HBASE-20723 Custom hbase.wal.dir results in data loss because we write recovered edits into a different place than where the recovering region server looks for them

Posted by ap...@apache.org.
HBASE-20723 Custom hbase.wal.dir results in data loss because we write recovered edits into a different place than where the recovering region server looks for them

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java


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

Branch: refs/heads/branch-1.3
Commit: 2702f23a4addaa459d6f221c0008be0c556f672d
Parents: 979438d
Author: tedyu <yu...@gmail.com>
Authored: Sat Jun 16 10:34:47 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 19:12:42 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/wal/DefaultWALProvider.java    |  8 +--
 .../apache/hadoop/hbase/wal/WALSplitter.java    | 60 ++++++++--------
 .../hbase/regionserver/wal/TestWALReplay.java   | 72 ++++++++++----------
 .../apache/hadoop/hbase/wal/TestWALFactory.java |  3 +-
 .../apache/hadoop/hbase/wal/TestWALSplit.java   |  8 +--
 5 files changed, 77 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2702f23a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
index 01234a1..9cb5da0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
@@ -18,9 +18,6 @@
  */
 package org.apache.hadoop.hbase.wal;
 
-import java.io.Closeable;
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -370,8 +367,9 @@ public class DefaultWALProvider implements WALProvider {
         ProtobufLogWriter.class, Writer.class);
     Writer writer = null;
     try {
-      writer = logWriterClass.newInstance();
-      writer.init(fs, path, conf, overwritable);
+      writer = logWriterClass.getDeclaredConstructor().newInstance();
+      FileSystem rootFs = FileSystem.get(path.toUri(), conf);
+      writer.init(rootFs, path, conf, overwritable);
       return writer;
     } catch (Exception e) {
       LOG.debug("Error instantiating log writer.", e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2702f23a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 50bb79f..c88e6d3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -140,7 +140,7 @@ public class WALSplitter {
   public static final boolean SPLIT_SKIP_ERRORS_DEFAULT = false;
 
   // Parameters for split process
-  protected final Path rootDir;
+  protected final Path walDir;
   protected final FileSystem fs;
   protected final Configuration conf;
 
@@ -187,14 +187,14 @@ public class WALSplitter {
   public final static String SPLIT_WRITER_CREATION_BOUNDED = "hbase.split.writer.creation.bounded";
 
   @VisibleForTesting
-  WALSplitter(final WALFactory factory, Configuration conf, Path rootDir,
+  WALSplitter(final WALFactory factory, Configuration conf, Path walDir,
       FileSystem fs, LastSequenceId idChecker,
       CoordinatedStateManager csm, RecoveryMode mode) {
     this.conf = HBaseConfiguration.create(conf);
     String codecClassName = conf
         .get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName());
     this.conf.set(HConstants.RPC_CODEC_CONF_KEY, codecClassName);
-    this.rootDir = rootDir;
+    this.walDir = walDir;
     this.fs = fs;
     this.sequenceIdChecker = idChecker;
     this.csm = (BaseCoordinatedStateManager)csm;
@@ -244,10 +244,10 @@ public class WALSplitter {
    * @return false if it is interrupted by the progress-able.
    * @throws IOException
    */
-  public static boolean splitLogFile(Path rootDir, FileStatus logfile, FileSystem fs,
+  public static boolean splitLogFile(Path walDir, FileStatus logfile, FileSystem fs,
       Configuration conf, CancelableProgressable reporter, LastSequenceId idChecker,
       CoordinatedStateManager cp, RecoveryMode mode, final WALFactory factory) throws IOException {
-    WALSplitter s = new WALSplitter(factory, conf, rootDir, fs, idChecker, cp, mode);
+    WALSplitter s = new WALSplitter(factory, conf, walDir, fs, idChecker, cp, mode);
     return s.splitLogFile(logfile, reporter);
   }
 
@@ -315,7 +315,7 @@ public class WALSplitter {
         in = getReader(logfile, skipErrors, reporter);
       } catch (CorruptedLogFileException e) {
         LOG.warn("Could not get reader, corrupted log file " + logPath, e);
-        ZKSplitLog.markCorrupted(rootDir, logfile.getPath().getName(), fs);
+        ZKSplitLog.markCorrupted(walDir, logfile.getPath().getName(), fs);
         isCorrupted = true;
       }
       if (in == null) {
@@ -406,7 +406,7 @@ public class WALSplitter {
       throw iie;
     } catch (CorruptedLogFileException e) {
       LOG.warn("Could not parse, corrupted log file " + logPath, e);
-      csm.getSplitLogWorkerCoordination().markCorrupted(rootDir,
+      csm.getSplitLogWorkerCoordination().markCorrupted(walDir,
         logfile.getPath().getName(), fs);
       isCorrupted = true;
     } catch (IOException e) {
@@ -537,18 +537,19 @@ public class WALSplitter {
    * <code>logEntry</code>: e.g. /hbase/some_table/2323432434/recovered.edits/2332.
    * This method also ensures existence of RECOVERED_EDITS_DIR under the region
    * creating it if necessary.
-   * @param fs
    * @param logEntry
-   * @param rootDir HBase root dir.
-   * @param fileBeingSplit the file being split currently. Used to generate tmp file name.
+   * @param fileNameBeingSplit the file being split currently. Used to generate tmp file name.
+   * @param conf
    * @return Path to file into which to dump split log edits.
    * @throws IOException
    */
   @SuppressWarnings("deprecation")
   @VisibleForTesting
-  static Path getRegionSplitEditsPath(final FileSystem fs,
-      final Entry logEntry, final Path rootDir, String fileBeingSplit)
+  static Path getRegionSplitEditsPath(final Entry logEntry, String fileNameBeingSplit,
+      Configuration conf)
   throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    Path rootDir = FSUtils.getRootDir(conf);
     Path tableDir = FSUtils.getTableDir(rootDir, logEntry.getKey().getTablename());
     String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
     Path regiondir = HRegion.getRegionDir(tableDir, encodedRegionName);
@@ -582,7 +583,7 @@ public class WALSplitter {
     // Append file name ends with RECOVERED_LOG_TMPFILE_SUFFIX to ensure
     // region's replayRecoveredEdits will not delete it
     String fileName = formatRecoveredEditsFileName(logEntry.getKey().getLogSeqNum());
-    fileName = getTmpRecoveredEditsFileName(fileName + "-" + fileBeingSplit);
+    fileName = getTmpRecoveredEditsFileName(fileName + "-" + fileNameBeingSplit);
     return new Path(dir, fileName);
   }
 
@@ -1354,7 +1355,8 @@ public class WALSplitter {
     }
 
     // delete the one with fewer wal entries
-    void deleteOneWithFewerEntries(WriterAndPath wap, Path dst) throws IOException {
+    void deleteOneWithFewerEntries(FileSystem rootFs, WriterAndPath wap, Path dst)
+        throws IOException {
       long dstMinLogSeqNum = -1L;
       try (WAL.Reader reader = walFactory.createReader(fs, dst)) {
         WAL.Entry entry = reader.next();
@@ -1378,8 +1380,8 @@ public class WALSplitter {
         }
       } else {
         LOG.warn("Found existing old edits file and we have less entries. Deleting " + wap.p
-            + ", length=" + fs.getFileStatus(wap.p).getLen());
-        if (!fs.delete(wap.p, false)) {
+            + ", length=" + rootFs.getFileStatus(wap.p).getLen());
+        if (!rootFs.delete(wap.p, false)) {
           LOG.warn("Failed deleting of " + wap.p);
           throw new IOException("Failed deleting of " + wap.p);
         }
@@ -1463,6 +1465,7 @@ public class WALSplitter {
       if (LOG.isTraceEnabled()) {
         LOG.trace("Closing " + wap.p);
       }
+      FileSystem rootFs = FileSystem.get(conf);
       try {
         wap.w.close();
       } catch (IOException ioe) {
@@ -1477,7 +1480,7 @@ public class WALSplitter {
       }
       if (wap.editsWritten == 0) {
         // just remove the empty recovered.edits file
-        if (fs.exists(wap.p) && !fs.delete(wap.p, false)) {
+        if (rootFs.exists(wap.p) && !rootFs.delete(wap.p, false)) {
           LOG.warn("Failed deleting empty " + wap.p);
           throw new IOException("Failed deleting empty  " + wap.p);
         }
@@ -1487,14 +1490,14 @@ public class WALSplitter {
       Path dst = getCompletedRecoveredEditsFilePath(wap.p,
           regionMaximumEditLogSeqNum.get(encodedRegionName));
       try {
-        if (!dst.equals(wap.p) && fs.exists(dst)) {
-          deleteOneWithFewerEntries(wap, dst);
+        if (!dst.equals(wap.p) && rootFs.exists(dst)) {
+          deleteOneWithFewerEntries(rootFs, wap, dst);
         }
         // Skip the unit tests which create a splitter that reads and
         // writes the data without touching disk.
         // TestHLogSplit#testThreading is an example.
-        if (fs.exists(wap.p)) {
-          if (!fs.rename(wap.p, dst)) {
+        if (rootFs.exists(wap.p)) {
+          if (!rootFs.rename(wap.p, dst)) {
             throw new IOException("Failed renaming " + wap.p + " to " + dst);
           }
           LOG.info("Rename " + wap.p + " to " + dst);
@@ -1568,7 +1571,7 @@ public class WALSplitter {
       if (blacklistedRegions.contains(region)) {
         return null;
       }
-      ret = createWAP(region, entry, rootDir);
+      ret = createWAP(region, entry);
       if (ret == null) {
         blacklistedRegions.add(region);
         return null;
@@ -1583,17 +1586,18 @@ public class WALSplitter {
     /**
      * @return a path with a write for that path. caller should close.
      */
-    WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir) throws IOException {
-      Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir,
-          fileBeingSplit.getPath().getName());
+    WriterAndPath createWAP(byte[] region, Entry entry) throws IOException {
+      Path regionedits = getRegionSplitEditsPath(entry,
+          fileBeingSplit.getPath().getName(), conf);
       if (regionedits == null) {
         return null;
       }
-      if (fs.exists(regionedits)) {
+      FileSystem rootFs = FileSystem.get(conf);
+      if (rootFs.exists(regionedits)) {
         LOG.warn("Found old edits file. It could be the "
             + "result of a previous failed split attempt. Deleting " + regionedits + ", length="
-            + fs.getFileStatus(regionedits).getLen());
-        if (!fs.delete(regionedits, false)) {
+            + rootFs.getFileStatus(regionedits).getLen());
+        if (!rootFs.delete(regionedits, false)) {
           LOG.warn("Failed delete of old " + regionedits);
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2702f23a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
index 2622f6d..a196f57 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
@@ -307,11 +307,11 @@ public class TestWALReplay {
 
     MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    Path basedir = FSUtils.getTableDir(hbaseWALRootDir, tableName);
+    Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
     deleteDir(basedir);
 
     HTableDescriptor htd = createBasic3FamilyHTD(tableName);
-    HRegion region2 = HRegion.createHRegion(hri, hbaseWALRootDir, this.conf, htd);
+    HRegion region2 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
     final byte [] rowName = tableName.getName();
 
@@ -337,7 +337,7 @@ public class TestWALReplay {
 
     WAL wal3 = createWAL(this.conf);
     try {
-      HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal3);
+      HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal3);
       long seqid = region.getOpenSeqNum();
       // The regions opens with sequenceId as 1. With 6k edits, its sequence number reaches 6k + 1.
       // When opened, this region would apply 6k edits, and increment the sequenceId by 1
@@ -369,13 +369,13 @@ public class TestWALReplay {
     final TableName tableName =
         TableName.valueOf("testRegionMadeOfBulkLoadedFilesOnly");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = new Path(this.hbaseWALRootDir, tableName.getNameAsString());
+    final Path basedir = new Path(this.hbaseRootDir, tableName.getNameAsString());
     deleteDir(basedir);
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
-    HRegion region2 = HRegion.createHRegion(hri, hbaseWALRootDir, this.conf, htd);
+    HRegion region2 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
     WAL wal = createWAL(this.conf);
-    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
 
     byte [] family = htd.getFamilies().iterator().next().getName();
     Path f =  new Path(basedir, "hfile");
@@ -404,7 +404,7 @@ public class TestWALReplay {
         WAL wal2 = createWAL(newConf);
 
         HRegion region2 = HRegion.openHRegion(newConf, FileSystem.get(newConf),
-          hbaseWALRootDir, hri, htd, wal2);
+          hbaseRootDir, hri, htd, wal2);
         long seqid2 = region2.getOpenSeqNum();
         assertTrue(seqid2 > -1);
         assertEquals(rowsInsertedCount, getScannedCount(region2.getScanner(new Scan())));
@@ -435,14 +435,14 @@ public class TestWALReplay {
     final TableName tableName =
         TableName.valueOf("testCompactedBulkLoadedFiles");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = new Path(this.hbaseWALRootDir, tableName.getNameAsString());
+    final Path basedir = new Path(this.hbaseRootDir, tableName.getNameAsString());
     deleteDir(basedir);
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region2 = HRegion.createHRegion(hri,
-        hbaseWALRootDir, this.conf, htd);
+        hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
     WAL wal = createWAL(this.conf);
-    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
 
     // Add an edit so something in the WAL
     byte [] row = tableName.getName();
@@ -476,7 +476,7 @@ public class TestWALReplay {
         WAL wal2 = createWAL(newConf);
 
         HRegion region2 = HRegion.openHRegion(newConf, FileSystem.get(newConf),
-            hbaseWALRootDir, hri, htd, wal2);
+            hbaseRootDir, hri, htd, wal2);
         long seqid2 = region2.getOpenSeqNum();
         assertTrue(seqid2 > -1);
         assertEquals(rowsInsertedCount, getScannedCount(region2.getScanner(new Scan())));
@@ -506,19 +506,19 @@ public class TestWALReplay {
     final TableName tableName =
         TableName.valueOf("testReplayEditsWrittenViaHRegion");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(this.hbaseWALRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region3 = HRegion.createHRegion(hri,
-            hbaseWALRootDir, this.conf, htd);
+            hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region3);
     // Write countPerFamily edits into the three families.  Do a flush on one
     // of the families during the load of edits so its seqid is not same as
     // others to test we do right thing when different seqids.
     WAL wal = createWAL(this.conf);
-    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
     long seqid = region.getOpenSeqNum();
     boolean first = true;
     for (HColumnDescriptor hcd: htd.getFamilies()) {
@@ -541,7 +541,7 @@ public class TestWALReplay {
     wal.shutdown();
     runWALSplit(this.conf);
     WAL wal2 = createWAL(this.conf);
-    HRegion region2 = HRegion.openHRegion(conf, this.fs, hbaseWALRootDir, hri, htd, wal2);
+    HRegion region2 = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal2);
     long seqid2 = region2.getOpenSeqNum();
     assertTrue(seqid + result.size() < seqid2);
     final Result result1b = region2.get(g);
@@ -616,19 +616,19 @@ public class TestWALReplay {
     final TableName tableName =
         TableName.valueOf("testReplayEditsWrittenViaHRegion");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(this.hbaseWALRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region3 = HRegion.createHRegion(hri,
-            hbaseWALRootDir, this.conf, htd);
+            hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region3);
     // Write countPerFamily edits into the three families.  Do a flush on one
     // of the families during the load of edits so its seqid is not same as
     // others to test we do right thing when different seqids.
     WAL wal = createWAL(this.conf);
-    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
     long seqid = region.getOpenSeqNum();
     for (HColumnDescriptor hcd: htd.getFamilies()) {
       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
@@ -661,7 +661,7 @@ public class TestWALReplay {
     // Let us try to split and recover
     runWALSplit(this.conf);
     WAL wal2 = createWAL(this.conf);
-    HRegion region2 = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal2);
+    HRegion region2 = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal2);
     long seqid2 = region2.getOpenSeqNum();
     assertTrue(seqid + result.size() < seqid2);
 
@@ -701,10 +701,10 @@ public class TestWALReplay {
     final TableName tableName =
         TableName.valueOf("testReplayEditsAfterAbortingFlush");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(this.hbaseWALRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
-    HRegion region3 = HRegion.createHRegion(hri, hbaseWALRootDir, this.conf, htd);
+    HRegion region3 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd);
     region3.close();
     region3.getWAL().close();
     // Write countPerFamily edits into the three families. Do a flush on one
@@ -718,7 +718,7 @@ public class TestWALReplay {
     customConf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY,
         CustomStoreFlusher.class.getName());
     HRegion region =
-      HRegion.openHRegion(this.hbaseWALRootDir, hri, htd, wal, customConf, rsServices, null);
+      HRegion.openHRegion(this.hbaseRootDir, hri, htd, wal, customConf, rsServices, null);
     int writtenRowCount = 10;
     List<HColumnDescriptor> families = new ArrayList<HColumnDescriptor>(
         htd.getFamilies());
@@ -772,7 +772,7 @@ public class TestWALReplay {
     WAL wal2 = createWAL(this.conf);
     Mockito.doReturn(false).when(rsServices).isAborted();
     HRegion region2 =
-      HRegion.openHRegion(this.hbaseWALRootDir, hri, htd, wal2, this.conf, rsServices, null);
+      HRegion.openHRegion(this.hbaseRootDir, hri, htd, wal2, this.conf, rsServices, null);
     scanner = region2.getScanner(new Scan());
     assertEquals(writtenRowCount, getScannedCount(scanner));
   }
@@ -802,12 +802,12 @@ public class TestWALReplay {
         TableName.valueOf("testReplayEditsWrittenIntoWAL");
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(hbaseWALRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
     deleteDir(basedir);
 
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region2 = HRegion.createHRegion(hri,
-            hbaseWALRootDir, this.conf, htd);
+            hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
     final WAL wal = createWAL(this.conf);
     final byte[] rowName = tableName.getName();
@@ -901,7 +901,7 @@ public class TestWALReplay {
     final TableName tableName = TableName.valueOf(currentTest.getMethodName());
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
     final Path basedir =
-        FSUtils.getTableDir(this.hbaseWALRootDir, tableName);
+        FSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
@@ -910,7 +910,7 @@ public class TestWALReplay {
     // Mock the WAL
     MockWAL wal = createMockWAL();
 
-    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
     for (HColumnDescriptor hcd : htd.getFamilies()) {
       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
     }
@@ -934,7 +934,7 @@ public class TestWALReplay {
     WALSplitter.splitLogFile(hbaseWALRootDir, listStatus[0],
         this.fs, this.conf, null, null, null, mode, wals);
     FileStatus[] listStatus1 = this.fs.listStatus(
-      new Path(FSUtils.getTableDir(hbaseWALRootDir, tableName), new Path(hri.getEncodedName(),
+      new Path(FSUtils.getTableDir(hbaseRootDir, tableName), new Path(hri.getEncodedName(),
           "recovered.edits")), new PathFilter() {
         @Override
         public boolean accept(Path p) {
@@ -962,17 +962,17 @@ public class TestWALReplay {
       IllegalAccessException {
     final TableName tableName = TableName.valueOf("testDatalossWhenInputError");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(this.hbaseWALRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
     final HTableDescriptor htd = createBasic1FamilyHTD(tableName);
-    HRegion region1 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseWALRootDir, this.hbaseWALRootDir, this.conf, htd);
+    HRegion region1 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.hbaseWALRootDir, this.conf, htd);
     Path regionDir = region1.getRegionFileSystem().getRegionDir();
     HBaseTestingUtility.closeRegionAndWAL(region1);
 
     WAL wal = createWAL(this.conf);
-    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseWALRootDir, hri, htd, wal);
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
     for (HColumnDescriptor hcd : htd.getFamilies()) {
       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
     }
@@ -1042,12 +1042,12 @@ public class TestWALReplay {
     HRegion region2;
     try {
       // log replay should fail due to the IOException, otherwise we may lose data.
-      region2 = HRegion.openHRegion(conf, spyFs, hbaseWALRootDir, hri, htd, wal2);
+      region2 = HRegion.openHRegion(conf, spyFs, hbaseRootDir, hri, htd, wal2);
       assertEquals(result.size(), region2.get(g).size());
     } catch (IOException e) {
       assertEquals("read over limit", e.getMessage());
     }
-    region2 = HRegion.openHRegion(conf, fs, hbaseWALRootDir, hri, htd, wal2);
+    region2 = HRegion.openHRegion(conf, fs, hbaseRootDir, hri, htd, wal2);
     assertEquals(result.size(), region2.get(g).size());
   }
 
@@ -1058,11 +1058,11 @@ public class TestWALReplay {
     final TableName tableName = TableName.valueOf("testReplayEditsWrittenIntoWAL");
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(hbaseWALRootDir, tableName);
+    final Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
     deleteDir(basedir);
 
     final HTableDescriptor htd = createBasic1FamilyHTD(tableName);
-    HRegion region = HBaseTestingUtility.createRegionAndWAL(hri, hbaseWALRootDir, this.hbaseWALRootDir, this.conf, htd);
+    HRegion region = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.hbaseWALRootDir, this.conf, htd);
     HBaseTestingUtility.closeRegionAndWAL(region);
     final byte[] family = htd.getColumnFamilies()[0].getName();
     final byte[] rowName = tableName.getName();
@@ -1086,7 +1086,7 @@ public class TestWALReplay {
     WALSplitter.splitLogFile(hbaseWALRootDir, second, fs, conf, null, null, null,
       RecoveryMode.LOG_SPLITTING, wals);
     WAL wal = createWAL(this.conf);
-    region = HRegion.openHRegion(conf, this.fs, hbaseWALRootDir, hri, htd, wal);
+    region = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal);
     assertTrue(region.getOpenSeqNum() > mvcc.getWritePoint());
     assertEquals(2, region.get(new Get(rowName)).size());
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2702f23a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
index 9b4a968..3c0bade 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
@@ -118,6 +118,7 @@ public class TestWALFactory {
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
+    FSUtils.setWALRootDir(TEST_UTIL.getConfiguration(), new Path("file:///tmp/wal"));
     // Make block sizes small.
     TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
     // needed for testAppendClose()
@@ -171,7 +172,7 @@ public class TestWALFactory {
     Path oldLogDir = new Path(hbaseWALDir, HConstants.HREGION_OLDLOGDIR_NAME);
     final int howmany = 3;
     HRegionInfo[] infos = new HRegionInfo[3];
-    Path tabledir = FSUtils.getTableDir(hbaseWALDir, tableName);
+    Path tabledir = FSUtils.getTableDir(hbaseDir, tableName);
     fs.mkdirs(tabledir);
     for(int i = 0; i < howmany; i++) {
       infos[i] = new HRegionInfo(tableName,

http://git-wip-us.apache.org/repos/asf/hbase/blob/2702f23a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
index 3ae155c..0d0bbec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -385,8 +385,8 @@ public class TestWALSplit {
         new Entry(new WALKey(encoded,
             TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
             new WALEdit());
-    Path p = WALSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR,
-        FILENAME_BEING_SPLIT);
+    Path p = WALSplitter.getRegionSplitEditsPath(entry,
+        FILENAME_BEING_SPLIT, conf);
     String parentOfParent = p.getParent().getParent().getName();
     assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
   }
@@ -411,8 +411,8 @@ public class TestWALSplit {
     assertEquals(parent.getName(), HConstants.RECOVERED_EDITS_DIR);
     fs.createNewFile(parent); // create a recovered.edits file
 
-    Path p = WALSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR,
-        FILENAME_BEING_SPLIT);
+    Path p = WALSplitter.getRegionSplitEditsPath(entry,
+        FILENAME_BEING_SPLIT, conf);
     String parentOfParent = p.getParent().getParent().getName();
     assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
     WALFactory.createRecoveredEditsWriter(fs, p, conf).close();


[34/50] [abbrv] hbase git commit: HBASE-20559 Backport HBASE-18083 (Make large/small file clean thread number configurable in HFileCleaner) to branch-1

Posted by ap...@apache.org.
HBASE-20559 Backport HBASE-18083 (Make large/small file clean thread number configurable in HFileCleaner) to branch-1

The last port commit of HBASE-20555

Signed-off-by: Yu Li <li...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 2434162594f22df0bba94bc40186bd5628501c8e
Parents: 976f07e
Author: TAK LON WU <wu...@amazon.com>
Authored: Mon Jul 23 11:23:57 2018 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:20 2018 -0800

----------------------------------------------------------------------
 .../hbase/master/cleaner/HFileCleaner.java      | 154 +++++++++++++------
 .../hbase/master/cleaner/TestHFileCleaner.java  |  13 +-
 2 files changed, 120 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/24341625/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
index 70548b4..8f0b4be 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -66,6 +67,16 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       "hbase.regionserver.hfilecleaner.small.queue.size";
   public final static int DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE = 10240;
 
+  // Configuration key for large file delete thread number
+  public final static String LARGE_HFILE_DELETE_THREAD_NUMBER =
+      "hbase.regionserver.hfilecleaner.large.thread.count";
+  public final static int DEFAULT_LARGE_HFILE_DELETE_THREAD_NUMBER = 1;
+
+  // Configuration key for small file delete thread number
+  public final static String SMALL_HFILE_DELETE_THREAD_NUMBER =
+      "hbase.regionserver.hfilecleaner.small.thread.count";
+  public final static int DEFAULT_SMALL_HFILE_DELETE_THREAD_NUMBER = 1;
+
   private static final Log LOG = LogFactory.getLog(HFileCleaner.class);
 
   StealJobQueue<HFileDeleteTask> largeFileQueue;
@@ -73,11 +84,13 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   private int throttlePoint;
   private int largeQueueInitSize;
   private int smallQueueInitSize;
+  private int largeFileDeleteThreadNumber;
+  private int smallFileDeleteThreadNumber;
   private List<Thread> threads = new ArrayList<Thread>();
   private boolean running;
 
-  private long deletedLargeFiles = 0L;
-  private long deletedSmallFiles = 0L;
+  private AtomicLong deletedLargeFiles = new AtomicLong();
+  private AtomicLong deletedSmallFiles = new AtomicLong();
 
   /**
    * @param period the period of time to sleep between each run
@@ -99,6 +112,10 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
         conf.getInt(SMALL_HFILE_QUEUE_INIT_SIZE, DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE);
     largeFileQueue = new StealJobQueue<>(largeQueueInitSize, smallQueueInitSize);
     smallFileQueue = largeFileQueue.getStealFromQueue();
+    largeFileDeleteThreadNumber =
+        conf.getInt(LARGE_HFILE_DELETE_THREAD_NUMBER, DEFAULT_LARGE_HFILE_DELETE_THREAD_NUMBER);
+    smallFileDeleteThreadNumber =
+        conf.getInt(SMALL_HFILE_DELETE_THREAD_NUMBER, DEFAULT_SMALL_HFILE_DELETE_THREAD_NUMBER);
     startHFileDeleteThreads();
   }
 
@@ -182,30 +199,34 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
     final String n = Thread.currentThread().getName();
     running = true;
     // start thread for large file deletion
-    Thread large = new Thread() {
-      @Override
-      public void run() {
-        consumerLoop(largeFileQueue);
-      }
-    };
-    large.setDaemon(true);
-    large.setName(n + "-HFileCleaner.large-" + System.currentTimeMillis());
-    large.start();
-    LOG.debug("Starting hfile cleaner for large files: " + large.getName());
-    threads.add(large);
+    for (int i = 0; i < largeFileDeleteThreadNumber; i++) {
+      Thread large = new Thread() {
+        @Override
+        public void run() {
+          consumerLoop(largeFileQueue);
+        }
+      };
+      large.setDaemon(true);
+      large.setName(n + "-HFileCleaner.large." + i + "-" + System.currentTimeMillis());
+      large.start();
+      LOG.debug("Starting hfile cleaner for large files: " + large.getName());
+      threads.add(large);
+    }
 
     // start thread for small file deletion
-    Thread small = new Thread() {
-      @Override
-      public void run() {
-        consumerLoop(smallFileQueue);
-      }
-    };
-    small.setDaemon(true);
-    small.setName(n + "-HFileCleaner.small-" + System.currentTimeMillis());
-    small.start();
-    LOG.debug("Starting hfile cleaner for small files: " + small.getName());
-    threads.add(small);
+    for (int i = 0; i < smallFileDeleteThreadNumber; i++) {
+      Thread small = new Thread() {
+        @Override
+        public void run() {
+          consumerLoop(smallFileQueue);
+        }
+      };
+      small.setDaemon(true);
+      small.setName(n + "-HFileCleaner.small." + i + "-" + System.currentTimeMillis());
+      small.start();
+      LOG.debug("Starting hfile cleaner for small files: " + small.getName());
+      threads.add(small);
+    }
   }
 
   protected void consumerLoop(BlockingQueue<HFileDeleteTask> queue) {
@@ -247,20 +268,20 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   // Currently only for testing purpose
   private void countDeletedFiles(boolean isLargeFile, boolean fromLargeQueue) {
     if (isLargeFile) {
-      if (deletedLargeFiles == Long.MAX_VALUE) {
+      if (deletedLargeFiles.get() == Long.MAX_VALUE) {
         LOG.info("Deleted more than Long.MAX_VALUE large files, reset counter to 0");
-        deletedLargeFiles = 0L;
+        deletedLargeFiles.set(0L);
       }
-      deletedLargeFiles++;
+      deletedLargeFiles.incrementAndGet();
     } else {
-      if (deletedSmallFiles == Long.MAX_VALUE) {
+      if (deletedSmallFiles.get() == Long.MAX_VALUE) {
         LOG.info("Deleted more than Long.MAX_VALUE small files, reset counter to 0");
-        deletedSmallFiles = 0L;
+        deletedSmallFiles.set(0L);
       }
       if (fromLargeQueue && LOG.isTraceEnabled()) {
         LOG.trace("Stolen a small file deletion task in large file thread");
       }
-      deletedSmallFiles++;
+      deletedSmallFiles.incrementAndGet();
     }
   }
 
@@ -353,12 +374,12 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
 
   @VisibleForTesting
   public long getNumOfDeletedLargeFiles() {
-    return deletedLargeFiles;
+    return deletedLargeFiles.get();
   }
 
   @VisibleForTesting
   public long getNumOfDeletedSmallFiles() {
-    return deletedSmallFiles;
+    return deletedSmallFiles.get();
   }
 
   @VisibleForTesting
@@ -378,19 +399,14 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
 
   @Override
   public void onConfigurationChange(Configuration conf) {
-    StringBuilder builder = new StringBuilder();
-    builder.append("Updating configuration for HFileCleaner, previous throttle point: ")
-        .append(throttlePoint).append(", largeQueueInitSize: ").append(largeQueueInitSize)
-        .append(", smallQueueInitSize: ").append(smallQueueInitSize);
+    if (!checkAndUpdateConfigurations(conf)) {
+      LOG.debug("Update configuration triggered but nothing changed for this cleaner");
+      return;
+    }
     stopHFileDeleteThreads();
-    this.throttlePoint =
-        conf.getInt(HFILE_DELETE_THROTTLE_THRESHOLD, DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD);
-    this.largeQueueInitSize =
-        conf.getInt(LARGE_HFILE_QUEUE_INIT_SIZE, DEFAULT_LARGE_HFILE_QUEUE_INIT_SIZE);
-    this.smallQueueInitSize =
-        conf.getInt(SMALL_HFILE_QUEUE_INIT_SIZE, DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE);
     // record the left over tasks
-    List<HFileDeleteTask> leftOverTasks = new ArrayList<>();
+    List<HFileDeleteTask> leftOverTasks =
+        new ArrayList<>(largeFileQueue.size() + smallFileQueue.size());
     for (HFileDeleteTask task : largeFileQueue) {
       leftOverTasks.add(task);
     }
@@ -400,13 +416,59 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
     largeFileQueue = new StealJobQueue<>(largeQueueInitSize, smallQueueInitSize);
     smallFileQueue = largeFileQueue.getStealFromQueue();
     threads.clear();
-    builder.append("; new throttle point: ").append(throttlePoint).append(", largeQueueInitSize: ")
-        .append(largeQueueInitSize).append(", smallQueueInitSize: ").append(smallQueueInitSize);
-    LOG.debug(builder.toString());
     startHFileDeleteThreads();
     // re-dispatch the left over tasks
     for (HFileDeleteTask task : leftOverTasks) {
       dispatch(task);
     }
   }
+
+  /**
+   * Check new configuration and update settings if value changed
+   * @param conf The new configuration
+   * @return true if any configuration for HFileCleaner changes, false if no change
+   */
+  private boolean checkAndUpdateConfigurations(Configuration conf) {
+    boolean updated = false;
+    int throttlePoint =
+        conf.getInt(HFILE_DELETE_THROTTLE_THRESHOLD, DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD);
+    if (throttlePoint != this.throttlePoint) {
+      LOG.debug("Updating throttle point, from " + this.throttlePoint + " to " + throttlePoint);
+      this.throttlePoint = throttlePoint;
+      updated = true;
+    }
+    int largeQueueInitSize =
+        conf.getInt(LARGE_HFILE_QUEUE_INIT_SIZE, DEFAULT_LARGE_HFILE_QUEUE_INIT_SIZE);
+    if (largeQueueInitSize != this.largeQueueInitSize) {
+      LOG.debug("Updating largeQueueInitSize, from " + this.largeQueueInitSize + " to "
+          + largeQueueInitSize);
+      this.largeQueueInitSize = largeQueueInitSize;
+      updated = true;
+    }
+    int smallQueueInitSize =
+        conf.getInt(SMALL_HFILE_QUEUE_INIT_SIZE, DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE);
+    if (smallQueueInitSize != this.smallQueueInitSize) {
+      LOG.debug("Updating smallQueueInitSize, from " + this.smallQueueInitSize + " to "
+          + smallQueueInitSize);
+      this.smallQueueInitSize = smallQueueInitSize;
+      updated = true;
+    }
+    int largeFileDeleteThreadNumber =
+        conf.getInt(LARGE_HFILE_DELETE_THREAD_NUMBER, DEFAULT_LARGE_HFILE_DELETE_THREAD_NUMBER);
+    if (largeFileDeleteThreadNumber != this.largeFileDeleteThreadNumber) {
+      LOG.debug("Updating largeFileDeleteThreadNumber, from " + this.largeFileDeleteThreadNumber
+          + " to " + largeFileDeleteThreadNumber);
+      this.largeFileDeleteThreadNumber = largeFileDeleteThreadNumber;
+      updated = true;
+    }
+    int smallFileDeleteThreadNumber =
+        conf.getInt(SMALL_HFILE_DELETE_THREAD_NUMBER, DEFAULT_SMALL_HFILE_DELETE_THREAD_NUMBER);
+    if (smallFileDeleteThreadNumber != this.smallFileDeleteThreadNumber) {
+      LOG.debug("Updating smallFileDeleteThreadNumber, from " + this.smallFileDeleteThreadNumber
+          + " to " + smallFileDeleteThreadNumber);
+      this.smallFileDeleteThreadNumber = smallFileDeleteThreadNumber;
+      updated = true;
+    }
+    return updated;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/24341625/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 18afafa..249780b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Random;
 
 import org.apache.commons.logging.Log;
@@ -321,6 +322,8 @@ public class TestHFileCleaner {
     final int UPDATE_QUEUE_INIT_SIZE = 1024;
     final int LARGE_FILE_NUM = 5;
     final int SMALL_FILE_NUM = 20;
+    final int LARGE_THREAD_NUM = 2;
+    final int SMALL_THREAD_NUM = 4;
 
     Configuration conf = UTIL.getConfiguration();
     // no cleaner policies = delete all files
@@ -363,6 +366,8 @@ public class TestHFileCleaner {
     newConf.setInt(HFileCleaner.HFILE_DELETE_THROTTLE_THRESHOLD, UPDATE_THROTTLE_POINT);
     newConf.setInt(HFileCleaner.LARGE_HFILE_QUEUE_INIT_SIZE, UPDATE_QUEUE_INIT_SIZE);
     newConf.setInt(HFileCleaner.SMALL_HFILE_QUEUE_INIT_SIZE, UPDATE_QUEUE_INIT_SIZE);
+    newConf.setInt(HFileCleaner.LARGE_HFILE_DELETE_THREAD_NUMBER, LARGE_THREAD_NUM);
+    newConf.setInt(HFileCleaner.SMALL_HFILE_DELETE_THREAD_NUMBER, SMALL_THREAD_NUM);
     LOG.debug("File deleted from large queue: " + cleaner.getNumOfDeletedLargeFiles()
         + "; from small queue: " + cleaner.getNumOfDeletedSmallFiles());
     cleaner.onConfigurationChange(newConf);
@@ -371,7 +376,13 @@ public class TestHFileCleaner {
     Assert.assertEquals(UPDATE_THROTTLE_POINT, cleaner.getThrottlePoint());
     Assert.assertEquals(UPDATE_QUEUE_INIT_SIZE, cleaner.getLargeQueueInitSize());
     Assert.assertEquals(UPDATE_QUEUE_INIT_SIZE, cleaner.getSmallQueueInitSize());
-    Assert.assertEquals(2, cleaner.getCleanerThreads().size());
+    Assert.assertEquals(LARGE_THREAD_NUM + SMALL_THREAD_NUM, cleaner.getCleanerThreads().size());
+
+    // make sure no cost when onConfigurationChange called with no change
+    List<Thread> oldThreads = cleaner.getCleanerThreads();
+    cleaner.onConfigurationChange(newConf);
+    List<Thread> newThreads = cleaner.getCleanerThreads();
+    Assert.assertArrayEquals(oldThreads.toArray(), newThreads.toArray());
 
     // wait until clean done and check
     t.join();


[23/50] [abbrv] hbase git commit: HBASE-20352 [Chore] Backport HBASE-18309 (Support multi threads in CleanerChore) to branch-1

Posted by ap...@apache.org.
HBASE-20352 [Chore] Backport HBASE-18309 (Support multi threads in CleanerChore) to branch-1

Signed-off-by: Yu Li <li...@apache.org>

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java


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

Branch: refs/heads/branch-1.3
Commit: 193d1dcb72c22252fc86ee8433c765c42349d3cc
Parents: 35e94c9
Author: Reid Chan <re...@outlook.com>
Authored: Wed Apr 11 14:16:08 2018 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:18 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java |   4 +
 .../hbase/master/cleaner/CleanerChore.java      | 405 +++++++++++++++----
 .../hadoop/hbase/master/cleaner/LogCleaner.java | 181 +++++++++
 .../org/apache/hadoop/hbase/util/FSUtils.java   |  75 ++++
 .../hadoop/hbase/util/FileStatusFilter.java     |  36 ++
 .../TestZooKeeperTableArchiveClient.java        |   3 +
 .../hbase/master/cleaner/TestCleanerChore.java  | 164 +++++++-
 .../hbase/master/cleaner/TestHFileCleaner.java  |   1 +
 .../master/cleaner/TestHFileLinkCleaner.java    |   1 +
 .../hbase/master/cleaner/TestLogsCleaner.java   |  57 +++
 10 files changed, 837 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/193d1dcb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 0906fca..67c7787 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -97,6 +97,7 @@ import org.apache.hadoop.hbase.master.balancer.BalancerChore;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
 import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
+import org.apache.hadoop.hbase.master.cleaner.CleanerChore;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
 import org.apache.hadoop.hbase.master.cleaner.ReplicationZKLockCleanerChore;
@@ -859,6 +860,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     status.markComplete("Initialization successful");
     LOG.info("Master has completed initialization");
     configurationManager.registerObserver(this.balancer);
+    configurationManager.registerObserver(this.logCleaner);
 
     // Set master as 'initialized'.
     setInitialized(true);
@@ -1176,6 +1178,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    this.service.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
    startProcedureExecutor();
 
+    // Initial cleaner chore
+    CleanerChore.initChorePool(conf);
    // Start log cleaner thread
    int cleanerInterval = conf.getInt("hbase.master.cleaner.interval", 60 * 1000);
    this.logCleaner =

http://git-wip-us.apache.org/repos/asf/hbase/blob/193d1dcb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
index d54b7aa..dc614fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
@@ -21,6 +21,11 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.ForkJoinTask;
+import java.util.concurrent.RecursiveTask;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -28,12 +33,16 @@ 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.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.FileStatusFilter;
+import org.apache.hadoop.ipc.RemoteException;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
@@ -42,21 +51,113 @@ import com.google.common.collect.Lists;
  * Abstract Cleaner that uses a chain of delegates to clean a directory of files
  * @param <T> Cleaner delegate class that is dynamically loaded from configuration
  */
-public abstract class CleanerChore<T extends FileCleanerDelegate> extends ScheduledChore {
+@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD",
+    justification="Static pool will be only updated once.")
+@InterfaceAudience.Private
+public abstract class CleanerChore<T extends FileCleanerDelegate> extends ScheduledChore
+    implements ConfigurationObserver {
 
   private static final Log LOG = LogFactory.getLog(CleanerChore.class.getName());
+  private static final int AVAIL_PROCESSORS = Runtime.getRuntime().availableProcessors();
 
-  private final FileSystem fs;
+  /**
+   * If it is an integer and >= 1, it would be the size;
+   * if 0.0 < size <= 1.0, size would be available processors * size.
+   * Pay attention that 1.0 is different from 1, former indicates it will use 100% of cores,
+   * while latter will use only 1 thread for chore to scan dir.
+   */
+  public static final String CHORE_POOL_SIZE = "hbase.cleaner.scan.dir.concurrent.size";
+  private static final String DEFAULT_CHORE_POOL_SIZE = "0.25";
+
+  private static class DirScanPool {
+    int size;
+    ForkJoinPool pool;
+    int cleanerLatch;
+    AtomicBoolean reconfigNotification;
+
+    DirScanPool(Configuration conf) {
+      String poolSize = conf.get(CHORE_POOL_SIZE, DEFAULT_CHORE_POOL_SIZE);
+      size = calculatePoolSize(poolSize);
+      // poolSize may be 0 or 0.0 from a careless configuration,
+      // double check to make sure.
+      size = size == 0 ? calculatePoolSize(DEFAULT_CHORE_POOL_SIZE) : size;
+      pool = new ForkJoinPool(size);
+      LOG.info("Cleaner pool size is " + size);
+      reconfigNotification = new AtomicBoolean(false);
+      cleanerLatch = 0;
+    }
+
+    /**
+     * Checks if pool can be updated. If so, mark for update later.
+     * @param conf configuration
+     */
+    synchronized void markUpdate(Configuration conf) {
+      int newSize = calculatePoolSize(conf.get(CHORE_POOL_SIZE, DEFAULT_CHORE_POOL_SIZE));
+      if (newSize == size) {
+        LOG.trace("Size from configuration is same as previous=" + newSize +
+          " no need to update.");
+        return;
+      }
+      size = newSize;
+      // Chore is working, update it later.
+      reconfigNotification.set(true);
+    }
+
+    /**
+     * Update pool with new size.
+     */
+    synchronized void updatePool(long timeout) {
+      long stopWaitTime = System.currentTimeMillis() + timeout;
+      while (cleanerLatch != 0 && timeout > 0) {
+        try {
+          wait(timeout);
+          timeout = stopWaitTime - System.currentTimeMillis();
+        } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+          break;
+        }
+      }
+      pool.shutdownNow();
+      LOG.info("Update chore's pool size from " + pool.getParallelism() + " to " + size);
+      pool = new ForkJoinPool(size);
+    }
+
+    synchronized void latchCountUp() {
+      cleanerLatch++;
+    }
+
+    synchronized void latchCountDown() {
+      cleanerLatch--;
+      notifyAll();
+    }
+
+    @SuppressWarnings({"FutureReturnValueIgnored","rawtypes","unchecked"})
+    synchronized void submit(ForkJoinTask task) {
+      pool.submit(task);
+    }
+  }
+  // It may be waste resources for each cleaner chore own its pool,
+  // so let's make pool for all cleaner chores.
+  private static volatile DirScanPool POOL;
+
+  protected final FileSystem fs;
   private final Path oldFileDir;
   private final Configuration conf;
   protected List<T> cleanersChain;
   protected Map<String, Object> params;
+  private AtomicBoolean enabled = new AtomicBoolean(true);
 
   public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Configuration conf,
                       FileSystem fs, Path oldFileDir, String confKey) {
     this(name, sleepPeriod, s, conf, fs, oldFileDir, confKey, null);
   }
 
+  public static void initChorePool(Configuration conf) {
+    if (POOL == null) {
+      POOL = new DirScanPool(conf);
+    }
+  }
+
   /**
    * @param name name of the chore being run
    * @param sleepPeriod the period of time to sleep between each run
@@ -70,6 +171,9 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
   public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Configuration conf,
       FileSystem fs, Path oldFileDir, String confKey, Map<String, Object> params) {
     super(name, s, sleepPeriod);
+
+    Preconditions.checkNotNull(POOL, "Chore's pool isn't initialized, please call"
+        + "CleanerChore.initChorePool(Configuration) before new a cleaner chore.");
     this.fs = fs;
     this.oldFileDir = oldFileDir;
     this.conf = conf;
@@ -79,6 +183,36 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
 
 
   /**
+   * Calculate size for cleaner pool.
+   * @param poolSize size from configuration
+   * @return size of pool after calculation
+   */
+  static int calculatePoolSize(String poolSize) {
+    if (poolSize.matches("[1-9][0-9]*")) {
+      // If poolSize is an integer, return it directly,
+      // but upmost to the number of available processors.
+      int size = Math.min(Integer.parseInt(poolSize), AVAIL_PROCESSORS);
+      if (size == AVAIL_PROCESSORS) {
+        LOG.warn("Use full core processors to scan dir, size=" + size);
+      }
+      return size;
+    } else if (poolSize.matches("0.[0-9]+|1.0")) {
+      // if poolSize is a double, return poolSize * availableProcessors;
+      // Ensure that we always return at least one.
+      int computedThreads = (int) (AVAIL_PROCESSORS * Double.valueOf(poolSize));
+      if (computedThreads < 1) {
+        LOG.debug("Computed " + computedThreads + " threads for CleanerChore, using 1 instead");
+        return 1;
+      }
+      return computedThreads;
+    } else {
+      LOG.error("Unrecognized value: " + poolSize + " for " + CHORE_POOL_SIZE +
+          ", use default config: " + DEFAULT_CHORE_POOL_SIZE + " instead.");
+      return calculatePoolSize(DEFAULT_CHORE_POOL_SIZE);
+    }
+  }
+
+  /**
    * Validate the file to see if it even belongs in the directory. If it is valid, then the file
    * will go through the cleaner delegates, but otherwise the file is just deleted.
    * @param file full {@link Path} of the file to be checked
@@ -104,6 +238,11 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
     }
   }
 
+  @Override
+  public void onConfigurationChange(Configuration conf) {
+    POOL.markUpdate(conf);
+  }
+
   /**
    * A utility method to create new instances of LogCleanerDelegate based on the class name of the
    * LogCleanerDelegate.
@@ -129,87 +268,33 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
 
   @Override
   protected void chore() {
-    try {
-      FileStatus[] files = FSUtils.listStatus(this.fs, this.oldFileDir);
-      checkAndDeleteEntries(files);
-    } catch (IOException e) {
-      e = RemoteExceptionHandler.checkIOException(e);
-      LOG.warn("Error while cleaning the logs", e);
-    }
-  }
-
-  /**
-   * Loop over the given directory entries, and check whether they can be deleted.
-   * If an entry is itself a directory it will be recursively checked and deleted itself iff
-   * all subentries are deleted (and no new subentries are added in the mean time)
-   *
-   * @param entries directory entries to check
-   * @return true if all entries were successfully deleted
-   */
-  private boolean checkAndDeleteEntries(FileStatus[] entries) {
-    if (entries == null) {
-      return true;
-    }
-    boolean allEntriesDeleted = true;
-    List<FileStatus> files = Lists.newArrayListWithCapacity(entries.length);
-    for (FileStatus child : entries) {
-      Path path = child.getPath();
-      if (child.isDirectory()) {
-        // for each subdirectory delete it and all entries if possible
-        if (!checkAndDeleteDirectory(path)) {
-          allEntriesDeleted = false;
+    if (getEnabled()) {
+      try {
+        POOL.latchCountUp();
+        if (runCleaner()) {
+          LOG.debug("Cleaned all WALs under " + oldFileDir);
+        } else {
+          LOG.warn("WALs outstanding under " + oldFileDir);
         }
-      } else {
-        // collect all files to attempt to delete in one batch
-        files.add(child);
+      } finally {
+        POOL.latchCountDown();
       }
+      // After each cleaner chore, checks if received reconfigure notification while cleaning.
+      // First in cleaner turns off notification, to avoid another cleaner updating pool again.
+      if (POOL.reconfigNotification.compareAndSet(true, false)) {
+        // This cleaner is waiting for other cleaners finishing their jobs.
+        // To avoid missing next chore, only wait 0.8 * period, then shutdown.
+        POOL.updatePool((long) (0.8 * getTimeUnit().toMillis(getPeriod())));
+      }
+    } else {
+      LOG.debug("Cleaner chore disabled! Not cleaning.");
     }
-    if (!checkAndDeleteFiles(files)) {
-      allEntriesDeleted = false;
-    }
-    return allEntriesDeleted;
   }
-  
-  /**
-   * Attempt to delete a directory and all files under that directory. Each child file is passed
-   * through the delegates to see if it can be deleted. If the directory has no children when the
-   * cleaners have finished it is deleted.
-   * <p>
-   * If new children files are added between checks of the directory, the directory will <b>not</b>
-   * be deleted.
-   * @param dir directory to check
-   * @return <tt>true</tt> if the directory was deleted, <tt>false</tt> otherwise.
-   */
-  @VisibleForTesting boolean checkAndDeleteDirectory(Path dir) {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Checking directory: " + dir);
-    }
-
-    try {
-      FileStatus[] children = FSUtils.listStatus(fs, dir);
-      boolean allChildrenDeleted = checkAndDeleteEntries(children);
-  
-      // if the directory still has children, we can't delete it, so we are done
-      if (!allChildrenDeleted) return false;
-    } catch (IOException e) {
-      e = RemoteExceptionHandler.checkIOException(e);
-      LOG.warn("Error while listing directory: " + dir, e);
-      // couldn't list directory, so don't try to delete, and don't return success
-      return false;
-    }
 
-    // otherwise, all the children (that we know about) have been deleted, so we should try to
-    // delete this directory. However, don't do so recursively so we don't delete files that have
-    // been added since we last checked.
-    try {
-      return fs.delete(dir, false);
-    } catch (IOException e) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Couldn't delete directory: " + dir, e);
-      }
-      // couldn't delete w/o exception, so we can't return success.
-      return false;
-    }
+  public Boolean runCleaner() {
+    CleanerTask task = new CleanerTask(this.oldFileDir, true);
+    POOL.submit(task);
+    return task.join();
   }
 
   /**
@@ -219,6 +304,10 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
    * @return true iff successfully deleted all files
    */
   private boolean checkAndDeleteFiles(List<FileStatus> files) {
+    if (files == null) {
+      return true;
+    }
+
     // first check to see if the path is valid
     List<FileStatus> validFiles = Lists.newArrayListWithCapacity(files.size());
     List<FileStatus> invalidFiles = Lists.newArrayList();
@@ -256,12 +345,19 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
     }
     
     Iterable<FileStatus> filesToDelete = Iterables.concat(invalidFiles, deletableValidFiles);
+    return deleteFiles(filesToDelete) == files.size();
+  }
+
+  /**
+   * Delete the given files
+   * @param filesToDelete files to delete
+   * @return number of deleted files
+   */
+  protected int deleteFiles(Iterable<FileStatus> filesToDelete) {
     int deletedFileCount = 0;
     for (FileStatus file : filesToDelete) {
       Path filePath = file.getPath();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Removing: " + filePath + " from archive");
-      }
+      LOG.trace("Removing " + file + " from archive");
       try {
         boolean success = this.fs.delete(filePath, false);
         if (success) {
@@ -271,12 +367,12 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
               + ", but couldn't. Run cleaner chain and attempt to delete on next pass.");
         }
       } catch (IOException e) {
-        e = RemoteExceptionHandler.checkIOException(e);
+        e = e instanceof RemoteException ?
+            ((RemoteException)e).unwrapRemoteException() : e;
         LOG.warn("Error while deleting: " + filePath, e);
       }
     }
-
-    return deletedFileCount == files.size();
+    return deletedFileCount;
   }
 
   @Override
@@ -289,4 +385,143 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
       }
     }
   }
+
+  @VisibleForTesting
+  int getChorePoolSize() {
+    return POOL.size;
+  }
+
+  /**
+   * @param enabled
+   */
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  private interface Action<T> {
+    T act() throws IOException;
+  }
+
+  private class CleanerTask extends RecursiveTask<Boolean> {
+    private final Path dir;
+    private final boolean root;
+
+    CleanerTask(final FileStatus dir, final boolean root) {
+      this(dir.getPath(), root);
+    }
+
+    CleanerTask(final Path dir, final boolean root) {
+      this.dir = dir;
+      this.root = root;
+    }
+
+    @Override
+    protected Boolean compute() {
+      LOG.debug("Cleaning under " + dir);
+      List<FileStatus> subDirs;
+      final List<FileStatus> files;
+      try {
+        subDirs = FSUtils.listStatusWithStatusFilter(fs, dir, new FileStatusFilter() {
+          @Override
+          public boolean accept(FileStatus f) {
+            return f.isDirectory();
+          }
+        });
+        files = FSUtils.listStatusWithStatusFilter(fs, dir, new FileStatusFilter() {
+          @Override
+          public boolean accept(FileStatus f) {
+            return f.isFile();
+          }
+        });
+      } catch (IOException ioe) {
+        LOG.warn(dir + " doesn't exist, just skip it. ", ioe);
+        return true;
+      }
+
+      boolean nullSubDirs = subDirs == null;
+      if (nullSubDirs) {
+        LOG.trace("There is no subdir under " + dir);
+      }
+      if (files == null) {
+        LOG.trace("There is no file under " + dir);
+      }
+
+      int capacity = nullSubDirs ? 0 : subDirs.size();
+      final List<CleanerTask> tasks = Lists.newArrayListWithCapacity(capacity);
+      if (!nullSubDirs) {
+        for (FileStatus subdir : subDirs) {
+          CleanerTask task = new CleanerTask(subdir, false);
+          tasks.add(task);
+          task.fork();
+        }
+      }
+
+      boolean result = true;
+      result &= deleteAction(new Action<Boolean>() {
+        @Override
+        public Boolean act() throws IOException {
+          return checkAndDeleteFiles(files);
+        }
+      }, "files");
+      result &= deleteAction(new Action<Boolean>() {
+        @Override
+        public Boolean act() throws IOException {
+          return getCleanResult(tasks);
+        }
+      }, "subdirs");
+      // if and only if files and subdirs under current dir are deleted successfully, and
+      // it is not the root dir, then task will try to delete it.
+      if (result && !root) {
+        result &= deleteAction(new Action<Boolean>() {
+          @Override
+          public Boolean act() throws IOException {
+            return fs.delete(dir, false);
+          }
+        }, "dir");
+      }
+      return result;
+    }
+
+    /**
+     * Perform a delete on a specified type.
+     * @param deletion a delete
+     * @param type possible values are 'files', 'subdirs', 'dirs'
+     * @return true if it deleted successfully, false otherwise
+     */
+    private boolean deleteAction(Action<Boolean> deletion, String type) {
+      boolean deleted;
+      try {
+        LOG.trace("Start deleting " + type + " under " + dir);
+        deleted = deletion.act();
+      } catch (IOException ioe) {
+        LOG.warn("Could not delete " + type + " under " + dir, ioe);
+        deleted = false;
+      }
+      LOG.trace("Finish deleting " + type + " under " + dir + " deleted=" + deleted);
+      return deleted;
+    }
+
+    /**
+     * Get cleaner results of subdirs.
+     * @param tasks subdirs cleaner tasks
+     * @return true if all subdirs deleted successfully, false for patial/all failures
+     * @throws IOException something happen during computation
+     */
+    private boolean getCleanResult(List<CleanerTask> tasks) throws IOException {
+      boolean cleaned = true;
+      try {
+        for (CleanerTask task : tasks) {
+          cleaned &= task.get();
+        }
+      } catch (InterruptedException | ExecutionException e) {
+        throw new IOException(e);
+      }
+      return cleaned;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/193d1dcb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
index 1cc8d48..0c30f95 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
@@ -19,8 +19,17 @@ package org.apache.hadoop.hbase.master.cleaner;
 
 import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
 
+import com.google.common.annotations.VisibleForTesting;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -37,6 +46,12 @@ import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
   private static final Log LOG = LogFactory.getLog(LogCleaner.class.getName());
 
+  public static final String OLD_WALS_CLEANER_SIZE = "hbase.oldwals.cleaner.thread.size";
+  public static final int OLD_WALS_CLEANER_DEFAULT_SIZE = 2;
+
+  private final LinkedBlockingQueue<CleanerContext> pendingDelete;
+  private List<Thread> oldWALsCleaner;
+
   /**
    * @param p the period of time to sleep between each run
    * @param s the stopper
@@ -47,10 +62,176 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
   public LogCleaner(final int p, final Stoppable s, Configuration conf, FileSystem fs,
       Path oldLogDir) {
     super("LogsCleaner", p, s, conf, fs, oldLogDir, HBASE_MASTER_LOGCLEANER_PLUGINS);
+    this.pendingDelete = new LinkedBlockingQueue<>();
+    int size = conf.getInt(OLD_WALS_CLEANER_SIZE, OLD_WALS_CLEANER_DEFAULT_SIZE);
+    this.oldWALsCleaner = createOldWalsCleaner(size);
   }
 
   @Override
   protected boolean validate(Path file) {
     return DefaultWALProvider.validateWALFilename(file.getName());
   }
+
+  @Override
+  public void onConfigurationChange(Configuration conf) {
+    super.onConfigurationChange(conf);
+
+    int newSize = conf.getInt(OLD_WALS_CLEANER_SIZE, OLD_WALS_CLEANER_DEFAULT_SIZE);
+    if (newSize == oldWALsCleaner.size()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Size from configuration is the same as previous which is " +
+            newSize + ", no need to update.");
+      }
+      return;
+    }
+    interruptOldWALsCleaner();
+    oldWALsCleaner = createOldWalsCleaner(newSize);
+  }
+
+  @Override
+  protected int deleteFiles(Iterable<FileStatus> filesToDelete) {
+    List<CleanerContext> results = new LinkedList<>();
+    for (FileStatus toDelete : filesToDelete) {
+      CleanerContext context = CleanerContext.createCleanerContext(toDelete);
+      if (context != null) {
+        pendingDelete.add(context);
+        results.add(context);
+      }
+    }
+
+    int deletedFiles = 0;
+    for (CleanerContext res : results) {
+      deletedFiles += res.getResult(500) ? 1 : 0;
+    }
+    return deletedFiles;
+  }
+
+  @Override
+  public synchronized void cleanup() {
+    super.cleanup();
+    interruptOldWALsCleaner();
+  }
+
+  @VisibleForTesting
+  int getSizeOfCleaners() {
+    return oldWALsCleaner.size();
+  }
+
+  private List<Thread> createOldWalsCleaner(int size) {
+    LOG.info("Creating OldWALs cleaners with size=" + size);
+
+    List<Thread> oldWALsCleaner = new ArrayList<>(size);
+    for (int i = 0; i < size; i++) {
+      Thread cleaner = new Thread(new Runnable() {
+        @Override
+        public void run() {
+          deleteFile();
+        }
+      });
+      cleaner.setName("OldWALsCleaner-" + i);
+      cleaner.setDaemon(true);
+      cleaner.start();
+      oldWALsCleaner.add(cleaner);
+    }
+    return oldWALsCleaner;
+  }
+
+  private void interruptOldWALsCleaner() {
+    for (Thread cleaner : oldWALsCleaner) {
+      cleaner.interrupt();
+    }
+    oldWALsCleaner.clear();
+  }
+
+  private void deleteFile() {
+    while (true) {
+      CleanerContext context = null;
+      boolean succeed = false;
+      boolean interrupted = false;
+      try {
+        context = pendingDelete.take();
+        if (context != null) {
+          FileStatus toClean = context.getTargetToClean();
+          succeed = this.fs.delete(toClean.getPath(), false);
+        }
+      } catch (InterruptedException ite) {
+        // It's most likely from configuration changing request
+        if (context != null) {
+          LOG.warn("Interrupted while cleaning oldWALs " +
+              context.getTargetToClean() + ", try to clean it next round.");
+        }
+        interrupted = true;
+      } catch (IOException e) {
+        // fs.delete() fails.
+        LOG.warn("Failed to clean oldwals with exception: " + e);
+        succeed = false;
+      } finally {
+        if (context != null) {
+          context.setResult(succeed);
+        }
+        if (interrupted) {
+          // Restore interrupt status
+          Thread.currentThread().interrupt();
+          break;
+        }
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Exiting cleaner.");
+    }
+  }
+
+  @Override
+  public synchronized void cancel(boolean mayInterruptIfRunning) {
+    super.cancel(mayInterruptIfRunning);
+    for (Thread t : oldWALsCleaner) {
+      t.interrupt();
+    }
+  }
+
+  private static final class CleanerContext {
+    // At most waits 60 seconds
+    static final long MAX_WAIT = 60 * 1000;
+
+    final FileStatus target;
+    volatile boolean result;
+    volatile boolean setFromCleaner = false;
+
+    static CleanerContext createCleanerContext(FileStatus status) {
+      return status != null ? new CleanerContext(status) : null;
+    }
+
+    private CleanerContext(FileStatus status) {
+      this.target = status;
+      this.result = false;
+    }
+
+    synchronized void setResult(boolean res) {
+      this.result = res;
+      this.setFromCleaner = true;
+      notify();
+    }
+
+    synchronized boolean getResult(long waitIfNotFinished) {
+      long totalTime = 0;
+      try {
+        while (!setFromCleaner) {
+          wait(waitIfNotFinished);
+          totalTime += waitIfNotFinished;
+          if (totalTime >= MAX_WAIT) {
+            LOG.warn("Spend too much time to delete oldwals " + target);
+            return result;
+          }
+        }
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting deletion of " + target);
+        return result;
+      }
+      return result;
+    }
+
+    FileStatus getTargetToClean() {
+      return target;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/193d1dcb/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 87c50d9..b3be54e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -31,8 +31,10 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -81,6 +83,7 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 
+import com.google.common.collect.Iterators;
 import com.google.common.primitives.Ints;
 
 /**
@@ -1731,6 +1734,78 @@ public abstract class FSUtils {
   }
 
   /**
+   * Filters FileStatuses in an array and returns a list
+   *
+   * @param input   An array of FileStatuses
+   * @param filter  A required filter to filter the array
+   * @return        A list of FileStatuses
+   */
+  public static List<FileStatus> filterFileStatuses(FileStatus[] input,
+      FileStatusFilter filter) {
+    if (input == null) return null;
+    return filterFileStatuses(Iterators.forArray(input), filter);
+  }
+
+  /**
+   * Filters FileStatuses in an iterator and returns a list
+   *
+   * @param input   An iterator of FileStatuses
+   * @param filter  A required filter to filter the array
+   * @return        A list of FileStatuses
+   */
+  public static List<FileStatus> filterFileStatuses(Iterator<FileStatus> input,
+      FileStatusFilter filter) {
+    if (input == null) return null;
+    ArrayList<FileStatus> results = new ArrayList<>();
+    while (input.hasNext()) {
+      FileStatus f = input.next();
+      if (filter.accept(f)) {
+        results.add(f);
+      }
+    }
+    return results;
+  }
+
+  /**
+   * Calls fs.listStatus() and treats FileNotFoundException as non-fatal
+   * This accommodates differences between hadoop versions, where hadoop 1
+   * does not throw a FileNotFoundException, and return an empty FileStatus[]
+   * while Hadoop 2 will throw FileNotFoundException.
+   *
+   * @param fs file system
+   * @param dir directory
+   * @param filter file status filter
+   * @return null if dir is empty or doesn't exist, otherwise FileStatus list
+   */
+  public static List<FileStatus> listStatusWithStatusFilter(final FileSystem fs,
+      final Path dir, final FileStatusFilter filter) throws IOException {
+    FileStatus [] status = null;
+    try {
+      status = fs.listStatus(dir);
+    } catch (FileNotFoundException fnfe) {
+      // if directory doesn't exist, return null
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(dir + " doesn't exist");
+      }
+    }
+
+    if (status == null || status.length < 1)  {
+      return null;
+    }
+
+    if (filter == null) {
+      return Arrays.asList(status);
+    } else {
+      List<FileStatus> status2 = filterFileStatuses(status, filter);
+      if (status2 == null || status2.isEmpty()) {
+        return null;
+      } else {
+        return status2;
+      }
+    }
+  }
+
+  /**
    * Throw an exception if an action is not permitted by a user on a file.
    *
    * @param ugi

http://git-wip-us.apache.org/repos/asf/hbase/blob/193d1dcb/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FileStatusFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FileStatusFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FileStatusFilter.java
new file mode 100644
index 0000000..b3189b0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FileStatusFilter.java
@@ -0,0 +1,36 @@
+/**
+ * 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.util;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface FileStatusFilter {
+  /**
+   * Tests whether or not the specified filestatus should be
+   * included in a filestatus list.
+   *
+   * @param  f  The filestatus to be tested
+   * @return  <code>true</code> if and only if the filestatus
+   *          should be included
+   */
+  boolean accept(FileStatus f);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/193d1dcb/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
index a8713e7..b2139a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.master.cleaner.CleanerChore;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.client.ClusterConnection;
@@ -170,6 +171,7 @@ public class TestZooKeeperTableArchiveClient {
     Configuration conf = UTIL.getConfiguration();
     // setup the delegate
     Stoppable stop = new StoppableImplementation();
+    CleanerChore.initChorePool(conf);
     HFileCleaner cleaner = setupAndCreateCleaner(conf, fs, archiveDir, stop);
     List<BaseHFileCleanerDelegate> cleaners = turnOnArchiving(STRING_TABLE_NAME, cleaner);
     final LongTermArchivingHFileCleaner delegate = (LongTermArchivingHFileCleaner) cleaners.get(0);
@@ -224,6 +226,7 @@ public class TestZooKeeperTableArchiveClient {
     // setup the delegate
     Stoppable stop = new StoppableImplementation();
     final ChoreService choreService = new ChoreService("TEST_SERVER_NAME");
+    CleanerChore.initChorePool(conf);
     HFileCleaner cleaner = setupAndCreateCleaner(conf, fs, archiveDir, stop);
     List<BaseHFileCleanerDelegate> cleaners = turnOnArchiving(STRING_TABLE_NAME, cleaner);
     final LongTermArchivingHFileCleaner delegate = (LongTermArchivingHFileCleaner) cleaners.get(0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/193d1dcb/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
index 0bd0da5..505fd2c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
@@ -17,14 +17,17 @@
  */
 package org.apache.hadoop.hbase.master.cleaner;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -34,6 +37,7 @@ import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.StoppableImplementation;
 import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
@@ -46,6 +50,11 @@ public class TestCleanerChore {
   private static final Log LOG = LogFactory.getLog(TestCleanerChore.class);
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
+  @Before
+  public void setup() throws Exception {
+    CleanerChore.initChorePool(UTIL.getConfiguration());
+  }
+
   @After
   public void cleanup() throws Exception {
     // delete and recreate the test directory, ensuring a clean test dir between tests
@@ -275,11 +284,8 @@ public class TestCleanerChore {
       }
     }).when(spy).isFileDeletable(Mockito.any(FileStatus.class));
 
-    // attempt to delete the directory, which
-    if (chore.checkAndDeleteDirectory(parent)) {
-      throw new Exception(
-          "Reported success deleting directory, should have failed when adding file mid-iteration");
-    }
+    // run the chore
+    chore.chore();
 
     // make sure all the directories + added file exist, but the original file is deleted
     assertTrue("Added file unexpectedly deleted", fs.exists(racyFile));
@@ -288,6 +294,154 @@ public class TestCleanerChore {
     Mockito.verify(spy, Mockito.times(1)).isFileDeletable(Mockito.any(FileStatus.class));
   }
 
+  @Test
+  public void testDeleteFileWithCleanerEnabled() throws Exception {
+    Stoppable stop = new StoppableImplementation();
+    Configuration conf = UTIL.getConfiguration();
+    Path testDir = UTIL.getDataTestDir();
+    FileSystem fs = UTIL.getTestFileSystem();
+    String confKey = "hbase.test.cleaner.delegates";
+    conf.set(confKey, AlwaysDelete.class.getName());
+
+    AllValidPaths chore = new AllValidPaths("test-file-cleaner", stop, conf, fs, testDir, confKey);
+
+    // Enable cleaner
+    chore.setEnabled(true);
+
+    // create the directory layout in the directory to clean
+    Path parent = new Path(testDir, "parent");
+    Path child = new Path(parent, "child");
+    Path file = new Path(child, "someFile");
+    fs.mkdirs(child);
+
+    // touch a new file
+    fs.create(file).close();
+    assertTrue("Test file didn't get created.", fs.exists(file));
+
+    // run the chore
+    chore.chore();
+
+    // verify all the files got deleted
+    assertFalse("File didn't get deleted", fs.exists(file));
+    assertFalse("Empty directory didn't get deleted", fs.exists(child));
+    assertFalse("Empty directory didn't get deleted", fs.exists(parent));
+  }
+
+  @Test
+  public void testDeleteFileWithCleanerDisabled() throws Exception {
+    Stoppable stop = new StoppableImplementation();
+    Configuration conf = UTIL.getConfiguration();
+    Path testDir = UTIL.getDataTestDir();
+    FileSystem fs = UTIL.getTestFileSystem();
+    String confKey = "hbase.test.cleaner.delegates";
+    conf.set(confKey, AlwaysDelete.class.getName());
+
+    AllValidPaths chore = new AllValidPaths("test-file-cleaner", stop, conf, fs, testDir, confKey);
+
+    // Disable cleaner
+    chore.setEnabled(false);
+
+    // create the directory layout in the directory to clean
+    Path parent = new Path(testDir, "parent");
+    Path child = new Path(parent, "child");
+    Path file = new Path(child, "someFile");
+    fs.mkdirs(child);
+
+    // touch a new file
+    fs.create(file).close();
+    assertTrue("Test file didn't get created.", fs.exists(file));
+
+    // run the chore
+    chore.chore();
+
+    // verify all the files got deleted
+    assertTrue("File got deleted with cleaner disabled", fs.exists(file));
+    assertTrue("Directory got deleted", fs.exists(child));
+    assertTrue("Directory got deleted", fs.exists(parent));
+  }
+
+  @Test
+  public void testOnConfigurationChange() throws Exception {
+    int availableProcessorNum = Runtime.getRuntime().availableProcessors();
+    if (availableProcessorNum == 1) { // no need to run this test
+      return;
+    }
+
+    // have at least 2 available processors/cores
+    int    initPoolSize = availableProcessorNum / 2;
+    int changedPoolSize = availableProcessorNum;
+
+    Stoppable stop = new StoppableImplementation();
+    Configuration conf = UTIL.getConfiguration();
+    Path testDir = UTIL.getDataTestDir();
+    FileSystem fs = UTIL.getTestFileSystem();
+    String confKey = "hbase.test.cleaner.delegates";
+    conf.set(confKey, AlwaysDelete.class.getName());
+    conf.set(CleanerChore.CHORE_POOL_SIZE, String.valueOf(initPoolSize));
+    final AllValidPaths chore =
+      new AllValidPaths("test-file-cleaner", stop, conf, fs, testDir, confKey);
+    chore.setEnabled(true);
+    // Create subdirs under testDir
+    int dirNums = 6;
+    Path[] subdirs = new Path[dirNums];
+    for (int i = 0; i < dirNums; i++) {
+      subdirs[i] = new Path(testDir, "subdir-" + i);
+      fs.mkdirs(subdirs[i]);
+    }
+    // Under each subdirs create 6 files
+    for (Path subdir : subdirs) {
+      createFiles(fs, subdir, 6);
+    }
+    // Start chore
+    Thread t = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        chore.chore();
+      }
+    });
+    t.setDaemon(true);
+    t.start();
+    // Change size of chore's pool
+    conf.set(CleanerChore.CHORE_POOL_SIZE, String.valueOf(changedPoolSize));
+    chore.onConfigurationChange(conf);
+    assertEquals(changedPoolSize, chore.getChorePoolSize());
+    // Stop chore
+    t.join();
+  }
+
+  @Test
+  public void testMinimumNumberOfThreads() throws Exception {
+    Stoppable stop = new StoppableImplementation();
+    Configuration conf = UTIL.getConfiguration();
+    Path testDir = UTIL.getDataTestDir();
+    FileSystem fs = UTIL.getTestFileSystem();
+    String confKey = "hbase.test.cleaner.delegates";
+    conf.set(confKey, AlwaysDelete.class.getName());
+    conf.set(CleanerChore.CHORE_POOL_SIZE, "2");
+    AllValidPaths chore = new AllValidPaths("test-file-cleaner", stop, conf, fs, testDir, confKey);
+    int numProcs = Runtime.getRuntime().availableProcessors();
+    // Sanity
+    assertEquals(numProcs, chore.calculatePoolSize(Integer.toString(numProcs)));
+    // The implementation does not allow us to set more threads than we have processors
+    assertEquals(numProcs, chore.calculatePoolSize(Integer.toString(numProcs + 2)));
+    // Force us into the branch that is multiplying 0.0 against the number of processors
+    assertEquals(1, chore.calculatePoolSize("0.0"));
+  }
+
+  private void createFiles(FileSystem fs, Path parentDir, int numOfFiles) throws IOException {
+    Random random = new Random();
+    for (int i = 0; i < numOfFiles; i++) {
+      int xMega = 1 + random.nextInt(3); // size of each file is between 1~3M
+      try (FSDataOutputStream fsdos = fs.create(new Path(parentDir, "file-" + i))) {
+        for (int m = 0; m < xMega; m++) {
+          byte[] M = new byte[1024 * 1024];
+          random.nextBytes(M);
+          fsdos.write(M);
+        }
+      }
+    }
+  }
+
   private static class AllValidPaths extends CleanerChore<BaseHFileCleanerDelegate> {
 
     public AllValidPaths(String name, Stoppable s, Configuration conf, FileSystem fs,

http://git-wip-us.apache.org/repos/asf/hbase/blob/193d1dcb/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 4665593..32d2afd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -56,6 +56,7 @@ public class TestHFileCleaner {
   public static void setupCluster() throws Exception {
     // have to use a minidfs cluster because the localfs doesn't modify file times correctly
     UTIL.startMiniDFSCluster(1);
+    CleanerChore.initChorePool(UTIL.getConfiguration());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hbase/blob/193d1dcb/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
index a872679..76709a4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
@@ -96,6 +96,7 @@ public class TestHFileLinkCleaner {
     final long ttl = 1000;
     conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, ttl);
     Server server = new DummyServer();
+    CleanerChore.initChorePool(conf);
     HFileCleaner cleaner = new HFileCleaner(1000, server, conf, fs, archiveDir);
 
     // Link backref cannot be removed

http://git-wip-us.apache.org/repos/asf/hbase/blob/193d1dcb/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index df5916c..db15e95 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -30,10 +30,12 @@ import java.net.URLEncoder;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Random;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -76,6 +78,8 @@ public class TestLogsCleaner {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniZKCluster();
+    TEST_UTIL.startMiniDFSCluster(1);
+    CleanerChore.initChorePool(TEST_UTIL.getConfiguration());
   }
 
   /**
@@ -84,6 +88,7 @@ public class TestLogsCleaner {
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniZKCluster();
+    TEST_UTIL.shutdownMiniDFSCluster();
   }
 
   @Test
@@ -253,6 +258,58 @@ public class TestLogsCleaner {
     }
   }
 
+  @Test
+  public void testOnConfigurationChange() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    conf.setInt(LogCleaner.OLD_WALS_CLEANER_SIZE, LogCleaner.OLD_WALS_CLEANER_DEFAULT_SIZE);
+    // Prepare environments
+    Server server = new DummyServer();
+    Path oldWALsDir = new Path(TEST_UTIL.getDefaultRootDirPath(),
+        HConstants.HREGION_OLDLOGDIR_NAME);
+    FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
+    final LogCleaner cleaner = new LogCleaner(3000, server, conf, fs, oldWALsDir);
+    assertEquals(LogCleaner.OLD_WALS_CLEANER_DEFAULT_SIZE, cleaner.getSizeOfCleaners());
+    // Create dir and files for test
+    fs.delete(oldWALsDir, true);
+    fs.mkdirs(oldWALsDir);
+    int numOfFiles = 10;
+    createFiles(fs, oldWALsDir, numOfFiles);
+    FileStatus[] status = fs.listStatus(oldWALsDir);
+    assertEquals(numOfFiles, status.length);
+    // Start cleaner chore
+    Thread thread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        cleaner.chore();
+      }
+    });
+    thread.setDaemon(true);
+    thread.start();
+    // change size of cleaners dynamically
+    int sizeToChange = 4;
+    conf.setInt(LogCleaner.OLD_WALS_CLEANER_SIZE, sizeToChange);
+    cleaner.onConfigurationChange(conf);
+    assertEquals(sizeToChange, cleaner.getSizeOfCleaners());
+    // Stop chore
+    thread.join();
+    status = fs.listStatus(oldWALsDir);
+    assertEquals(0, status.length);
+  }
+
+  private void createFiles(FileSystem fs, Path parentDir, int numOfFiles) throws IOException {
+    Random random = new Random();
+    for (int i = 0; i < numOfFiles; i++) {
+      int xMega = 1 + random.nextInt(3); // size of each file is between 1~3M
+      try (FSDataOutputStream fsdos = fs.create(new Path(parentDir, "file-" + i))) {
+        for (int m = 0; m < xMega; m++) {
+          byte[] M = new byte[1024 * 1024];
+          random.nextBytes(M);
+          fsdos.write(M);
+        }
+      }
+    }
+  }
+
   static class DummyServer implements Server {
 
     @Override


[30/50] [abbrv] hbase git commit: HBASE-20672 New metrics ReadRequestRate and WriteRequestRate

Posted by ap...@apache.org.
HBASE-20672 New metrics ReadRequestRate and WriteRequestRate

Signed-off-by: Andrew Purtell <ap...@apache.org>

Conflicts:
	hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java


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

Branch: refs/heads/branch-1.3
Commit: 08246957e7b37f71ed3874e153ec79e69a87fc5a
Parents: 7bfec2a
Author: Ankit <ja...@salesforce.com>
Authored: Thu Jun 7 15:03:03 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:19 2018 -0800

----------------------------------------------------------------------
 .../regionserver/MetricsRegionServerSource.java |  6 ++++
 .../MetricsRegionServerWrapper.java             | 10 +++++++
 .../MetricsRegionServerSourceImpl.java          |  4 +++
 .../MetricsRegionServerWrapperImpl.java         | 29 ++++++++++++++++++++
 .../MetricsRegionServerWrapperStub.java         | 10 +++++++
 5 files changed, 59 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/08246957/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
index 1991105..e3eab04 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
@@ -225,6 +225,12 @@ public interface MetricsRegionServerSource extends BaseSource {
   String WRITE_REQUEST_COUNT = "writeRequestCount";
   String WRITE_REQUEST_COUNT_DESC =
       "Number of mutation requests this region server has answered.";
+  String READ_REQUEST_RATE = "readRequestRate";
+  String READ_REQUEST_RATE_DESC =
+      "Rate of answering the read requests by this region server per second.";
+  String WRITE_REQUEST_RATE = "writeRequestRate";
+  String WRITE_REQUEST_RATE_DESC =
+      "Rate of answering the mutation requests by this region server per second.";
   String CHECK_MUTATE_FAILED_COUNT = "checkMutateFailedCount";
   String CHECK_MUTATE_FAILED_COUNT_DESC =
       "Number of Check and Mutate calls that failed the checks.";

http://git-wip-us.apache.org/repos/asf/hbase/blob/08246957/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
index 7232063..6b34ca6 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
@@ -133,6 +133,16 @@ public interface MetricsRegionServerWrapper {
   long getWriteRequestsCount();
 
   /**
+   * Get the rate of read requests per second to regions hosted on this region server.
+   */
+  double getReadRequestsRate();
+
+  /**
+   * Get the rate of write requests per second to regions hosted on this region server.
+   */
+  double getWriteRequestsRate();
+
+  /**
    * Get the number of CAS operations that failed.
    */
   long getCheckAndMutateChecksFailed();

http://git-wip-us.apache.org/repos/asf/hbase/blob/08246957/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
index 1c79612..3156039 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
@@ -329,6 +329,10 @@ public class MetricsRegionServerSourceImpl
           .addGauge(Interns.info(RS_START_TIME_NAME, RS_START_TIME_DESC),
               rsWrap.getStartCode())
           .addGauge(Interns.info(AVERAGE_REGION_SIZE, AVERAGE_REGION_SIZE_DESC), rsWrap.getAverageRegionSize())
+          .addGauge(Interns.info(READ_REQUEST_RATE, READ_REQUEST_RATE_DESC),
+              rsWrap.getReadRequestsRate())
+          .addGauge(Interns.info(WRITE_REQUEST_RATE, WRITE_REQUEST_RATE_DESC),
+              rsWrap.getWriteRequestsRate())
           .addCounter(Interns.info(TOTAL_REQUEST_COUNT, TOTAL_REQUEST_COUNT_DESC),
               rsWrap.getTotalRequestCount())
           .addCounter(Interns.info(READ_REQUEST_COUNT, READ_REQUEST_COUNT_DESC),

http://git-wip-us.apache.org/repos/asf/hbase/blob/08246957/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index 35fd1ae..5679713 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -65,6 +65,8 @@ class MetricsRegionServerWrapperImpl
   private volatile double requestsPerSecond = 0.0;
   private volatile long readRequestsCount = 0;
   private volatile long writeRequestsCount = 0;
+  private volatile double readRequestsRate = 0;
+  private volatile double writeRequestsRate = 0;
   private volatile long checkAndMutateChecksFailed = 0;
   private volatile long checkAndMutateChecksPassed = 0;
   private volatile long storefileIndexSize = 0;
@@ -392,6 +394,16 @@ class MetricsRegionServerWrapperImpl
   }
 
   @Override
+  public double getReadRequestsRate() {
+    return readRequestsRate;
+  }
+
+  @Override
+  public double getWriteRequestsRate() {
+    return writeRequestsRate;
+  }
+
+  @Override
   public long getRpcGetRequestsCount() {
     return regionServer.rpcServices.rpcGetRequestCount.get();
   }
@@ -503,6 +515,8 @@ class MetricsRegionServerWrapperImpl
 
     private long lastRan = 0;
     private long lastRequestCount = 0;
+    private long lastReadRequestsCount = 0;
+    private long lastWriteRequestsCount = 0;
 
     @Override
     synchronized public void run() {
@@ -615,6 +629,21 @@ class MetricsRegionServerWrapperImpl
           requestsPerSecond =
               (currentRequestCount - lastRequestCount) / ((currentTime - lastRan) / 1000.0);
           lastRequestCount = currentRequestCount;
+
+          long intervalReadRequestsCount = tempReadRequestsCount - lastReadRequestsCount;
+          long intervalWriteRequestsCount = tempWriteRequestsCount - lastWriteRequestsCount;
+
+          double readRequestsRatePerMilliSecond = ((double)intervalReadRequestsCount/
+              (double)period);
+          double writeRequestsRatePerMilliSecond = ((double)intervalWriteRequestsCount/
+              (double)period);
+
+          readRequestsRate = readRequestsRatePerMilliSecond * 1000.0;
+          writeRequestsRate = writeRequestsRatePerMilliSecond * 1000.0;
+
+          lastReadRequestsCount = tempReadRequestsCount;
+          lastWriteRequestsCount = tempWriteRequestsCount;
+
         }
         lastRan = currentTime;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/08246957/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
index 32971a4..e2cd641 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
@@ -101,6 +101,11 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe
   }
 
   @Override
+  public double getReadRequestsRate() {
+    return 10.0;
+  }
+
+  @Override
   public long getReadRequestsCount() {
     return 997;
   }
@@ -111,6 +116,11 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe
   }
 
   @Override
+  public double getWriteRequestsRate() {
+    return 10.0;
+  }
+
+  @Override
   public long getRpcGetRequestsCount() {
     return 521;
   }


[49/50] [abbrv] hbase git commit: HBASE-21582 If call HBaseAdmin#snapshotAsync but forget call isSnapshotFinished, then SnapshotHFileCleaner will skip to run every time

Posted by ap...@apache.org.
HBASE-21582 If call HBaseAdmin#snapshotAsync but forget call isSnapshotFinished, then SnapshotHFileCleaner will skip to run every time

Conflicts:
	hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java


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

Branch: refs/heads/branch-1.3
Commit: d3729417462dc70a1fe3bdf5ffb3bea136987d4c
Parents: e063aa8
Author: huzheng <op...@gmail.com>
Authored: Tue Dec 11 20:27:56 2018 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Dec 13 10:25:10 2018 -0800

----------------------------------------------------------------------
 .../hbase/master/snapshot/SnapshotManager.java  | 55 +++++++++++++++-----
 .../master/cleaner/TestSnapshotFromMaster.java  | 32 +++++++++++-
 .../master/snapshot/TestSnapshotManager.java    | 36 +++++++++++--
 3 files changed, 104 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d3729417/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
index f480092..2859cc5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
@@ -27,7 +27,11 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -85,6 +89,9 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.zookeeper.KeeperException;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 /**
  * This class manages the procedure of taking and restoring snapshots. There is only one
  * SnapshotManager for the master.
@@ -113,7 +120,9 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    * At this point, if the user asks for the snapshot/restore status, the result will be
    * snapshot done if exists or failed if it doesn't exists.
    */
-  private static final int SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT = 60 * 1000;
+  public static final String HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS =
+      "hbase.snapshot.sentinels.cleanup.timeoutMillis";
+  public static final long SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLS_DEFAULT = 60 * 1000L;
 
   /** Enable or disable snapshot support */
   public static final String HBASE_SNAPSHOT_ENABLED = "hbase.snapshot.enabled";
@@ -144,8 +153,12 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
   // The map is always accessed and modified under the object lock using synchronized.
   // snapshotTable() will insert an Handler in the table.
   // isSnapshotDone() will remove the handler requested if the operation is finished.
-  private Map<TableName, SnapshotSentinel> snapshotHandlers =
+  private final Map<TableName, SnapshotSentinel> snapshotHandlers =
       new HashMap<TableName, SnapshotSentinel>();
+  private final ScheduledExecutorService scheduleThreadPool =
+        Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
+              .setNameFormat("SnapshotHandlerChoreCleaner").setDaemon(true).build());
+  private ScheduledFuture<?> snapshotHandlerChoreCleanerTask;
 
   // Restore Sentinels map, with table name as key.
   // The map is always accessed and modified under the object lock using synchronized.
@@ -173,17 +186,29 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    * @param coordinator procedure coordinator instance.  exposed for testing.
    * @param pool HBase ExecutorServcie instance, exposed for testing.
    */
-  public SnapshotManager(final MasterServices master, final MetricsMaster metricsMaster,
-      ProcedureCoordinator coordinator, ExecutorService pool)
+  @VisibleForTesting
+  SnapshotManager(final MasterServices master, ProcedureCoordinator coordinator,
+      ExecutorService pool, int sentinelCleanInterval)
       throws IOException, UnsupportedOperationException {
     this.master = master;
 
     this.rootDir = master.getMasterFileSystem().getRootDir();
-    checkSnapshotSupport(master.getConfiguration(), master.getMasterFileSystem());
+    Configuration conf = master.getConfiguration();
+    checkSnapshotSupport(conf, master.getMasterFileSystem());
 
     this.coordinator = coordinator;
     this.executorService = pool;
     resetTempDir();
+    initSnapshotHandlerChoreCleanerTask(sentinelCleanInterval);
+  }
+
+  private void initSnapshotHandlerChoreCleanerTask(long sentinelCleanInterval) {
+    snapshotHandlerChoreCleanerTask = this.scheduleThreadPool.scheduleAtFixedRate(new Runnable() {
+      @Override
+      public void run() {
+        cleanupSentinels();
+      }
+    }, sentinelCleanInterval, sentinelCleanInterval, TimeUnit.SECONDS);
   }
 
   /**
@@ -261,7 +286,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    *
    * @throws IOException if we can't reach the filesystem
    */
-  void resetTempDir() throws IOException {
+  private void resetTempDir() throws IOException {
     // cleanup any existing snapshots.
     Path tmpdir = SnapshotDescriptionUtils.getWorkingSnapshotDir(rootDir);
     if (master.getMasterFileSystem().getFileSystem().exists(tmpdir)) {
@@ -277,7 +302,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    * @throws SnapshotDoesNotExistException If the specified snapshot does not exist.
    * @throws IOException For filesystem IOExceptions
    */
-  public void deleteSnapshot(SnapshotDescription snapshot) throws SnapshotDoesNotExistException, IOException {
+  public void deleteSnapshot(SnapshotDescription snapshot) throws IOException {
     // check to see if it is completed
     if (!isSnapshotCompleted(snapshot)) {
       throw new SnapshotDoesNotExistException(snapshot);
@@ -991,14 +1016,15 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    */
   private synchronized void cleanupSentinels(final Map<TableName, SnapshotSentinel> sentinels) {
     long currentTime = EnvironmentEdgeManager.currentTime();
-    Iterator<Map.Entry<TableName, SnapshotSentinel>> it =
-        sentinels.entrySet().iterator();
+    long sentinelsCleanupTimeoutMillis =
+        master.getConfiguration().getLong(HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS,
+          SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLS_DEFAULT);
+    Iterator<Map.Entry<TableName, SnapshotSentinel>> it = sentinels.entrySet().iterator();
     while (it.hasNext()) {
       Map.Entry<TableName, SnapshotSentinel> entry = it.next();
       SnapshotSentinel sentinel = entry.getValue();
-      if (sentinel.isFinished() &&
-          (currentTime - sentinel.getCompletionTimestamp()) > SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT)
-      {
+      if (sentinel.isFinished()
+          && (currentTime - sentinel.getCompletionTimestamp()) > sentinelsCleanupTimeoutMillis) {
         it.remove();
       }
     }
@@ -1018,7 +1044,9 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
     for (SnapshotSentinel snapshotHandler: this.snapshotHandlers.values()) {
       snapshotHandler.cancel(why);
     }
-
+    if (snapshotHandlerChoreCleanerTask != null) {
+      snapshotHandlerChoreCleanerTask.cancel(true);
+    }
     // pass the stop onto all the restore handlers
     for (SnapshotSentinel restoreHandler: this.restoreHandlers.values()) {
       restoreHandler.cancel(why);
@@ -1157,6 +1185,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
     this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency);
     this.executorService = master.getExecutorService();
     resetTempDir();
+    initSnapshotHandlerChoreCleanerTask(10);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d3729417/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
index 0bffa92..0f2c85d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.master.cleaner;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -25,6 +26,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Set;
+import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -34,6 +36,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
@@ -41,7 +47,6 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.snapshot.DisabledTableSnapshotHandler;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotHFileCleaner;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
@@ -120,11 +125,11 @@ public class TestSnapshotFromMaster {
     conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, "");
     // Enable snapshot
     conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
+    conf.setLong(SnapshotManager.HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS, 3 * 1000L);
     conf.setLong(SnapshotHFileCleaner.HFILE_CACHE_REFRESH_PERIOD_CONF_KEY, cacheRefreshPeriod);
     conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
       ConstantSizeRegionSplitPolicy.class.getName());
     conf.setInt("hbase.hfile.compactions.cleaner.interval", 20 * 1000);
-
   }
 
   @Before
@@ -402,4 +407,27 @@ public class TestSnapshotFromMaster {
   private static void ensureHFileCleanersRun() {
     UTIL.getHBaseCluster().getMaster().getHFileCleaner().chore();
   }
+
+  @Test
+  public void testAsyncSnapshotWillNotBlockSnapshotHFileCleaner() throws Exception {
+    // Write some data
+    Table table = UTIL.getConnection().getTable(TABLE_NAME);
+    for (int i = 0; i < 10; i++) {
+      Put put = new Put(Bytes.toBytes(i)).addColumn(TEST_FAM, Bytes.toBytes("q"), Bytes.toBytes(i));
+      table.put(put);
+    }
+    final String snapshotName = "testAsyncSnapshotWillNotBlockSnapshotHFileCleaner01";
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName(snapshotName)
+        .setTable(TABLE_NAME.getNameAsString()).setType(SnapshotDescription.Type.FLUSH).build();
+    UTIL.getHBaseAdmin().takeSnapshotAsync(snapshot);
+    UTIL.waitFor(10 * 1000L, 200L, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return UTIL.getHBaseAdmin().listSnapshots(Pattern.compile(snapshotName)).size() == 1;
+      }
+    });
+    assertTrue(master.getSnapshotManager().isTakingAnySnapshot());
+    Thread.sleep(11 * 1000L);
+    assertFalse(master.getSnapshotManager().isTakingAnySnapshot());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d3729417/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
index 066d6f7..cf79fa3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.MetricsMaster;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner;
 import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
@@ -50,7 +49,6 @@ public class TestSnapshotManager {
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
   MasterServices services = Mockito.mock(MasterServices.class);
-  MetricsMaster metrics = Mockito.mock(MetricsMaster.class);
   ProcedureCoordinator coordinator = Mockito.mock(ProcedureCoordinator.class);
   ExecutorService pool = Mockito.mock(ExecutorService.class);
   MasterFileSystem mfs = Mockito.mock(MasterFileSystem.class);
@@ -67,14 +65,44 @@ public class TestSnapshotManager {
     return getNewManager(UTIL.getConfiguration());
   }
 
-  private SnapshotManager getNewManager(final Configuration conf)
+  private SnapshotManager getNewManager(Configuration conf) throws IOException, KeeperException {
+    return getNewManager(conf, 1);
+  }
+
+  private SnapshotManager getNewManager(Configuration conf, int intervalSeconds)
       throws IOException, KeeperException {
     Mockito.reset(services);
     Mockito.when(services.getConfiguration()).thenReturn(conf);
     Mockito.when(services.getMasterFileSystem()).thenReturn(mfs);
     Mockito.when(mfs.getFileSystem()).thenReturn(fs);
     Mockito.when(mfs.getRootDir()).thenReturn(UTIL.getDataTestDir());
-    return new SnapshotManager(services, metrics, coordinator, pool);
+    return new SnapshotManager(services, coordinator, pool, intervalSeconds);
+  }
+
+  @Test
+  public void testCleanFinishedHandler() throws Exception {
+    TableName tableName = TableName.valueOf("testCleanFinishedHandler");
+    Configuration conf = UTIL.getConfiguration();
+    try {
+      conf.setLong(SnapshotManager.HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS, 5 * 1000L);
+      SnapshotManager manager = getNewManager(conf, 1);
+      TakeSnapshotHandler handler = Mockito.mock(TakeSnapshotHandler.class);
+      assertFalse("Manager is in process when there is no current handler",
+        manager.isTakingSnapshot(tableName));
+      manager.setSnapshotHandlerForTesting(tableName, handler);
+      Mockito.when(handler.isFinished()).thenReturn(false);
+      assertTrue(manager.isTakingAnySnapshot());
+      assertTrue("Manager isn't in process when handler is running",
+        manager.isTakingSnapshot(tableName));
+      Mockito.when(handler.isFinished()).thenReturn(true);
+      assertFalse("Manager is process when handler isn't running",
+        manager.isTakingSnapshot(tableName));
+      assertTrue(manager.isTakingAnySnapshot());
+      Thread.sleep(6 * 1000);
+      assertFalse(manager.isTakingAnySnapshot());
+    } finally {
+      conf.unset(SnapshotManager.HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS);
+    }
   }
 
   @Test


[15/50] [abbrv] hbase git commit: HBASE-18248 Warn if monitored RPC task has been tied up beyond a configurable threshold

Posted by ap...@apache.org.
HBASE-18248 Warn if monitored RPC task has been tied up beyond a configurable threshold


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

Branch: refs/heads/branch-1.3
Commit: ac241dcde6850df351d6dbe8b00a319488fb465a
Parents: c35fa2a
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Aug 9 18:11:28 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:17 2018 -0800

----------------------------------------------------------------------
 .../monitoring/MonitoredRPCHandlerImpl.java     |  8 +-
 .../hadoop/hbase/monitoring/MonitoredTask.java  |  2 +
 .../hbase/monitoring/MonitoredTaskImpl.java     | 16 +++-
 .../hadoop/hbase/monitoring/TaskMonitor.java    | 90 +++++++++++++++++---
 .../hbase/monitoring/TestTaskMonitor.java       | 29 ++++++-
 5 files changed, 128 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ac241dcd/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java
index a29595b..08c8c9f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java
@@ -251,6 +251,12 @@ public class MonitoredRPCHandlerImpl extends MonitoredTaskImpl
     if (getState() != State.RUNNING) {
       return super.toString();
     }
-    return super.toString() + ", rpcMethod=" + getRPC();
+    return super.toString()
+        + ", queuetimems=" + getRPCQueueTime()
+        + ", starttimems=" + getRPCStartTime()
+        + ", clientaddress=" + clientAddress
+        + ", remoteport=" + remotePort
+        + ", packetlength=" + getRPCPacketLength()
+        + ", rpcMethod=" + getRPC();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac241dcd/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java
index 34fd8ce..5ba0969 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java
@@ -46,6 +46,7 @@ public interface MonitoredTask extends Cloneable {
   State getState();
   long getStateTime();
   long getCompletionTimestamp();
+  long getWarnTime();
 
   void markComplete(String msg);
   void pause(String msg);
@@ -55,6 +56,7 @@ public interface MonitoredTask extends Cloneable {
 
   void setStatus(String status);
   void setDescription(String description);
+  void setWarnTime(final long t);
 
   List<StatusJournalEntry> getStatusJournal();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac241dcd/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
index 5270e7d..ed04212 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
@@ -34,7 +34,8 @@ class MonitoredTaskImpl implements MonitoredTask {
   private long startTime;
   private long statusTime;
   private long stateTime;
-  
+  private long warnTime;
+
   private volatile String status;
   private volatile String description;
   
@@ -49,6 +50,7 @@ class MonitoredTaskImpl implements MonitoredTask {
     startTime = System.currentTimeMillis();
     statusTime = startTime;
     stateTime = startTime;
+    warnTime = startTime;
   }
 
   private static class StatusJournalEntryImpl implements StatusJournalEntry {
@@ -118,7 +120,12 @@ class MonitoredTaskImpl implements MonitoredTask {
   public long getStateTime() {
     return stateTime;
   }
-  
+
+  @Override
+  public long getWarnTime() {
+    return warnTime;
+  }
+
   @Override
   public long getCompletionTimestamp() {
     if (state == State.COMPLETE || state == State.ABORTED) {
@@ -171,6 +178,11 @@ class MonitoredTaskImpl implements MonitoredTask {
   }
 
   @Override
+  public void setWarnTime(long t) {
+    this.warnTime = t;
+  }
+
+  @Override
   public void cleanup() {
     if (state == State.RUNNING) {
       setState(State.ABORTED);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac241dcd/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java
index b620fac..07bab2e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java
@@ -30,9 +30,12 @@ import java.util.List;
 import org.apache.commons.collections.buffer.CircularFifoBuffer;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Threads;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 
 /**
@@ -44,15 +47,37 @@ import com.google.common.collect.Lists;
 public class TaskMonitor {
   private static final Log LOG = LogFactory.getLog(TaskMonitor.class);
 
-  // Don't keep around any tasks that have completed more than
-  // 60 seconds ago
-  private static final long EXPIRATION_TIME = 60*1000;
-
-  @VisibleForTesting
-  static final int MAX_TASKS = 1000;
+  public static final String MAX_TASKS_KEY = "hbase.taskmonitor.max.tasks";
+  public static final int DEFAULT_MAX_TASKS = 1000;
+  public static final String RPC_WARN_TIME_KEY = "hbase.taskmonitor.rpc.warn.time";
+  public static final long DEFAULT_RPC_WARN_TIME = 0;
+  public static final String EXPIRATION_TIME_KEY = "hbase.taskmonitor.expiration.time";
+  public static final long DEFAULT_EXPIRATION_TIME = 60*1000;
+  public static final String MONITOR_INTERVAL_KEY = "hbase.taskmonitor.monitor.interval";
+  public static final long DEFAULT_MONITOR_INTERVAL = 10*1000;
   
   private static TaskMonitor instance;
-  private CircularFifoBuffer tasks = new CircularFifoBuffer(MAX_TASKS);
+
+  private final int maxTasks;
+  private final long rpcWarnTime;
+  private final long expirationTime;
+  private final CircularFifoBuffer tasks;
+  private final long monitorInterval;
+  private Thread monitorThread;
+
+  TaskMonitor() {
+    this(HBaseConfiguration.create());
+  }
+
+  TaskMonitor(Configuration conf) {
+    maxTasks = conf.getInt(MAX_TASKS_KEY, DEFAULT_MAX_TASKS);
+    expirationTime = conf.getLong(EXPIRATION_TIME_KEY, DEFAULT_EXPIRATION_TIME);
+    rpcWarnTime = conf.getLong(RPC_WARN_TIME_KEY, DEFAULT_RPC_WARN_TIME);
+    tasks = new CircularFifoBuffer(maxTasks);
+    monitorInterval = conf.getLong(MONITOR_INTERVAL_KEY, DEFAULT_MONITOR_INTERVAL);
+    monitorThread = new Thread(new MonitorRunnable());
+    Threads.setDaemonThreadRunning(monitorThread, "Monitor thread for TaskMonitor");
+  }
 
   /**
    * Get singleton instance.
@@ -60,7 +85,7 @@ public class TaskMonitor {
    */
   public static synchronized TaskMonitor get() {
     if (instance == null) {
-      instance = new TaskMonitor();
+      instance = new TaskMonitor(HBaseConfiguration.create());
     }
     return instance;
   }
@@ -89,6 +114,23 @@ public class TaskMonitor {
     return proxy;
   }
 
+  private synchronized void warnStuckTasks() {
+    if (rpcWarnTime > 0) {
+      final long now = EnvironmentEdgeManager.currentTime();
+      for (Iterator<TaskAndWeakRefPair> it = tasks.iterator();
+          it.hasNext();) {
+        TaskAndWeakRefPair pair = it.next();
+        MonitoredTask stat = pair.get();
+        if ((stat instanceof MonitoredRPCHandler) &&
+            (stat.getState() == MonitoredTaskImpl.State.RUNNING) &&
+            (now >= stat.getWarnTime() + rpcWarnTime)) {
+          LOG.warn("Task may be stuck: " + stat);
+          stat.setWarnTime(now);
+        }
+      }
+    }
+  }
+
   private synchronized void purgeExpiredTasks() {
     for (Iterator<TaskAndWeakRefPair> it = tasks.iterator();
          it.hasNext();) {
@@ -129,12 +171,11 @@ public class TaskMonitor {
 
   private boolean canPurge(MonitoredTask stat) {
     long cts = stat.getCompletionTimestamp();
-    return (cts > 0 && System.currentTimeMillis() - cts > EXPIRATION_TIME);
+    return (cts > 0 && EnvironmentEdgeManager.currentTime() - cts > expirationTime);
   }
-  
 
   public void dumpAsText(PrintWriter out) {
-    long now = System.currentTimeMillis();
+    long now = EnvironmentEdgeManager.currentTime();
     
     List<MonitoredTask> tasks = getTasks();
     for (MonitoredTask task : tasks) {
@@ -154,6 +195,12 @@ public class TaskMonitor {
     }
   }
 
+  public synchronized void shutdown() {
+    if (this.monitorThread != null) {
+      monitorThread.interrupt();
+    }
+  }
+
   /**
    * This class encapsulates an object as well as a weak reference to a proxy
    * that passes through calls to that object. In art form:
@@ -208,4 +255,23 @@ public class TaskMonitor {
       return method.invoke(delegatee, args);
     }    
   }
+
+  private class MonitorRunnable implements Runnable {
+    private boolean running = true;
+
+    @Override
+    public void run() {
+      while (running) {
+        try {
+          Thread.sleep(monitorInterval);
+          if (tasks.isFull()) {
+            purgeExpiredTasks();
+          }
+          warnStuckTasks();
+        } catch (InterruptedException e) {
+          running = false;
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac241dcd/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java
index 8d6cdd7..3734530 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java
@@ -22,7 +22,9 @@ import static org.junit.Assert.*;
 
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -54,6 +56,8 @@ public class TestTaskMonitor {
     // If we mark its completion time back a few minutes, it should get gced
     task.expireNow();
     assertEquals(0, tm.getTasks().size());
+
+    tm.shutdown();
   }
   
   @Test
@@ -85,19 +89,23 @@ public class TestTaskMonitor {
     // Now it should be aborted 
     MonitoredTask taskFromTm = tm.getTasks().get(0);
     assertEquals(MonitoredTask.State.ABORTED, taskFromTm.getState());
+
+    tm.shutdown();
   }
   
   @Test
   public void testTaskLimit() throws Exception {
     TaskMonitor tm = new TaskMonitor();
-    for (int i = 0; i < TaskMonitor.MAX_TASKS + 10; i++) {
+    for (int i = 0; i < TaskMonitor.DEFAULT_MAX_TASKS + 10; i++) {
       tm.createStatus("task " + i);
     }
     // Make sure it was limited correctly
-    assertEquals(TaskMonitor.MAX_TASKS, tm.getTasks().size());
+    assertEquals(TaskMonitor.DEFAULT_MAX_TASKS, tm.getTasks().size());
     // Make sure we culled the earlier tasks, not later
     // (i.e. tasks 0 through 9 should have been deleted)
     assertEquals("task 10", tm.getTasks().get(0).getDescription());
+
+    tm.shutdown();
   }
 
   @Test
@@ -122,6 +130,23 @@ public class TestTaskMonitor {
     assertEquals(1, task.getStatusJournal().size());
     task.setStatus("status3");
     assertEquals("status3", task.getStatusJournal().get(1).getStatus());
+
+    tm.shutdown();
+  }
+
+  @Test
+  public void testWarnStuckTasks() throws Exception {
+    final int INTERVAL = 1000;
+    Configuration conf = new Configuration();
+    conf.setLong(TaskMonitor.RPC_WARN_TIME_KEY, INTERVAL);
+    conf.setLong(TaskMonitor.MONITOR_INTERVAL_KEY, INTERVAL);
+    final TaskMonitor tm = new TaskMonitor(conf);
+    MonitoredRPCHandler t = tm.createRPCStatus("test task");
+    long then = EnvironmentEdgeManager.currentTime();
+    t.setRPC("testMethod", new Object[0], then);
+    Thread.sleep(INTERVAL * 2);
+    assertTrue("We did not warn", t.getWarnTime() > then);
+    tm.shutdown();
   }
 
 }


[31/50] [abbrv] hbase git commit: HBASE-20770 WAL cleaner logs way too much; gets clogged when lots of work to do

Posted by ap...@apache.org.
HBASE-20770 WAL cleaner logs way too much; gets clogged when lots of work to do

General log cleanup; setting stuff that can flood the log to TRACE.

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 7bfec2a8989083c70c7fcc1b8c7a1b7af4d65eb8
Parents: 03885b4
Author: Michael Stack <st...@apache.org>
Authored: Sat Jun 23 23:29:11 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:19 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/security/HBaseSaslRpcClient.java   | 16 ++++++++--------
 .../hadoop/hbase/security/SaslClientHandler.java    | 12 ++++++------
 .../java/org/apache/hadoop/hbase/ipc/RpcServer.java |  4 ++--
 .../hbase/master/balancer/RegionLocationFinder.java |  3 +--
 .../hadoop/hbase/master/cleaner/CleanerChore.java   |  6 ++----
 5 files changed, 19 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfec2a8/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java
index bb6763f..f18b489 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java
@@ -200,8 +200,8 @@ public class HBaseSaslRpcClient {
           return false;
         }
         saslToken = new byte[len];
-        if (LOG.isDebugEnabled())
-          LOG.debug("Will read input token of size " + saslToken.length
+        if (LOG.isTraceEnabled())
+          LOG.trace("Will read input token of size " + saslToken.length
               + " for processing by initSASLContext");
         inStream.readFully(saslToken);
       }
@@ -209,8 +209,8 @@ public class HBaseSaslRpcClient {
       while (!saslClient.isComplete()) {
         saslToken = saslClient.evaluateChallenge(saslToken);
         if (saslToken != null) {
-          if (LOG.isDebugEnabled())
-            LOG.debug("Will send token of size " + saslToken.length
+          if (LOG.isTraceEnabled())
+            LOG.trace("Will send token of size " + saslToken.length
                 + " from initSASLContext.");
           outStream.writeInt(saslToken.length);
           outStream.write(saslToken, 0, saslToken.length);
@@ -219,14 +219,14 @@ public class HBaseSaslRpcClient {
         if (!saslClient.isComplete()) {
           readStatus(inStream);
           saslToken = new byte[inStream.readInt()];
-          if (LOG.isDebugEnabled())
-            LOG.debug("Will read input token of size " + saslToken.length
+          if (LOG.isTraceEnabled())
+            LOG.trace("Will read input token of size " + saslToken.length
                 + " for processing by initSASLContext");
           inStream.readFully(saslToken);
         }
       }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("SASL client context established. Negotiated QoP: "
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("SASL client context established. Negotiated QoP: "
             + saslClient.getNegotiatedProperty(Sasl.QOP));
       }
       return true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfec2a8/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java
index c79cde7..f43cb24 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java
@@ -212,16 +212,16 @@ public class SaslClientHandler extends ChannelDuplexHandler {
           }
         }
         saslToken = new byte[len];
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Will read input token of size " + saslToken.length
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Will read input token of size " + saslToken.length
               + " for processing by initSASLContext");
         }
         in.readBytes(saslToken);
 
         saslToken = evaluateChallenge(saslToken);
         if (saslToken != null) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Will send token of size " + saslToken.length + " from initSASLContext.");
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Will send token of size " + saslToken.length + " from initSASLContext.");
           }
           writeSaslToken(ctx, saslToken);
         }
@@ -230,8 +230,8 @@ public class SaslClientHandler extends ChannelDuplexHandler {
       if (saslClient.isComplete()) {
         String qop = (String) saslClient.getNegotiatedProperty(Sasl.QOP);
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("SASL client context established. Negotiated QoP: " + qop);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("SASL client context established. Negotiated QoP: " + qop);
         }
 
         boolean useWrap = qop != null && !"auth".equalsIgnoreCase(qop);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfec2a8/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index afae42d..975309c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -1514,8 +1514,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
           throw e;
         }
         if (replyToken != null) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Will send token of size " + replyToken.length
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Will send token of size " + replyToken.length
                 + " from saslServer.");
           }
           doRawSaslReply(SaslStatus.SUCCESS, new BytesWritable(replyToken), null,

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfec2a8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
index 6c5cb19..1ccd55f2a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
@@ -284,8 +284,7 @@ class RegionLocationFinder {
         blockDistbn = cache.get(hri);
         return blockDistbn;
       } else {
-        LOG.debug("HDFSBlocksDistribution not found in cache for region "
-            + hri.getRegionNameAsString());
+        LOG.trace("HDFSBlocksDistribution not found in cache for region " + hri.getRegionNameAsString());
         blockDistbn = internalGetTopBlockLocation(hri);
         cache.put(hri, blockDistbn);
         return blockDistbn;

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfec2a8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
index 28023c6..b3f1f0a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
@@ -292,9 +292,7 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
         POOL.updatePool((long) (0.8 * getTimeUnit().toMillis(getPeriod())));
       }
     } else {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Cleaner chore disabled! Not cleaning.");
-      }
+      LOG.trace("Cleaner chore disabled! Not cleaning.");
     }
   }
 
@@ -432,7 +430,7 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
 
     @Override
     protected Boolean compute() {
-      LOG.debug("Cleaning under " + dir);
+      LOG.trace("Cleaning under " + dir);
       List<FileStatus> subDirs;
       final List<FileStatus> files;
       try {


[20/50] [abbrv] hbase git commit: HBASE-19905 ReplicationSyncUp tool will not exit if a peer replication is disabled

Posted by ap...@apache.org.
HBASE-19905 ReplicationSyncUp tool will not exit if a peer replication is disabled

Signed-off-by: Ashish Singhi <as...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 8a4b2b54cdadfbe2ee730733f10bd3ec57b05340
Parents: 5c799c1
Author: Ashish Singhi <as...@apache.org>
Authored: Sun Feb 4 18:24:32 2018 +0530
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:18 2018 -0800

----------------------------------------------------------------------
 .../replication/regionserver/ReplicationSourceManager.java    | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8a4b2b54/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
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 c99d15c..8d19e22 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
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationListener;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
+import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
@@ -735,6 +736,12 @@ public class ReplicationSourceManager implements ReplicationListener {
             replicationQueues.removeQueue(peerId);
             continue;
           }
+          if (server instanceof ReplicationSyncUp.DummyServer
+              && peer.getPeerState().equals(PeerState.DISABLED)) {
+            LOG.warn("Peer " + actualPeerId + " is disbaled. ReplicationSyncUp tool will skip "
+                + "replicating data to this peer.");
+            continue;
+          }
           // track sources in walsByIdRecoveredQueues
           Map<String, SortedSet<String>> walsByGroup = new HashMap<String, SortedSet<String>>();
           walsByIdRecoveredQueues.put(peerId, walsByGroup);


[10/50] [abbrv] hbase git commit: HBASE-17118 StoreScanner leaked in KeyValueHeap (binlijin)

Posted by ap...@apache.org.
HBASE-17118 StoreScanner leaked in KeyValueHeap (binlijin)

HBASE-17118 StoreScanner leaked in KeyValueHeap (addendum)

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 25135dd0a13538e4dd4ffdd34dcb3120a010be3b
Parents: 286ade8
Author: tedyu <yu...@gmail.com>
Authored: Thu Nov 17 08:46:11 2016 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:16 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/KeyValueHeap.java | 68 ++++++++++++--------
 1 file changed, 42 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/25135dd0/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
index ac76bfd..5073c7e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
@@ -24,6 +24,8 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.PriorityQueue;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -44,6 +46,7 @@ import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
 @InterfaceAudience.Private
 public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
     implements KeyValueScanner, InternalScanner {
+  private static final Log LOG = LogFactory.getLog(KeyValueHeap.class);
   protected PriorityQueue<KeyValueScanner> heap = null;
 
   /**
@@ -289,35 +292,48 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
     heap.add(current);
     current = null;
 
-    KeyValueScanner scanner;
-    while ((scanner = heap.poll()) != null) {
-      Cell topKey = scanner.peek();
-      if (comparator.getComparator().compare(seekKey, topKey) <= 0) {
-        // Top KeyValue is at-or-after Seek KeyValue. We only know that all
-        // scanners are at or after seekKey (because fake keys of
-        // scanners where a lazy-seek operation has been done are not greater
-        // than their real next keys) but we still need to enforce our
-        // invariant that the top scanner has done a real seek. This way
-        // StoreScanner and RegionScanner do not have to worry about fake keys.
-        heap.add(scanner);
-        current = pollRealKV();
-        return current != null;
-      }
+    KeyValueScanner scanner = null;
+    try {
+      while ((scanner = heap.poll()) != null) {
+        Cell topKey = scanner.peek();
+        if (comparator.getComparator().compare(seekKey, topKey) <= 0) {
+          // Top KeyValue is at-or-after Seek KeyValue. We only know that all
+          // scanners are at or after seekKey (because fake keys of
+          // scanners where a lazy-seek operation has been done are not greater
+          // than their real next keys) but we still need to enforce our
+          // invariant that the top scanner has done a real seek. This way
+          // StoreScanner and RegionScanner do not have to worry about fake
+          // keys.
+          heap.add(scanner);
+          scanner = null;
+          current = pollRealKV();
+          return current != null;
+        }
 
-      boolean seekResult;
-      if (isLazy && heap.size() > 0) {
-        // If there is only one scanner left, we don't do lazy seek.
-        seekResult = scanner.requestSeek(seekKey, forward, useBloom);
-      } else {
-        seekResult = NonLazyKeyValueScanner.doRealSeek(
-            scanner, seekKey, forward);
-      }
+        boolean seekResult;
+        if (isLazy && heap.size() > 0) {
+          // If there is only one scanner left, we don't do lazy seek.
+          seekResult = scanner.requestSeek(seekKey, forward, useBloom);
+        } else {
+          seekResult = NonLazyKeyValueScanner.doRealSeek(scanner, seekKey,
+              forward);
+        }
 
-      if (!seekResult) {
-        scanner.close();
-      } else {
-        heap.add(scanner);
+        if (!seekResult) {
+          scanner.close();
+        } else {
+          heap.add(scanner);
+        }
+      }
+    } catch (Exception e) {
+      if (scanner != null) {
+        try {
+          scanner.close();
+        } catch (Exception ce) {
+          LOG.warn("close KeyValueScanner error", ce);
+        }
       }
+      throw e;
     }
 
     // Heap is returning empty, scanner is done


[35/50] [abbrv] hbase git commit: HBASE-20401 Make `MAX_WAIT` and `waitIfNotFinished` in CleanerContext configurable

Posted by ap...@apache.org.
HBASE-20401 Make `MAX_WAIT` and `waitIfNotFinished` in CleanerContext configurable

Signed-off-by: Reid Chan <re...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 614b5f6e724db594b37d900d5b0fa4ada636eee5
Parents: 2434162
Author: TAK LON WU <wu...@amazon.com>
Authored: Sun Jul 22 21:16:45 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:20 2018 -0800

----------------------------------------------------------------------
 .../hbase/master/cleaner/HFileCleaner.java      | 65 ++++++++++++++++----
 .../hadoop/hbase/master/cleaner/LogCleaner.java | 65 +++++++++++++++-----
 .../hbase/master/cleaner/TestHFileCleaner.java  | 15 ++++-
 .../hbase/master/cleaner/TestLogsCleaner.java   | 24 ++++++--
 4 files changed, 138 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/614b5f6e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
index 8f0b4be..6691f66 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -77,6 +78,16 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       "hbase.regionserver.hfilecleaner.small.thread.count";
   public final static int DEFAULT_SMALL_HFILE_DELETE_THREAD_NUMBER = 1;
 
+  public static final String HFILE_DELETE_THREAD_TIMEOUT_MSEC =
+      "hbase.regionserver.hfilecleaner.thread.timeout.msec";
+  @VisibleForTesting
+  static final long DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC = 60 * 1000L;
+
+  public static final String HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC =
+      "hbase.regionserver.hfilecleaner.thread.check.interval.msec";
+  @VisibleForTesting
+  static final long DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC = 1000L;
+
   private static final Log LOG = LogFactory.getLog(HFileCleaner.class);
 
   StealJobQueue<HFileDeleteTask> largeFileQueue;
@@ -86,6 +97,8 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   private int smallQueueInitSize;
   private int largeFileDeleteThreadNumber;
   private int smallFileDeleteThreadNumber;
+  private long cleanerThreadTimeoutMsec;
+  private long cleanerThreadCheckIntervalMsec;
   private List<Thread> threads = new ArrayList<Thread>();
   private boolean running;
 
@@ -116,6 +129,11 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
         conf.getInt(LARGE_HFILE_DELETE_THREAD_NUMBER, DEFAULT_LARGE_HFILE_DELETE_THREAD_NUMBER);
     smallFileDeleteThreadNumber =
         conf.getInt(SMALL_HFILE_DELETE_THREAD_NUMBER, DEFAULT_SMALL_HFILE_DELETE_THREAD_NUMBER);
+    cleanerThreadTimeoutMsec =
+        conf.getLong(HFILE_DELETE_THREAD_TIMEOUT_MSEC, DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC);
+    cleanerThreadCheckIntervalMsec =
+        conf.getLong(HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC,
+            DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC);
     startHFileDeleteThreads();
   }
 
@@ -147,7 +165,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
     }
     // wait for each submitted task to finish
     for (HFileDeleteTask task : tasks) {
-      if (task.getResult()) {
+      if (task.getResult(cleanerThreadCheckIntervalMsec)) {
         deletedFiles++;
       }
     }
@@ -160,7 +178,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
    * @return HFileDeleteTask to track progress
    */
   private HFileDeleteTask deleteFile(FileStatus file) {
-    HFileDeleteTask task = new HFileDeleteTask(file);
+    HFileDeleteTask task = new HFileDeleteTask(file, cleanerThreadTimeoutMsec);
     boolean enqueued = dispatch(task);
     return enqueued ? task : null;
   }
@@ -299,17 +317,17 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   }
 
   static class HFileDeleteTask implements Comparable<HFileDeleteTask> {
-    private static final long MAX_WAIT = 60 * 1000L;
-    private static final long WAIT_UNIT = 1000L;
 
     boolean done = false;
     boolean result;
     final Path filePath;
     final long fileLength;
+    final long timeoutMsec;
 
-    public HFileDeleteTask(FileStatus file) {
+    public HFileDeleteTask(FileStatus file, long timeoutMsec) {
       this.filePath = file.getPath();
       this.fileLength = file.getLen();
+      this.timeoutMsec = timeoutMsec;
     }
 
     public synchronized void setResult(boolean result) {
@@ -318,17 +336,19 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       notify();
     }
 
-    public synchronized boolean getResult() {
-      long waitTime = 0;
+    public synchronized boolean getResult(long waitIfNotFinished) {
+      long waitTimeMsec = 0;
       try {
         while (!done) {
-          wait(WAIT_UNIT);
-          waitTime += WAIT_UNIT;
+          long startTimeNanos = System.nanoTime();
+          wait(waitIfNotFinished);
+          waitTimeMsec += TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTimeNanos,
+              TimeUnit.NANOSECONDS);
           if (done) {
             return this.result;
           }
-          if (waitTime > MAX_WAIT) {
-            LOG.warn("Wait more than " + MAX_WAIT + " ms for deleting " + this.filePath
+          if (waitTimeMsec > timeoutMsec) {
+            LOG.warn("Wait more than " + timeoutMsec + " ms for deleting " + this.filePath
                 + ", exit...");
             return false;
           }
@@ -397,6 +417,16 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
     return throttlePoint;
   }
 
+  @VisibleForTesting
+  long getCleanerThreadTimeoutMsec() {
+    return cleanerThreadTimeoutMsec;
+  }
+
+  @VisibleForTesting
+  long getCleanerThreadCheckIntervalMsec() {
+    return cleanerThreadCheckIntervalMsec;
+  }
+
   @Override
   public void onConfigurationChange(Configuration conf) {
     if (!checkAndUpdateConfigurations(conf)) {
@@ -469,6 +499,19 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       this.smallFileDeleteThreadNumber = smallFileDeleteThreadNumber;
       updated = true;
     }
+    long cleanerThreadTimeoutMsec =
+        conf.getLong(HFILE_DELETE_THREAD_TIMEOUT_MSEC, DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC);
+    if (cleanerThreadTimeoutMsec != this.cleanerThreadTimeoutMsec) {
+      this.cleanerThreadTimeoutMsec = cleanerThreadTimeoutMsec;
+      updated = true;
+    }
+    long cleanerThreadCheckIntervalMsec =
+        conf.getLong(HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC,
+            DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC);
+    if (cleanerThreadCheckIntervalMsec != this.cleanerThreadCheckIntervalMsec) {
+      this.cleanerThreadCheckIntervalMsec = cleanerThreadCheckIntervalMsec;
+      updated = true;
+    }
     return updated;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/614b5f6e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
index 0c30f95..8b60803 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -46,11 +47,24 @@ import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
   private static final Log LOG = LogFactory.getLog(LogCleaner.class.getName());
 
-  public static final String OLD_WALS_CLEANER_SIZE = "hbase.oldwals.cleaner.thread.size";
-  public static final int OLD_WALS_CLEANER_DEFAULT_SIZE = 2;
+  public static final String OLD_WALS_CLEANER_THREAD_SIZE = "hbase.oldwals.cleaner.thread.size";
+  public static final int DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE = 2;
+
+  public static final String OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC =
+      "hbase.oldwals.cleaner.thread.timeout.msec";
+  @VisibleForTesting
+  static final long DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC = 60 * 1000L;
+
+  public static final String OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC =
+      "hbase.oldwals.cleaner.thread.check.interval.msec";
+  @VisibleForTesting
+  static final long DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC = 500L;
+
 
   private final LinkedBlockingQueue<CleanerContext> pendingDelete;
   private List<Thread> oldWALsCleaner;
+  private long cleanerThreadTimeoutMsec;
+  private long cleanerThreadCheckIntervalMsec;
 
   /**
    * @param p the period of time to sleep between each run
@@ -63,8 +77,12 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
       Path oldLogDir) {
     super("LogsCleaner", p, s, conf, fs, oldLogDir, HBASE_MASTER_LOGCLEANER_PLUGINS);
     this.pendingDelete = new LinkedBlockingQueue<>();
-    int size = conf.getInt(OLD_WALS_CLEANER_SIZE, OLD_WALS_CLEANER_DEFAULT_SIZE);
+    int size = conf.getInt(OLD_WALS_CLEANER_THREAD_SIZE, DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE);
     this.oldWALsCleaner = createOldWalsCleaner(size);
+    this.cleanerThreadTimeoutMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
+        DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC);
+    this.cleanerThreadCheckIntervalMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
+        DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC);
   }
 
   @Override
@@ -76,7 +94,7 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
   public void onConfigurationChange(Configuration conf) {
     super.onConfigurationChange(conf);
 
-    int newSize = conf.getInt(OLD_WALS_CLEANER_SIZE, OLD_WALS_CLEANER_DEFAULT_SIZE);
+    int newSize = conf.getInt(OLD_WALS_CLEANER_THREAD_SIZE, DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE);
     if (newSize == oldWALsCleaner.size()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Size from configuration is the same as previous which is " +
@@ -86,13 +104,18 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
     }
     interruptOldWALsCleaner();
     oldWALsCleaner = createOldWalsCleaner(newSize);
+    cleanerThreadTimeoutMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
+        DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC);
+    cleanerThreadCheckIntervalMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
+        DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC);
   }
 
   @Override
   protected int deleteFiles(Iterable<FileStatus> filesToDelete) {
     List<CleanerContext> results = new LinkedList<>();
     for (FileStatus toDelete : filesToDelete) {
-      CleanerContext context = CleanerContext.createCleanerContext(toDelete);
+      CleanerContext context = CleanerContext.createCleanerContext(toDelete,
+          cleanerThreadTimeoutMsec);
       if (context != null) {
         pendingDelete.add(context);
         results.add(context);
@@ -101,7 +124,7 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
 
     int deletedFiles = 0;
     for (CleanerContext res : results) {
-      deletedFiles += res.getResult(500) ? 1 : 0;
+      deletedFiles += res.getResult(cleanerThreadCheckIntervalMsec) ? 1 : 0;
     }
     return deletedFiles;
   }
@@ -117,6 +140,16 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
     return oldWALsCleaner.size();
   }
 
+  @VisibleForTesting
+  long getCleanerThreadTimeoutMsec() {
+    return cleanerThreadTimeoutMsec;
+  }
+
+  @VisibleForTesting
+  long getCleanerThreadCheckIntervalMsec() {
+    return cleanerThreadCheckIntervalMsec;
+  }
+
   private List<Thread> createOldWalsCleaner(int size) {
     LOG.info("Creating OldWALs cleaners with size=" + size);
 
@@ -190,20 +223,20 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
   }
 
   private static final class CleanerContext {
-    // At most waits 60 seconds
-    static final long MAX_WAIT = 60 * 1000;
 
     final FileStatus target;
     volatile boolean result;
     volatile boolean setFromCleaner = false;
+    long timeoutMsec;
 
-    static CleanerContext createCleanerContext(FileStatus status) {
-      return status != null ? new CleanerContext(status) : null;
+    static CleanerContext createCleanerContext(FileStatus status, long timeoutMsec) {
+      return status != null ? new CleanerContext(status, timeoutMsec) : null;
     }
 
-    private CleanerContext(FileStatus status) {
+    private CleanerContext(FileStatus status, long timeoutMsec) {
       this.target = status;
       this.result = false;
+      this.timeoutMsec = timeoutMsec;
     }
 
     synchronized void setResult(boolean res) {
@@ -213,13 +246,15 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
     }
 
     synchronized boolean getResult(long waitIfNotFinished) {
-      long totalTime = 0;
+      long totalTimeMsec = 0;
       try {
         while (!setFromCleaner) {
+          long startTimeNanos = System.nanoTime();
           wait(waitIfNotFinished);
-          totalTime += waitIfNotFinished;
-          if (totalTime >= MAX_WAIT) {
-            LOG.warn("Spend too much time to delete oldwals " + target);
+          totalTimeMsec += TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTimeNanos,
+              TimeUnit.NANOSECONDS);
+          if (totalTimeMsec >= timeoutMsec) {
+            LOG.warn("Spend too much time " + totalTimeMsec + " ms to delete oldwals " + target);
             return result;
           }
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/614b5f6e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 249780b..099d7ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -36,10 +36,10 @@ import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@@ -324,6 +324,8 @@ public class TestHFileCleaner {
     final int SMALL_FILE_NUM = 20;
     final int LARGE_THREAD_NUM = 2;
     final int SMALL_THREAD_NUM = 4;
+    final long THREAD_TIMEOUT_MSEC = 30 * 1000L;
+    final long THREAD_CHECK_INTERVAL_MSEC = 500L;
 
     Configuration conf = UTIL.getConfiguration();
     // no cleaner policies = delete all files
@@ -341,6 +343,10 @@ public class TestHFileCleaner {
     Assert.assertEquals(ORIGINAL_THROTTLE_POINT, cleaner.getThrottlePoint());
     Assert.assertEquals(ORIGINAL_QUEUE_INIT_SIZE, cleaner.getLargeQueueInitSize());
     Assert.assertEquals(ORIGINAL_QUEUE_INIT_SIZE, cleaner.getSmallQueueInitSize());
+    Assert.assertEquals(HFileCleaner.DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC,
+        cleaner.getCleanerThreadTimeoutMsec());
+    Assert.assertEquals(HFileCleaner.DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC,
+        cleaner.getCleanerThreadCheckIntervalMsec());
 
     // clean up archive directory and create files for testing
     fs.delete(archivedHfileDir, true);
@@ -368,6 +374,11 @@ public class TestHFileCleaner {
     newConf.setInt(HFileCleaner.SMALL_HFILE_QUEUE_INIT_SIZE, UPDATE_QUEUE_INIT_SIZE);
     newConf.setInt(HFileCleaner.LARGE_HFILE_DELETE_THREAD_NUMBER, LARGE_THREAD_NUM);
     newConf.setInt(HFileCleaner.SMALL_HFILE_DELETE_THREAD_NUMBER, SMALL_THREAD_NUM);
+    newConf.setLong(HFileCleaner.HFILE_DELETE_THREAD_TIMEOUT_MSEC, THREAD_TIMEOUT_MSEC);
+    newConf.setLong(HFileCleaner.HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC,
+        THREAD_CHECK_INTERVAL_MSEC);
+    cleaner.onConfigurationChange(newConf);
+
     LOG.debug("File deleted from large queue: " + cleaner.getNumOfDeletedLargeFiles()
         + "; from small queue: " + cleaner.getNumOfDeletedSmallFiles());
     cleaner.onConfigurationChange(newConf);
@@ -377,6 +388,8 @@ public class TestHFileCleaner {
     Assert.assertEquals(UPDATE_QUEUE_INIT_SIZE, cleaner.getLargeQueueInitSize());
     Assert.assertEquals(UPDATE_QUEUE_INIT_SIZE, cleaner.getSmallQueueInitSize());
     Assert.assertEquals(LARGE_THREAD_NUM + SMALL_THREAD_NUM, cleaner.getCleanerThreads().size());
+    Assert.assertEquals(THREAD_TIMEOUT_MSEC, cleaner.getCleanerThreadTimeoutMsec());
+    Assert.assertEquals(THREAD_CHECK_INTERVAL_MSEC, cleaner.getCleanerThreadCheckIntervalMsec());
 
     // make sure no cost when onConfigurationChange called with no change
     List<Thread> oldThreads = cleaner.getCleanerThreads();

http://git-wip-us.apache.org/repos/asf/hbase/blob/614b5f6e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index db15e95..2578ec9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -242,7 +242,7 @@ public class TestLogsCleaner {
         new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log1")),
         new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log2"))
     );
-    
+
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testZooKeeperAbort-normal", null);
     try {
       cleaner.setConf(conf, zkw);
@@ -261,14 +261,23 @@ public class TestLogsCleaner {
   @Test
   public void testOnConfigurationChange() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setInt(LogCleaner.OLD_WALS_CLEANER_SIZE, LogCleaner.OLD_WALS_CLEANER_DEFAULT_SIZE);
+    conf.setInt(LogCleaner.OLD_WALS_CLEANER_THREAD_SIZE,
+        LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE);
+    conf.setLong(LogCleaner.OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
+        LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC);
+    conf.setLong(LogCleaner.OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
+        LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC);
     // Prepare environments
     Server server = new DummyServer();
     Path oldWALsDir = new Path(TEST_UTIL.getDefaultRootDirPath(),
         HConstants.HREGION_OLDLOGDIR_NAME);
     FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
     final LogCleaner cleaner = new LogCleaner(3000, server, conf, fs, oldWALsDir);
-    assertEquals(LogCleaner.OLD_WALS_CLEANER_DEFAULT_SIZE, cleaner.getSizeOfCleaners());
+    assertEquals(LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE, cleaner.getSizeOfCleaners());
+    assertEquals(LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
+        cleaner.getCleanerThreadTimeoutMsec());
+    assertEquals(LogCleaner.DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
+        cleaner.getCleanerThreadCheckIntervalMsec());
     // Create dir and files for test
     fs.delete(oldWALsDir, true);
     fs.mkdirs(oldWALsDir);
@@ -287,9 +296,16 @@ public class TestLogsCleaner {
     thread.start();
     // change size of cleaners dynamically
     int sizeToChange = 4;
-    conf.setInt(LogCleaner.OLD_WALS_CLEANER_SIZE, sizeToChange);
+    long threadTimeoutToChange = 30 * 1000L;
+    long threadCheckIntervalToChange = 250L;
+    conf.setInt(LogCleaner.OLD_WALS_CLEANER_THREAD_SIZE, sizeToChange);
+    conf.setLong(LogCleaner.OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC, threadTimeoutToChange);
+    conf.setLong(LogCleaner.OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
+        threadCheckIntervalToChange);
     cleaner.onConfigurationChange(conf);
     assertEquals(sizeToChange, cleaner.getSizeOfCleaners());
+    assertEquals(threadTimeoutToChange, cleaner.getCleanerThreadTimeoutMsec());
+    assertEquals(threadCheckIntervalToChange, cleaner.getCleanerThreadCheckIntervalMsec());
     // Stop chore
     thread.join();
     status = fs.listStatus(oldWALsDir);


[44/50] [abbrv] hbase git commit: HBASE-21126 Configurable number of allowed failures for ZooKeeper Canary

Posted by ap...@apache.org.
HBASE-21126 Configurable number of allowed failures for ZooKeeper Canary

Signed-off-by: Josh Elser <el...@apache.org>

Conflicts:
	hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java


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

Branch: refs/heads/branch-1.3
Commit: 09069df2f8a9cb19ce368a54770d333f0e36fe5d
Parents: 1ecfca3
Author: David Manning <da...@salesforce.com>
Authored: Fri Aug 31 18:32:15 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 19:25:40 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/tool/Canary.java    | 51 +++++++++++++++-----
 .../hadoop/hbase/tool/TestCanaryTool.java       | 35 +++++++++-----
 2 files changed, 62 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/09069df2/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index dcaa057..081ef90 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -580,6 +580,7 @@ public final class Canary implements Tool {
   private boolean failOnError = true;
   private boolean regionServerMode = false;
   private boolean zookeeperMode = false;
+  private long permittedFailures = 0;
   private boolean regionServerAllRegions = false;
   private boolean writeSniffing = false;
   private long configuredWriteTableTimeout = DEFAULT_TIMEOUT;
@@ -723,6 +724,19 @@ public final class Canary implements Tool {
             }
             this.configuredReadTableTimeouts.put(nameTimeout[0], timeoutVal);
           }
+        } else if (cmd.equals("-permittedZookeeperFailures")) {
+          i++;
+
+          if (i == args.length) {
+            System.err.println("-permittedZookeeperFailures needs a numeric value argument.");
+            printUsageAndExit();
+          }
+          try {
+            this.permittedFailures = Long.parseLong(args[i]);
+          } catch (NumberFormatException e) {
+            System.err.println("-permittedZookeeperFailures needs a numeric value argument.");
+            printUsageAndExit();
+          }
         } else {
           // no options match
           System.err.println(cmd + " options is invalid.");
@@ -744,6 +758,10 @@ public final class Canary implements Tool {
         printUsageAndExit();
       }
     }
+    if (this.permittedFailures != 0 && !this.zookeeperMode) {
+      System.err.println("-permittedZookeeperFailures requires -zookeeper mode.");
+      printUsageAndExit();
+    }
     if (!this.configuredReadTableTimeouts.isEmpty() && (this.regionServerMode || this.zookeeperMode)) {
       System.err.println("-readTableTimeouts can only be configured in region mode.");
       printUsageAndExit();
@@ -842,6 +860,8 @@ public final class Canary implements Tool {
     System.err.println("      only works in regionserver mode.");
     System.err.println("   -zookeeper    Tries to grab zookeeper.znode.parent ");
     System.err.println("      on each zookeeper instance");
+    System.err.println("   -permittedZookeeperFailures <N>    Ignore first N failures when attempting to ");
+    System.err.println("      connect to individual zookeeper nodes in the ensemble");
     System.err.println("   -daemon        Continuous check at defined intervals.");
     System.err.println("   -interval <N>  Interval between checks (sec)");
     System.err.println("   -e             Use table/regionserver as regular expression");
@@ -884,17 +904,18 @@ public final class Canary implements Tool {
       monitor =
           new RegionServerMonitor(connection, monitorTargets, this.useRegExp,
               (StdOutSink) this.sink, this.executor, this.regionServerAllRegions,
-              this.treatFailureAsError);
+              this.treatFailureAsError, this.permittedFailures);
     } else if (this.sink instanceof ZookeeperStdOutSink || this.zookeeperMode) {
       monitor =
           new ZookeeperMonitor(connection, monitorTargets, this.useRegExp,
-              (StdOutSink) this.sink, this.executor, this.treatFailureAsError);
+              (StdOutSink) this.sink, this.executor, this.treatFailureAsError,
+              this.permittedFailures);
     } else {
       monitor =
           new RegionMonitor(connection, monitorTargets, this.useRegExp,
               (StdOutSink) this.sink, this.executor, this.writeSniffing,
               this.writeTableName, this.treatFailureAsError, this.configuredReadTableTimeouts,
-              this.configuredWriteTableTimeout);
+              this.configuredWriteTableTimeout, this.permittedFailures);
     }
     return monitor;
   }
@@ -911,6 +932,7 @@ public final class Canary implements Tool {
 
     protected boolean done = false;
     protected int errorCode = 0;
+    protected long allowedFailures = 0;
     protected Sink sink;
     protected ExecutorService executor;
 
@@ -927,7 +949,8 @@ public final class Canary implements Tool {
         return true;
       }
       if (treatFailureAsError &&
-          (sink.getReadFailureCount() > 0 || sink.getWriteFailureCount() > 0)) {
+          (sink.getReadFailureCount() > allowedFailures || sink.getWriteFailureCount() > allowedFailures)) {
+        LOG.error("Too many failures detected, treating failure as error, failing the Canary.");
         errorCode = FAILURE_EXIT_CODE;
         return true;
       }
@@ -940,7 +963,7 @@ public final class Canary implements Tool {
     }
 
     protected Monitor(Connection connection, String[] monitorTargets, boolean useRegExp, Sink sink,
-        ExecutorService executor, boolean treatFailureAsError) {
+        ExecutorService executor, boolean treatFailureAsError, long allowedFailures) {
       if (null == connection) throw new IllegalArgumentException("connection shall not be null");
 
       this.connection = connection;
@@ -949,6 +972,7 @@ public final class Canary implements Tool {
       this.treatFailureAsError = treatFailureAsError;
       this.sink = sink;
       this.executor = executor;
+      this.allowedFailures = allowedFailures;
     }
 
     @Override
@@ -991,8 +1015,8 @@ public final class Canary implements Tool {
     public RegionMonitor(Connection connection, String[] monitorTargets, boolean useRegExp,
         StdOutSink sink, ExecutorService executor, boolean writeSniffing, TableName writeTableName,
         boolean treatFailureAsError, HashMap<String, Long> configuredReadTableTimeouts,
-        long configuredWriteTableTimeout) {
-      super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError);
+        long configuredWriteTableTimeout, long allowedFailures) {
+      super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError, allowedFailures);
       Configuration conf = connection.getConfiguration();
       this.writeSniffing = writeSniffing;
       this.writeTableName = writeTableName;
@@ -1286,8 +1310,8 @@ public final class Canary implements Tool {
     private final int timeout;
 
     protected ZookeeperMonitor(Connection connection, String[] monitorTargets, boolean useRegExp,
-        StdOutSink sink, ExecutorService executor, boolean treatFailureAsError)  {
-      super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError);
+        StdOutSink sink, ExecutorService executor, boolean treatFailureAsError, long allowedFailures)  {
+      super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError, allowedFailures);
       Configuration configuration = connection.getConfiguration();
       znode =
           configuration.get(ZOOKEEPER_ZNODE_PARENT,
@@ -1300,6 +1324,11 @@ public final class Canary implements Tool {
       for (InetSocketAddress server : parser.getServerAddresses()) {
         hosts.add(server.toString());
       }
+      if (allowedFailures > (hosts.size() - 1) / 2) {
+        LOG.warn(String.format("Confirm allowable number of failed ZooKeeper nodes, as quorum will " +
+                        "already be lost. Setting of %d failures is unexpected for %d ensemble size.",
+                allowedFailures, hosts.size()));
+      }
     }
 
     @Override public void run() {
@@ -1348,8 +1377,8 @@ public final class Canary implements Tool {
 
     public RegionServerMonitor(Connection connection, String[] monitorTargets, boolean useRegExp,
         StdOutSink sink, ExecutorService executor, boolean allRegions,
-        boolean treatFailureAsError) {
-      super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError);
+        boolean treatFailureAsError, long allowedFailures) {
+      super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError, allowedFailures);
       this.allRegions = allRegions;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/09069df2/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
index b89561f..8654a51 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
@@ -83,20 +83,14 @@ public class TestCanaryTool {
 
   @Test
   public void testBasicZookeeperCanaryWorks() throws Exception {
-    Integer port =
-        Iterables.getOnlyElement(testingUtility.getZkCluster().getClientPortList(), null);
-    testingUtility.getConfiguration().set(HConstants.ZOOKEEPER_QUORUM,
-        "localhost:" + port + "/hbase");
-    ExecutorService executor = new ScheduledThreadPoolExecutor(2);
-    Canary.ZookeeperStdOutSink sink = spy(new Canary.ZookeeperStdOutSink());
-    Canary canary = new Canary(executor, sink);
-    String[] args = { "-t", "10000", "-zookeeper" };
-    ToolRunner.run(testingUtility.getConfiguration(), canary, args);
+    final String[] args = { "-t", "10000", "-zookeeper" };
+    testZookeeperCanaryWithArgs(args);
+  }
 
-    String baseZnode = testingUtility.getConfiguration()
-        .get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
-    verify(sink, atLeastOnce())
-        .publishReadTiming(eq(baseZnode), eq("localhost:" + port), anyLong());
+  @Test
+  public void testZookeeperCanaryPermittedFailuresArgumentWorks() throws Exception {
+    final String[] args = { "-t", "10000", "-zookeeper", "-treatFailureAsError", "-permittedZookeeperFailures", "1" };
+    testZookeeperCanaryWithArgs(args);
   }
 
   @Test
@@ -237,4 +231,19 @@ public class TestCanaryTool {
     assertEquals("verify no read error count", 0, canary.getReadFailures().size());
   }
 
+  private void testZookeeperCanaryWithArgs(String[] args) throws Exception {
+    Integer port =
+      Iterables.getOnlyElement(testingUtility.getZkCluster().getClientPortList(), null);
+    testingUtility.getConfiguration().set(HConstants.ZOOKEEPER_QUORUM,
+      "localhost:" + port + "/hbase");
+    ExecutorService executor = new ScheduledThreadPoolExecutor(2);
+    Canary.ZookeeperStdOutSink sink = spy(new Canary.ZookeeperStdOutSink());
+    Canary canary = new Canary(executor, sink);
+    assertEquals(0, ToolRunner.run(testingUtility.getConfiguration(), canary, args));
+
+    String baseZnode = testingUtility.getConfiguration()
+      .get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
+    verify(sink, atLeastOnce())
+      .publishReadTiming(eq(baseZnode), eq("localhost:" + port), anyLong());
+  }
 }
\ No newline at end of file


[25/50] [abbrv] hbase git commit: HBASE-20523 PE tool should support configuring client side buffering sizes (Ram)

Posted by ap...@apache.org.
HBASE-20523 PE tool should support configuring client side buffering sizes (Ram)

Conflicts:
	hbase-server/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java


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

Branch: refs/heads/branch-1.3
Commit: 017cb75bd306226dbae7904b6181b99c7576be22
Parents: be4915e
Author: Vasudevan <ra...@intel.com>
Authored: Mon May 7 14:42:29 2018 +0530
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:19 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/PerformanceEvaluation.java     | 22 +++++++++++++++++++-
 .../hadoop/hbase/TestPerformanceEvaluation.java | 13 +++++++++++-
 2 files changed, 33 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/017cb75b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 66a2dce..c2418e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.BufferedMutatorParams;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Consistency;
@@ -621,6 +622,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     int families = 1;
     int caching = 30;
     boolean addColumns = true;
+    long bufferSize = 2l * 1024l * 1024l;
 
     public TestOptions() {}
 
@@ -664,6 +666,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
       this.columns = that.columns;
       this.families = that.families;
       this.caching = that.caching;
+      this.bufferSize = that.bufferSize;
     }
 
     public int getCaching() {
@@ -830,6 +833,14 @@ public class PerformanceEvaluation extends Configured implements Tool {
       this.valueSize = valueSize;
     }
 
+    public void setBufferSize(long bufferSize) {
+      this.bufferSize = bufferSize;
+    }
+
+    public long getBufferSize() {
+      return this.bufferSize;
+    }
+
     public void setPeriod(int period) {
       this.period = period;
     }
@@ -1251,7 +1262,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
 
     @Override
     void onStartup() throws IOException {
-      this.mutator = connection.getBufferedMutator(TableName.valueOf(opts.tableName));
+      BufferedMutatorParams p = new BufferedMutatorParams(TableName.valueOf(opts.tableName));
+      p.writeBufferSize(opts.bufferSize);
+      this.mutator = connection.getBufferedMutator(p);
     }
 
     @Override
@@ -2004,6 +2017,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     System.err.println(" columns         Columns to write per row. Default: 1");
     System.err.println(" families        Specify number of column families for the table. Default: 1");
     System.err.println(" caching         Scan caching to use. Default: 30");
+    System.err.println(" bufferSize      Set the value of client side buffering. Default: 2MB");
     System.err.println();
     System.err.println(" Note: -D properties will be applied to the conf used. ");
     System.err.println("  For example: ");
@@ -2240,6 +2254,12 @@ public class PerformanceEvaluation extends Configured implements Tool {
         continue;
       }
 
+      final String bufferSize = "--bufferSize=";
+      if (cmd.startsWith(bufferSize)) {
+        opts.bufferSize = Long.parseLong(cmd.substring(bufferSize.length()));
+        continue;
+      }
+
       if (isCommandClass(cmd)) {
         opts.cmdName = cmd;
         opts.numClientThreads = Integer.parseInt(args.remove());

http://git-wip-us.apache.org/repos/asf/hbase/blob/017cb75b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
index 3414e0a..cade7a4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -42,4 +43,14 @@ public class TestPerformanceEvaluation {
         mapper.readValue(optionsString, PerformanceEvaluation.TestOptions.class);
     assertTrue(optionsDeserialized.isAutoFlush());
   }
-}
\ No newline at end of file
+
+  @Test
+  public void testSetBufferSizeOption() {
+    PerformanceEvaluation.TestOptions opts = new PerformanceEvaluation.TestOptions();
+    long bufferSize = opts.getBufferSize();
+    assertEquals(bufferSize, 2l * 1024l * 1024l);
+    opts.setBufferSize(64l * 1024l);
+    bufferSize = opts.getBufferSize();
+    assertEquals(bufferSize, 64l * 1024l);
+  }
+}


[21/50] [abbrv] hbase git commit: HBASE-20052 TestRegionOpen#testNonExistentRegionReplica fails due to NPE

Posted by ap...@apache.org.
HBASE-20052 TestRegionOpen#testNonExistentRegionReplica fails due to NPE


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

Branch: refs/heads/branch-1.3
Commit: 8bc84fdc1b67ce2b28a2a8774defe669dfca1501
Parents: 453935f
Author: tedyu <yu...@gmail.com>
Authored: Thu Feb 22 12:41:13 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:18 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java   | 3 +--
 .../java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java | 2 --
 2 files changed, 1 insertion(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8bc84fdc/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 7672204..ba5249b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -906,12 +906,11 @@ public class HRegionFileSystem {
 
       // Write HRI to a file in case we need to recover hbase:meta
       regionFs.writeRegionInfoOnFilesystem(false);
-      return regionFs;
     } else {
       if (LOG.isDebugEnabled())
         LOG.debug("Skipping creation of .regioninfo file for " + regionInfo);
     }
-    return null;
+    return regionFs;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/8bc84fdc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
index 62aabce..f45df18 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -98,7 +97,6 @@ public class TestRegionOpen {
     assertEquals(2, exec.getCompletedTaskCount());
   }
 
-  @Ignore // Needs rewrite since HBASE-19391 which returns null out of createRegionOnFileSystem
   @Test
   public void testNonExistentRegionReplica() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());


[09/50] [abbrv] hbase git commit: HBASE-19816 Refresh repliation sinks on UnknownHostException

Posted by ap...@apache.org.
HBASE-19816 Refresh repliation sinks on UnknownHostException

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/branch-1.3
Commit: 109219d3c9e9eaac0342f6b70109215ce3fbeda6
Parents: 217ee60
Author: Scott Wilson <sw...@siftscience.com>
Authored: Wed Jan 17 13:18:30 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:16 2018 -0800

----------------------------------------------------------------------
 .../regionserver/HBaseInterClusterReplicationEndpoint.java        | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/109219d3/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index 85bd11a..e5f2a14 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -327,7 +328,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
               "call to the remote cluster timed out, which is usually " +
               "caused by a machine failure or a massive slowdown",
               this.socketTimeoutMultiplier);
-          } else if (ioe instanceof ConnectException) {
+          } else if (ioe instanceof ConnectException || ioe instanceof UnknownHostException) {
             LOG.warn("Peer is unavailable, rechecking all sinks: ", ioe);
             replicationSinkMgr.chooseSinks();
           } else {


[22/50] [abbrv] hbase git commit: HBASE-20047 AuthenticationTokenIdentifier should provide a toString

Posted by ap...@apache.org.
HBASE-20047 AuthenticationTokenIdentifier should provide a toString

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/branch-1.3
Commit: 35e94c98785095f83c7dd0c7b97b04678fb9c875
Parents: ecfa9a8
Author: maoling <ma...@sina.com>
Authored: Mon Mar 12 22:01:16 2018 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:18 2018 -0800

----------------------------------------------------------------------
 .../hbase/security/token/AuthenticationTokenIdentifier.java   | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/35e94c98/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenIdentifier.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenIdentifier.java
index 4299003..568ace7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenIdentifier.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenIdentifier.java
@@ -186,4 +186,11 @@ public class AuthenticationTokenIdentifier extends TokenIdentifier {
   public int hashCode() {
     return (int)sequenceNumber;
   }
+
+  @Override
+  public String toString() {
+    return "(username=" + username + ", keyId="
+            + keyId + ", issueDate=" + issueDate
+            + ", expirationDate=" + expirationDate + ", sequenceNumber=" + sequenceNumber + ")";
+  }
 }


[43/50] [abbrv] hbase git commit: HBASE-18549 Add metrics for failed replication queue recovery

Posted by ap...@apache.org.
HBASE-18549 Add metrics for failed replication queue recovery

Signed-off-by: Andrew Purtell <ap...@apache.org>

Conflicts:
	hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
	hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java


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

Branch: refs/heads/branch-1.3
Commit: 1ecfca3a5c7587a3a69f718ffbeec0d36cbcf4d1
Parents: 9675ad3
Author: Xu Cang <xc...@salesforce.com>
Authored: Wed Aug 29 16:39:57 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 19:25:33 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/replication/ReplicationQueuesZKImpl.java    | 5 ++++-
 .../regionserver/MetricsReplicationSourceSource.java         | 3 +++
 .../regionserver/MetricsReplicationGlobalSourceSource.java   | 8 ++++++++
 .../regionserver/MetricsReplicationSourceSourceImpl.java     | 5 +++++
 .../hadoop/hbase/replication/regionserver/MetricsSource.java | 4 ++++
 .../replication/regionserver/ReplicationSourceManager.java   | 3 +++
 .../hadoop/hbase/replication/TestReplicationEndpoint.java    | 8 +++++++-
 7 files changed, 34 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 3554a08..5ae2f5b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -490,7 +490,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
       }
       return new Pair<>(newCluster, logQueue);
     } catch (KeeperException e) {
-      LOG.warn("Got exception in copyQueueFromLockedRS: ", e);
+      LOG.warn("Got exception in copyQueueFromLockedRS: "+
+        " Possible problem: check if znode size exceeds jute.maxBuffer value. "
+          + "If so, increase it for both client and server side." ,e);
+
     } catch (InterruptedException e) {
       LOG.warn(e);
       Thread.currentThread().interrupt();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
index 1ed5a6b..4a2c807 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
@@ -47,6 +47,7 @@ public interface MetricsReplicationSourceSource {
   public static final String SOURCE_REPEATED_LOG_FILE_BYTES = "source.repeatedLogFileBytes";
   public static final String SOURCE_COMPLETED_LOGS = "source.completedLogs";
   public static final String SOURCE_COMPLETED_RECOVERY_QUEUES = "source.completedRecoverQueues";
+  public static final String SOURCE_FAILED_RECOVERY_QUEUES = "source.failedRecoverQueues";
 
   void setLastShippedAge(long age);
   void incrSizeOfLogQueue(int size);
@@ -70,4 +71,6 @@ public interface MetricsReplicationSourceSource {
   void incrRepeatedFileBytes(final long bytes);
   void incrCompletedWAL();
   void incrCompletedRecoveryQueue();
+  void incrFailedRecoveryQueue();
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
index f3a4ae5..4b62382 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
@@ -46,6 +46,7 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
   private final MutableFastCounter repeatedFileBytes;
   private final MutableFastCounter completedWAL;
   private final MutableFastCounter completedRecoveryQueue;
+  private final MutableFastCounter failedRecoveryQueue;
 
   public MetricsReplicationGlobalSourceSource(MetricsReplicationSourceImpl rms) {
     this.rms = rms;
@@ -82,6 +83,8 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
     repeatedFileBytes = rms.getMetricsRegistry().getCounter(SOURCE_REPEATED_LOG_FILE_BYTES, 0L);
     completedWAL = rms.getMetricsRegistry().getCounter(SOURCE_COMPLETED_LOGS, 0L);
     completedRecoveryQueue = rms.getMetricsRegistry().getCounter(SOURCE_COMPLETED_RECOVERY_QUEUES, 0L);
+    failedRecoveryQueue = rms.getMetricsRegistry()
+        .getCounter(SOURCE_FAILED_RECOVERY_QUEUES, 0L);
   }
 
   @Override public void setLastShippedAge(long age) {
@@ -189,9 +192,14 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
   public void incrCompletedWAL() {
     completedWAL.incr(1L);
   }
+
   @Override
   public void incrCompletedRecoveryQueue() {
     completedRecoveryQueue.incr(1L);
   }
 
+  @Override
+  public void incrFailedRecoveryQueue() {
+    failedRecoveryQueue.incr(1L);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
index 783a61f..58f091c 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
@@ -256,4 +256,9 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
   public void incrCompletedRecoveryQueue() {
     completedRecoveryQueue.incr(1L);
   }
+
+  @Override
+  public void incrFailedRecoveryQueue() {
+    /*no op*/
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
index 56baa05..4db48ca 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
@@ -290,6 +290,10 @@ public class MetricsSource {
     globalSourceSource.incrCompletedRecoveryQueue();
   }
 
+  public void incrFailedRecoveryQueue() {
+    globalSourceSource.incrFailedRecoveryQueue();
+  }
+
   public Map<String, MetricsReplicationSourceSource> getSingleSourceSourceByTable() {
     return singleSourceSourceByTable;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
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 8d19e22..d619f48 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
@@ -47,6 +47,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableDescriptors;
@@ -535,6 +536,8 @@ public class ReplicationSourceManager implements ReplicationListener {
     try {
       this.executor.execute(transfer);
     } catch (RejectedExecutionException ex) {
+      CompatibilitySingletonFactory.getInstance(MetricsReplicationSourceFactory.class)
+        .getGlobalSource().incrFailedRecoveryQueue();
       LOG.info("Cancelling the transfer of " + rsZnode + " because of " + ex.getMessage());
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecfca3a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
index c3822c1..dd2e2f0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
@@ -18,9 +18,11 @@
 
 package org.apache.hadoop.hbase.replication;
 
+import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -285,6 +287,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
     MetricsReplicationSourceSource singleSourceSource = new MetricsReplicationSourceSourceImpl(singleRms, id);
     MetricsReplicationSourceSource globalSourceSource = new MetricsReplicationGlobalSourceSource(globalRms);
     MetricsReplicationSourceSource spyglobalSourceSource = spy(globalSourceSource);
+    doNothing().when(spyglobalSourceSource).incrFailedRecoveryQueue();
     Map<String, MetricsReplicationSourceSource> singleSourceSourceByTable = new HashMap<>();
     MetricsSource source = new MetricsSource(id, singleSourceSource, spyglobalSourceSource,
         singleSourceSourceByTable);
@@ -304,6 +307,9 @@ public class TestReplicationEndpoint extends TestReplicationBase {
     // cannot put more concreate value here to verify because the age is arbitrary.
     // as long as it's greater than 0, we see it as correct answer.
     Assert.assertTrue(msr.getLastShippedAge() > 0);
+
+    source.incrFailedRecoveryQueue();
+    verify(spyglobalSourceSource).incrFailedRecoveryQueue();
   }
 
   private void doPut(byte[] row) throws IOException {


[03/50] [abbrv] hbase git commit: HBASE-17991 Add more details about compaction queue on /dump (Guangxu Cheng)

Posted by ap...@apache.org.
HBASE-17991 Add more details about compaction queue on /dump (Guangxu Cheng)


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

Branch: refs/heads/branch-1.3
Commit: 3e0d693935e9a03cf4b9a078b32f2b7cc4e0c95f
Parents: ed4f7d1
Author: tedyu <yu...@gmail.com>
Authored: Thu May 4 08:28:02 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:16 2018 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/CompactSplitThread.java    | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3e0d6939/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
index 6f95a09..71a8803 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
@@ -460,6 +460,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
     private int queuedPriority;
     private ThreadPoolExecutor parent;
     private User user;
+    private long time;
 
     public CompactionRunner(Store store, Region region,
         CompactionContext compaction, ThreadPoolExecutor parent, User user) {
@@ -471,12 +472,14 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
           ? store.getCompactPriority() : compaction.getRequest().getPriority();
       this.parent = parent;
       this.user = user;
+      this.time =  System.currentTimeMillis();
     }
 
     @Override
     public String toString() {
       return (this.compaction != null) ? ("Request = " + compaction.getRequest())
-          : ("Store = " + store.toString() + ", pri = " + queuedPriority);
+          : ("regionName = " + region.toString() + ", storeName = " + store.toString() +
+             ", priority = " + queuedPriority + ", time = " + time);
     }
 
     private void doCompaction(User user) {


[36/50] [abbrv] hbase git commit: HBASE-20558 Port HBASE-17854 (Use StealJobQueue in HFileCleaner after HBASE-17215) to branch-1

Posted by ap...@apache.org.
HBASE-20558 Port HBASE-17854 (Use StealJobQueue in HFileCleaner after HBASE-17215) to branch-1

The third port commit of HBASE-20555

Signed-off-by: Zach York <zy...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 976f07e87cbeb35b80a7ca8eea46f973e932bf83
Parents: 30b1dc0
Author: TAK LON WU <wu...@amazon.com>
Authored: Mon Jul 9 16:34:06 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:20 2018 -0800

----------------------------------------------------------------------
 .../hbase/master/cleaner/HFileCleaner.java      | 98 +++++++++++++-------
 .../apache/hadoop/hbase/util/StealJobQueue.java | 29 +++++-
 .../hbase/master/cleaner/TestHFileCleaner.java  | 28 +++---
 .../hadoop/hbase/util/TestStealJobQueue.java    | 14 +--
 4 files changed, 113 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/976f07e8/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
index defe851..70548b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -35,6 +34,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.util.StealJobQueue;
 
 /**
  * This Chore, every time it runs, will clear the HFiles in the hfile archive
@@ -56,23 +56,23 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       "hbase.regionserver.thread.hfilecleaner.throttle";
   public final static int DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD = 64 * 1024 * 1024;// 64M
 
-  // Configuration key for large queue size
-  public final static String LARGE_HFILE_DELETE_QUEUE_SIZE =
+  // Configuration key for large queue initial size
+  public final static String LARGE_HFILE_QUEUE_INIT_SIZE =
       "hbase.regionserver.hfilecleaner.large.queue.size";
-  public final static int DEFAULT_LARGE_HFILE_DELETE_QUEUE_SIZE = 1048576;
+  public final static int DEFAULT_LARGE_HFILE_QUEUE_INIT_SIZE = 10240;
 
-  // Configuration key for small queue size
-  public final static String SMALL_HFILE_DELETE_QUEUE_SIZE =
+  // Configuration key for small queue initial size
+  public final static String SMALL_HFILE_QUEUE_INIT_SIZE =
       "hbase.regionserver.hfilecleaner.small.queue.size";
-  public final static int DEFAULT_SMALL_HFILE_DELETE_QUEUE_SIZE = 1048576;
+  public final static int DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE = 10240;
 
   private static final Log LOG = LogFactory.getLog(HFileCleaner.class);
 
-  BlockingQueue<HFileDeleteTask> largeFileQueue;
+  StealJobQueue<HFileDeleteTask> largeFileQueue;
   BlockingQueue<HFileDeleteTask> smallFileQueue;
   private int throttlePoint;
-  private int largeQueueSize;
-  private int smallQueueSize;
+  private int largeQueueInitSize;
+  private int smallQueueInitSize;
   private List<Thread> threads = new ArrayList<Thread>();
   private boolean running;
 
@@ -93,12 +93,12 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       directory, MASTER_HFILE_CLEANER_PLUGINS, params);
     throttlePoint =
         conf.getInt(HFILE_DELETE_THROTTLE_THRESHOLD, DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD);
-    largeQueueSize =
-        conf.getInt(LARGE_HFILE_DELETE_QUEUE_SIZE, DEFAULT_LARGE_HFILE_DELETE_QUEUE_SIZE);
-    smallQueueSize =
-        conf.getInt(SMALL_HFILE_DELETE_QUEUE_SIZE, DEFAULT_SMALL_HFILE_DELETE_QUEUE_SIZE);
-    largeFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(largeQueueSize);
-    smallFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(smallQueueSize);
+    largeQueueInitSize =
+        conf.getInt(LARGE_HFILE_QUEUE_INIT_SIZE, DEFAULT_LARGE_HFILE_QUEUE_INIT_SIZE);
+    smallQueueInitSize =
+        conf.getInt(SMALL_HFILE_QUEUE_INIT_SIZE, DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE);
+    largeFileQueue = new StealJobQueue<>(largeQueueInitSize, smallQueueInitSize);
+    smallFileQueue = largeFileQueue.getStealFromQueue();
     startHFileDeleteThreads();
   }
 
@@ -151,6 +151,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   private boolean dispatch(HFileDeleteTask task) {
     if (task.fileLength >= this.throttlePoint) {
       if (!this.largeFileQueue.offer(task)) {
+        // should never arrive here as long as we use PriorityQueue
         if (LOG.isTraceEnabled()) {
           LOG.trace("Large file deletion queue is full");
         }
@@ -158,6 +159,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       }
     } else {
       if (!this.smallFileQueue.offer(task)) {
+        // should never arrive here as long as we use PriorityQueue
         if (LOG.isTraceEnabled()) {
           LOG.trace("Small file deletion queue is full");
         }
@@ -231,7 +233,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
           }
           task.setResult(succeed);
           if (succeed) {
-            countDeletedFiles(queue == largeFileQueue);
+            countDeletedFiles(task.fileLength >= throttlePoint, queue == largeFileQueue);
           }
         }
       }
@@ -243,8 +245,8 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   }
 
   // Currently only for testing purpose
-  private void countDeletedFiles(boolean isLarge) {
-    if (isLarge) {
+  private void countDeletedFiles(boolean isLargeFile, boolean fromLargeQueue) {
+    if (isLargeFile) {
       if (deletedLargeFiles == Long.MAX_VALUE) {
         LOG.info("Deleted more than Long.MAX_VALUE large files, reset counter to 0");
         deletedLargeFiles = 0L;
@@ -255,6 +257,9 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
         LOG.info("Deleted more than Long.MAX_VALUE small files, reset counter to 0");
         deletedSmallFiles = 0L;
       }
+      if (fromLargeQueue && LOG.isTraceEnabled()) {
+        LOG.trace("Stolen a small file deletion task in large file thread");
+      }
       deletedSmallFiles++;
     }
   }
@@ -272,7 +277,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
     }
   }
 
-  static class HFileDeleteTask {
+  static class HFileDeleteTask implements Comparable<HFileDeleteTask> {
     private static final long MAX_WAIT = 60 * 1000L;
     private static final long WAIT_UNIT = 1000L;
 
@@ -314,6 +319,31 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       }
       return this.result;
     }
+
+    @Override
+    public int compareTo(HFileDeleteTask o) {
+      long sub = this.fileLength - o.fileLength;
+      // smaller value with higher priority in PriorityQueue, and we intent to delete the larger
+      // file first.
+      return (sub > 0) ? -1 : (sub < 0 ? 1 : 0);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || !(o instanceof HFileDeleteTask)) {
+        return false;
+      }
+      HFileDeleteTask otherTask = (HFileDeleteTask) o;
+      return this.filePath.equals(otherTask.filePath) && (this.fileLength == otherTask.fileLength);
+    }
+
+    @Override
+    public int hashCode() {
+      return filePath.hashCode();
+    }
   }
 
   @VisibleForTesting
@@ -332,13 +362,13 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   }
 
   @VisibleForTesting
-  public long getLargeQueueSize() {
-    return largeQueueSize;
+  public long getLargeQueueInitSize() {
+    return largeQueueInitSize;
   }
 
   @VisibleForTesting
-  public long getSmallQueueSize() {
-    return smallQueueSize;
+  public long getSmallQueueInitSize() {
+    return smallQueueInitSize;
   }
 
   @VisibleForTesting
@@ -350,15 +380,15 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   public void onConfigurationChange(Configuration conf) {
     StringBuilder builder = new StringBuilder();
     builder.append("Updating configuration for HFileCleaner, previous throttle point: ")
-        .append(throttlePoint).append(", largeQueueSize: ").append(largeQueueSize)
-        .append(", smallQueueSize: ").append(smallQueueSize);
+        .append(throttlePoint).append(", largeQueueInitSize: ").append(largeQueueInitSize)
+        .append(", smallQueueInitSize: ").append(smallQueueInitSize);
     stopHFileDeleteThreads();
     this.throttlePoint =
         conf.getInt(HFILE_DELETE_THROTTLE_THRESHOLD, DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD);
-    this.largeQueueSize =
-        conf.getInt(LARGE_HFILE_DELETE_QUEUE_SIZE, DEFAULT_LARGE_HFILE_DELETE_QUEUE_SIZE);
-    this.smallQueueSize =
-        conf.getInt(SMALL_HFILE_DELETE_QUEUE_SIZE, DEFAULT_SMALL_HFILE_DELETE_QUEUE_SIZE);
+    this.largeQueueInitSize =
+        conf.getInt(LARGE_HFILE_QUEUE_INIT_SIZE, DEFAULT_LARGE_HFILE_QUEUE_INIT_SIZE);
+    this.smallQueueInitSize =
+        conf.getInt(SMALL_HFILE_QUEUE_INIT_SIZE, DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE);
     // record the left over tasks
     List<HFileDeleteTask> leftOverTasks = new ArrayList<>();
     for (HFileDeleteTask task : largeFileQueue) {
@@ -367,11 +397,11 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
     for (HFileDeleteTask task : smallFileQueue) {
       leftOverTasks.add(task);
     }
-    largeFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(largeQueueSize);
-    smallFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(smallQueueSize);
+    largeFileQueue = new StealJobQueue<>(largeQueueInitSize, smallQueueInitSize);
+    smallFileQueue = largeFileQueue.getStealFromQueue();
     threads.clear();
-    builder.append("; new throttle point: ").append(throttlePoint).append(", largeQueueSize: ")
-        .append(largeQueueSize).append(", smallQueueSize: ").append(smallQueueSize);
+    builder.append("; new throttle point: ").append(throttlePoint).append(", largeQueueInitSize: ")
+        .append(largeQueueInitSize).append(", smallQueueInitSize: ").append(smallQueueInitSize);
     LOG.debug(builder.toString());
     startHFileDeleteThreads();
     // re-dispatch the left over tasks

http://git-wip-us.apache.org/repos/asf/hbase/blob/976f07e8/hbase-server/src/main/java/org/apache/hadoop/hbase/util/StealJobQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/StealJobQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/StealJobQueue.java
index 74f0747..c6d8ee0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/StealJobQueue.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/StealJobQueue.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.PriorityBlockingQueue;
 import java.util.concurrent.TimeUnit;
@@ -27,6 +25,8 @@ import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
 /**
  * This queue allows a ThreadPoolExecutor to steal jobs from another ThreadPoolExecutor.
  * This queue also acts as the factory for creating the PriorityBlockingQueue to be used in the
@@ -48,6 +48,24 @@ public class StealJobQueue<T> extends PriorityBlockingQueue<T> {
 
   public StealJobQueue() {
     this.stealFromQueue = new PriorityBlockingQueue<T>() {
+
+      @Override
+      public boolean offer(T t) {
+        lock.lock();
+        try {
+          notEmpty.signal();
+          return super.offer(t);
+        } finally {
+          lock.unlock();
+        }
+      }
+    };
+  }
+
+  public StealJobQueue(int initCapacity, int stealFromQueueInitCapacity) {
+    super(initCapacity);
+    this.stealFromQueue = new PriorityBlockingQueue<T>(stealFromQueueInitCapacity) {
+
       @Override
       public boolean offer(T t) {
         lock.lock();
@@ -61,6 +79,10 @@ public class StealJobQueue<T> extends PriorityBlockingQueue<T> {
     };
   }
 
+  /**
+   * Get a queue whose job might be stolen by the consumer of this original queue
+   * @return the queue whose job could be stolen
+   */
   public BlockingQueue<T> getStealFromQueue() {
     return stealFromQueue;
   }
@@ -108,8 +130,9 @@ public class StealJobQueue<T> extends PriorityBlockingQueue<T> {
           retVal = stealFromQueue.poll();
         }
         if (retVal == null) {
-          if (nanos <= 0)
+          if (nanos <= 0) {
             return null;
+          }
           nanos = notEmpty.awaitNanos(nanos);
         } else {
           return retVal;

http://git-wip-us.apache.org/repos/asf/hbase/blob/976f07e8/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 5712729..18afafa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -316,9 +316,9 @@ public class TestHFileCleaner {
   public void testOnConfigurationChange() throws Exception {
     // constants
     final int ORIGINAL_THROTTLE_POINT = 512 * 1024;
-    final int ORIGINAL_QUEUE_SIZE = 512;
+    final int ORIGINAL_QUEUE_INIT_SIZE = 512;
     final int UPDATE_THROTTLE_POINT = 1024;// small enough to change large/small check
-    final int UPDATE_QUEUE_SIZE = 1024;
+    final int UPDATE_QUEUE_INIT_SIZE = 1024;
     final int LARGE_FILE_NUM = 5;
     final int SMALL_FILE_NUM = 20;
 
@@ -326,8 +326,8 @@ public class TestHFileCleaner {
     // no cleaner policies = delete all files
     conf.setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, "");
     conf.setInt(HFileCleaner.HFILE_DELETE_THROTTLE_THRESHOLD, ORIGINAL_THROTTLE_POINT);
-    conf.setInt(HFileCleaner.LARGE_HFILE_DELETE_QUEUE_SIZE, ORIGINAL_QUEUE_SIZE);
-    conf.setInt(HFileCleaner.SMALL_HFILE_DELETE_QUEUE_SIZE, ORIGINAL_QUEUE_SIZE);
+    conf.setInt(HFileCleaner.LARGE_HFILE_QUEUE_INIT_SIZE, ORIGINAL_QUEUE_INIT_SIZE);
+    conf.setInt(HFileCleaner.SMALL_HFILE_QUEUE_INIT_SIZE, ORIGINAL_QUEUE_INIT_SIZE);
     Server server = new DummyServer();
     Path archivedHfileDir =
         new Path(UTIL.getDataTestDirOnTestFS(), HConstants.HFILE_ARCHIVE_DIRECTORY);
@@ -336,8 +336,8 @@ public class TestHFileCleaner {
     FileSystem fs = UTIL.getDFSCluster().getFileSystem();
     final HFileCleaner cleaner = new HFileCleaner(1000, server, conf, fs, archivedHfileDir);
     Assert.assertEquals(ORIGINAL_THROTTLE_POINT, cleaner.getThrottlePoint());
-    Assert.assertEquals(ORIGINAL_QUEUE_SIZE, cleaner.getLargeQueueSize());
-    Assert.assertEquals(ORIGINAL_QUEUE_SIZE, cleaner.getSmallQueueSize());
+    Assert.assertEquals(ORIGINAL_QUEUE_INIT_SIZE, cleaner.getLargeQueueInitSize());
+    Assert.assertEquals(ORIGINAL_QUEUE_INIT_SIZE, cleaner.getSmallQueueInitSize());
 
     // clean up archive directory and create files for testing
     fs.delete(archivedHfileDir, true);
@@ -353,22 +353,24 @@ public class TestHFileCleaner {
     };
     t.setDaemon(true);
     t.start();
-    // let the cleaner run for some while
-    Thread.sleep(20);
+    // wait until file clean started
+    while (cleaner.getNumOfDeletedSmallFiles() == 0) {
+      Thread.yield();
+    }
 
     // trigger configuration change
     Configuration newConf = new Configuration(conf);
     newConf.setInt(HFileCleaner.HFILE_DELETE_THROTTLE_THRESHOLD, UPDATE_THROTTLE_POINT);
-    newConf.setInt(HFileCleaner.LARGE_HFILE_DELETE_QUEUE_SIZE, UPDATE_QUEUE_SIZE);
-    newConf.setInt(HFileCleaner.SMALL_HFILE_DELETE_QUEUE_SIZE, UPDATE_QUEUE_SIZE);
-    cleaner.onConfigurationChange(newConf);
+    newConf.setInt(HFileCleaner.LARGE_HFILE_QUEUE_INIT_SIZE, UPDATE_QUEUE_INIT_SIZE);
+    newConf.setInt(HFileCleaner.SMALL_HFILE_QUEUE_INIT_SIZE, UPDATE_QUEUE_INIT_SIZE);
     LOG.debug("File deleted from large queue: " + cleaner.getNumOfDeletedLargeFiles()
         + "; from small queue: " + cleaner.getNumOfDeletedSmallFiles());
+    cleaner.onConfigurationChange(newConf);
 
     // check values after change
     Assert.assertEquals(UPDATE_THROTTLE_POINT, cleaner.getThrottlePoint());
-    Assert.assertEquals(UPDATE_QUEUE_SIZE, cleaner.getLargeQueueSize());
-    Assert.assertEquals(UPDATE_QUEUE_SIZE, cleaner.getSmallQueueSize());
+    Assert.assertEquals(UPDATE_QUEUE_INIT_SIZE, cleaner.getLargeQueueInitSize());
+    Assert.assertEquals(UPDATE_QUEUE_INIT_SIZE, cleaner.getSmallQueueInitSize());
     Assert.assertEquals(2, cleaner.getCleanerThreads().size());
 
     // wait until clean done and check

http://git-wip-us.apache.org/repos/asf/hbase/blob/976f07e8/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java
index 59a4115..531e4f4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java
@@ -18,10 +18,9 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CountDownLatch;
@@ -29,14 +28,17 @@ import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static org.junit.Assert.*;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 
 @Category(SmallTests.class)
 public class TestStealJobQueue {
 
   StealJobQueue<Integer> stealJobQueue;
-  BlockingQueue stealFromQueue;
+  BlockingQueue<Integer> stealFromQueue;
 
   @Before
   public void setup() {


[42/50] [abbrv] hbase git commit: HBASE-20734 Colocate recovered edits directory with hbase.wal.dir

Posted by ap...@apache.org.
HBASE-20734 Colocate recovered edits directory with hbase.wal.dir

Signed-off-by: Andrew Purtell <ap...@apache.org>
Signed-off-by: Reid Chan <re...@apache.org>

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java


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

Branch: refs/heads/branch-1.3
Commit: 9675ad387627588bdf0c37f56f701a0d0074d196
Parents: 1d362f6
Author: Zach York <zy...@apache.org>
Authored: Wed Jun 27 16:18:53 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 19:22:42 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 190 ++++++++++++-------
 .../org/apache/hadoop/hbase/util/FSUtils.java   |  27 +++
 .../apache/hadoop/hbase/wal/WALSplitter.java    | 166 ++++++++--------
 .../hadoop/hbase/regionserver/TestHRegion.java  |   8 +-
 .../hbase/regionserver/TestRecoveredEdits.java  |   2 +-
 .../hbase/regionserver/wal/TestWALReplay.java   |   6 +-
 .../apache/hadoop/hbase/wal/TestWALFactory.java |   2 +-
 .../apache/hadoop/hbase/wal/TestWALSplit.java   |  50 ++---
 8 files changed, 263 insertions(+), 188 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9675ad38/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
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 9d8c0c6..d41e7cb 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
@@ -41,6 +41,7 @@ import java.util.NavigableSet;
 import java.util.RandomAccess;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ConcurrentHashMap;
@@ -309,6 +310,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private final int rowLockWaitDuration;
   static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
 
+  private Path regionDir;
+  private FileSystem walFS;
+
   // The internal wait duration to acquire a lock before read/update
   // from the region. It is not per row. The purpose of this wait time
   // is to avoid waiting a long time while the region is busy, so that
@@ -840,7 +844,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     if (ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) {
       // Recover any edits if available.
       maxSeqId = Math.max(maxSeqId,
-        replayRecoveredEditsIfAny(this.fs.getRegionDir(), maxSeqIdInStores, reporter, status));
+        replayRecoveredEditsIfAny(maxSeqIdInStores, reporter, status));
       // Make sure mvcc is up to max.
       this.mvcc.advanceTo(maxSeqId);
     }
@@ -884,8 +888,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // is opened before recovery completes. So we add a safety bumper to avoid new sequence number
     // overlaps used sequence numbers
     if (this.writestate.writesEnabled) {
-      nextSeqid = WALSplitter.writeRegionSequenceIdFile(this.fs.getFileSystem(), this.fs
-          .getRegionDir(), nextSeqid, (this.recovering ? (this.flushPerChanges + 10000000) : 1));
+      nextSeqid = WALSplitter.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(),
+          nextSeqid, (this.recovering ? (this.flushPerChanges + 10000000) : 1));
     } else {
       nextSeqid++;
     }
@@ -1023,11 +1027,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       getRegionServerServices().getServerName(), storeFiles);
     WALUtil.writeRegionEventMarker(wal, getTableDesc(), getRegionInfo(), regionEventDesc, mvcc);
 
-    // Store SeqId in HDFS when a region closes
+    // Store SeqId in WAL FileSystem when a region closes
     // checking region folder exists is due to many tests which delete the table folder while a
     // table is still online
-    if (this.fs.getFileSystem().exists(this.fs.getRegionDir())) {
-      WALSplitter.writeRegionSequenceIdFile(this.fs.getFileSystem(), this.fs.getRegionDir(),
+    if (getWalFileSystem().exists(getWALRegionDir())) {
+      WALSplitter.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(),
         mvcc.getReadPoint(), 0);
     }
   }
@@ -1702,6 +1706,32 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return this.fs;
   }
 
+  /** @return the WAL {@link HRegionFileSystem} used by this region */
+  HRegionFileSystem getRegionWALFileSystem() throws IOException {
+    return new HRegionFileSystem(conf, getWalFileSystem(),
+        FSUtils.getWALTableDir(conf, htableDescriptor.getTableName()), fs.getRegionInfo());
+  }
+
+  /** @return WAL {@link FileSystem} being used by this region */
+  FileSystem getWalFileSystem() throws IOException {
+    if (walFS == null) {
+      walFS = FSUtils.getWALFileSystem(conf);
+    }
+    return walFS;
+  }
+
+  /**
+   * @return the Region Directory under the WALRootDir
+   * @throws IOException if there is an error getting WALRootDir
+   */
+  @VisibleForTesting
+  public Path getWALRegionDir() throws IOException {
+    if (regionDir == null) {
+      regionDir = FSUtils.getWALRegionDir(conf, fs.getRegionInfo());
+    }
+    return regionDir;
+  }
+
   @Override
   public long getEarliestFlushTimeForAllStores() {
     return Collections.min(lastStoreFlushTimeMap.values());
@@ -4106,8 +4136,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
    * @throws IOException
    */
-  protected long replayRecoveredEditsIfAny(final Path regiondir,
-      Map<byte[], Long> maxSeqIdInStores,
+  protected long replayRecoveredEditsIfAny(Map<byte[], Long> maxSeqIdInStores,
       final CancelableProgressable reporter, final MonitoredTask status)
       throws IOException {
     long minSeqIdForTheRegion = -1;
@@ -4118,30 +4147,92 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
     long seqid = minSeqIdForTheRegion;
 
-    FileSystem fs = this.fs.getFileSystem();
-    NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regiondir);
+    FileSystem walFS = getWalFileSystem();
+    Path regionDir = getWALRegionDir();
+    FileSystem rootFS = getFilesystem();
+    Path defaultRegionDir = getRegionDir(FSUtils.getRootDir(conf), getRegionInfo());
+
+    // This is to ensure backwards compatability with HBASE-20723 where recovered edits can appear
+    // under the root dir even if walDir is set.
+    NavigableSet<Path> filesUnderRootDir = null;
+    if (!regionDir.equals(defaultRegionDir)) {
+      filesUnderRootDir =
+          WALSplitter.getSplitEditFilesSorted(rootFS, defaultRegionDir);
+      seqid = Math.max(seqid,
+          replayRecoveredEditsForPaths(minSeqIdForTheRegion, rootFS, filesUnderRootDir, reporter,
+              defaultRegionDir));
+    }
+    NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(walFS, regionDir);
+    seqid = Math.max(seqid, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
+        files, reporter, regionDir));
+
+    if (seqid > minSeqIdForTheRegion) {
+      // Then we added some edits to memory. Flush and cleanup split edit files.
+      internalFlushcache(null, seqid, stores.values(), status, false);
+    }
+    // Now delete the content of recovered edits.  We're done w/ them.
+    if (files.size() > 0 && this.conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
+      // For debugging data loss issues!
+      // If this flag is set, make use of the hfile archiving by making recovered.edits a fake
+      // column family. Have to fake out file type too by casting our recovered.edits as storefiles
+      String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regionDir).getName();
+      Set<StoreFile> fakeStoreFiles = new HashSet<>(files.size());
+      for (Path file: files) {
+        fakeStoreFiles.add(
+            new StoreFile(walFS, file, this.conf, null, null));
+      }
+      getRegionWALFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles);
+    } else {
+      if (filesUnderRootDir != null) {
+        for (Path file : filesUnderRootDir) {
+          if (!rootFS.delete(file, false)) {
+            LOG.error("Failed delete of {} under root directory." + file);
+          } else {
+            LOG.debug("Deleted recovered.edits root directory file=" + file);
+          }
+        }
+      }
+      for (Path file: files) {
+        if (!walFS.delete(file, false)) {
+          LOG.error("Failed delete of " + file);
+        } else {
+          LOG.debug("Deleted recovered.edits file=" + file);
+        }
+      }
+    }
+    return seqid;
+  }
+
+  private long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, FileSystem fs,
+      final NavigableSet<Path> files, final CancelableProgressable reporter, final Path regionDir)
+      throws IOException {
+    long seqid = minSeqIdForTheRegion;
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Found " + (files == null ? 0 : files.size())
-        + " recovered edits file(s) under " + regiondir);
+      LOG.debug("Found " + (files == null ? 0 : files.size()) +
+          " recovered edits file(s) under " + regionDir);
     }
 
-    if (files == null || files.isEmpty()) return seqid;
+    if (files == null || files.isEmpty()) {
+      return seqid;
+    }
 
-    for (Path edits: files) {
-      if (edits == null || !fs.exists(edits)) {
+    for (Path edits : files) {
+      if (edits == null || !walFS.exists(edits)) {
         LOG.warn("Null or non-existent edits file: " + edits);
         continue;
       }
-      if (isZeroLengthThenDelete(fs, edits)) continue;
+      if (isZeroLengthThenDelete(walFS, edits)) {
+        continue;
+      }
 
       long maxSeqId;
       String fileName = edits.getName();
       maxSeqId = Math.abs(Long.parseLong(fileName));
       if (maxSeqId <= minSeqIdForTheRegion) {
         if (LOG.isDebugEnabled()) {
-          String msg = "Maximum sequenceid for this wal is " + maxSeqId
-            + " and minimum sequenceid for the region is " + minSeqIdForTheRegion
-            + ", skipped the whole file, path=" + edits;
+          String msg = "Maximum sequenceid for this wal is " + maxSeqId +
+              " and minimum sequenceid for the region is " + minSeqIdForTheRegion
+              + ", skipped the whole file, path=" + edits;
           LOG.debug(msg);
         }
         continue;
@@ -4150,77 +4241,42 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       try {
         // replay the edits. Replay can return -1 if everything is skipped, only update
         // if seqId is greater
-        seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter));
+        seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter, fs));
       } catch (IOException e) {
-        boolean skipErrors = conf.getBoolean(
-            HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
-            conf.getBoolean(
-                "hbase.skip.errors",
+        boolean skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
+            conf.getBoolean("hbase.skip.errors",
                 HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS));
         if (conf.get("hbase.skip.errors") != null) {
-          LOG.warn(
-              "The property 'hbase.skip.errors' has been deprecated. Please use " +
+          LOG.warn("The property 'hbase.skip.errors' has been deprecated. Please use " +
               HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + " instead.");
         }
         if (skipErrors) {
-          Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
-          LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
-              + "=true so continuing. Renamed " + edits +
-              " as " + p, e);
+          Path p = WALSplitter.moveAsideBadEditsFile(walFS, edits);
+          LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS +
+              "=true so continuing. Renamed " + edits + " as " + p, e);
         } else {
           throw e;
         }
       }
     }
-    // The edits size added into rsAccounting during this replaying will not
-    // be required any more. So just clear it.
-    if (this.rsAccounting != null) {
-      this.rsAccounting.clearRegionReplayEditsSize(getRegionInfo().getRegionName());
-    }
-    if (seqid > minSeqIdForTheRegion) {
-      // Then we added some edits to memory. Flush and cleanup split edit files.
-      internalFlushcache(null, seqid, stores.values(), status, false);
-    }
-    // Now delete the content of recovered edits.  We're done w/ them.
-    if (files.size() > 0 && this.conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
-      // For debugging data loss issues!
-      // If this flag is set, make use of the hfile archiving by making recovered.edits a fake
-      // column family. Have to fake out file type too by casting our recovered.edits as storefiles
-      String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regiondir).getName();
-      Set<StoreFile> fakeStoreFiles = new HashSet<StoreFile>(files.size());
-      for (Path file: files) {
-        fakeStoreFiles.add(new StoreFile(getRegionFileSystem().getFileSystem(), file, this.conf,
-          null, null));
-      }
-      getRegionFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles);
-    } else {
-      for (Path file: files) {
-        if (!fs.delete(file, false)) {
-          LOG.error("Failed delete of " + file);
-        } else {
-          LOG.debug("Deleted recovered.edits file=" + file);
-        }
-      }
-    }
     return seqid;
   }
 
-  /*
+  /**
    * @param edits File of recovered edits.
    * @param maxSeqIdInStores Maximum sequenceid found in each store.  Edits in wal
-   * must be larger than this to be replayed for each store.
-   * @param reporter
+   *                         must be larger than this to be replayed for each store.
+   * @param reporter CacelableProgressable reporter
    * @return the sequence id of the last edit added to this region out of the
    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
    * @throws IOException
    */
-  private long replayRecoveredEdits(final Path edits,
-      Map<byte[], Long> maxSeqIdInStores, final CancelableProgressable reporter)
+  private long replayRecoveredEdits(final Path edits, Map<byte[], Long> maxSeqIdInStores,
+      final CancelableProgressable reporter, final FileSystem fs)
     throws IOException {
     String msg = "Replaying edits from " + edits;
     LOG.info(msg);
     MonitoredTask status = TaskMonitor.get().createStatus(msg);
-    FileSystem fs = this.fs.getFileSystem();
 
     status.setStatus("Opening recovered edits");
     WAL.Reader reader = null;
@@ -8174,7 +8230,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT +
       ClassSize.ARRAY +
-      48 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
+      50 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
       (14 * Bytes.SIZEOF_LONG) +
       5 * Bytes.SIZEOF_BOOLEAN);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9675ad38/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 5784b74..4afb1e2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -1060,6 +1060,20 @@ public abstract class FSUtils {
   }
 
   /**
+   * Returns the WAL region directory based on the region info
+   * @param conf configuration to determine WALRootDir
+   * @param regionInfo used to get region and table
+   * @return the region directory used to store WALs under the WALRootDir
+   * @throws IOException if there is an exception determining the WALRootDir
+   */
+  public static Path getWALRegionDir(final Configuration conf,
+      final HRegionInfo regionInfo)
+      throws IOException {
+    return new Path(getWALTableDir(conf, regionInfo.getTable()),
+        regionInfo.getEncodedName());
+  }
+
+  /**
    * Checks if meta region exists
    *
    * @param fs file system
@@ -1195,6 +1209,19 @@ public abstract class FSUtils {
   }
 
   /**
+   * Returns the Table directory under the WALRootDir for the specified table name
+   * @param conf configuration used to get the WALRootDir
+   * @param tableName Table to get the directory for
+   * @return a path to the WAL table directory for the specified table
+   * @throws IOException if there is an exception determining the WALRootDir
+   */
+  public static Path getWALTableDir(final Configuration conf, final TableName tableName)
+      throws IOException {
+    return new Path(new Path(getWALRootDir(conf), tableName.getNamespaceAsString()),
+        tableName.getQualifierAsString());
+  }
+
+  /**
    * Returns the {@link org.apache.hadoop.hbase.TableName} object representing
    * the table directory under
    * path rootdir

http://git-wip-us.apache.org/repos/asf/hbase/blob/9675ad38/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 45dbb11..47a5596 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -141,7 +141,7 @@ public class WALSplitter {
 
   // Parameters for split process
   protected final Path walDir;
-  protected final FileSystem fs;
+  protected final FileSystem walFS;
   protected final Configuration conf;
 
   // Major subcomponents of the split process.
@@ -188,14 +188,14 @@ public class WALSplitter {
 
   @VisibleForTesting
   WALSplitter(final WALFactory factory, Configuration conf, Path walDir,
-      FileSystem fs, LastSequenceId idChecker,
+      FileSystem walFS, LastSequenceId idChecker,
       CoordinatedStateManager csm, RecoveryMode mode) {
     this.conf = HBaseConfiguration.create(conf);
     String codecClassName = conf
         .get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName());
     this.conf.set(HConstants.RPC_CODEC_CONF_KEY, codecClassName);
     this.walDir = walDir;
-    this.fs = fs;
+    this.walFS = walFS;
     this.sequenceIdChecker = idChecker;
     this.csm = (BaseCoordinatedStateManager)csm;
     this.walFactory = factory;
@@ -236,7 +236,7 @@ public class WALSplitter {
    * <p>
    * @param rootDir
    * @param logfile
-   * @param fs
+   * @param walFS FileSystem to use for WAL reading and splitting
    * @param conf
    * @param reporter
    * @param idChecker
@@ -244,10 +244,10 @@ public class WALSplitter {
    * @return false if it is interrupted by the progress-able.
    * @throws IOException
    */
-  public static boolean splitLogFile(Path walDir, FileStatus logfile, FileSystem fs,
+  public static boolean splitLogFile(Path walDir, FileStatus logfile, FileSystem walFS,
       Configuration conf, CancelableProgressable reporter, LastSequenceId idChecker,
       CoordinatedStateManager cp, RecoveryMode mode, final WALFactory factory) throws IOException {
-    WALSplitter s = new WALSplitter(factory, conf, walDir, fs, idChecker, cp, mode);
+    WALSplitter s = new WALSplitter(factory, conf, walDir, walFS, idChecker, cp, mode);
     return s.splitLogFile(logfile, reporter);
   }
 
@@ -315,7 +315,7 @@ public class WALSplitter {
         in = getReader(logfile, skipErrors, reporter);
       } catch (CorruptedLogFileException e) {
         LOG.warn("Could not get reader, corrupted log file " + logPath, e);
-        ZKSplitLog.markCorrupted(walDir, logfile.getPath().getName(), fs);
+        ZKSplitLog.markCorrupted(walDir, logfile.getPath().getName(), walFS);
         isCorrupted = true;
       }
       if (in == null) {
@@ -407,7 +407,7 @@ public class WALSplitter {
     } catch (CorruptedLogFileException e) {
       LOG.warn("Could not parse, corrupted log file " + logPath, e);
       csm.getSplitLogWorkerCoordination().markCorrupted(walDir,
-        logfile.getPath().getName(), fs);
+        logfile.getPath().getName(), walFS);
       isCorrupted = true;
     } catch (IOException e) {
       e = RemoteExceptionHandler.checkIOException(e);
@@ -455,31 +455,30 @@ public class WALSplitter {
    */
   public static void finishSplitLogFile(String logfile,
       Configuration conf)  throws IOException {
-    Path rootdir = FSUtils.getWALRootDir(conf);
-    Path oldLogDir = new Path(rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
+    Path walDir = FSUtils.getWALRootDir(conf);
+    Path oldLogDir = new Path(walDir, HConstants.HREGION_OLDLOGDIR_NAME);
     Path logPath;
-    if (FSUtils.isStartingWithPath(rootdir, logfile)) {
+    if (FSUtils.isStartingWithPath(walDir, logfile)) {
       logPath = new Path(logfile);
     } else {
-      logPath = new Path(rootdir, logfile);
+      logPath = new Path(walDir, logfile);
     }
-    finishSplitLogFile(rootdir, oldLogDir, logPath, conf);
+    finishSplitLogFile(walDir, oldLogDir, logPath, conf);
   }
 
-  private static void finishSplitLogFile(Path rootdir, Path oldLogDir,
+  private static void finishSplitLogFile(Path walDir, Path oldLogDir,
       Path logPath, Configuration conf) throws IOException {
     List<Path> processedLogs = new ArrayList<Path>();
     List<Path> corruptedLogs = new ArrayList<Path>();
-    FileSystem fs;
-    fs = rootdir.getFileSystem(conf);
-    if (ZKSplitLog.isCorrupted(rootdir, logPath.getName(), fs)) {
+    FileSystem walFS = walDir.getFileSystem(conf);
+    if (ZKSplitLog.isCorrupted(walDir, logPath.getName(), walFS)) {
       corruptedLogs.add(logPath);
     } else {
       processedLogs.add(logPath);
     }
-    archiveLogs(corruptedLogs, processedLogs, oldLogDir, fs, conf);
-    Path stagingDir = ZKSplitLog.getSplitLogDir(rootdir, logPath.getName());
-    fs.delete(stagingDir, true);
+    archiveLogs(corruptedLogs, processedLogs, oldLogDir, walFS, conf);
+    Path stagingDir = ZKSplitLog.getSplitLogDir(walDir, logPath.getName());
+    walFS.delete(stagingDir, true);
   }
 
   /**
@@ -490,28 +489,28 @@ public class WALSplitter {
    * @param corruptedLogs
    * @param processedLogs
    * @param oldLogDir
-   * @param fs
+   * @param walFS FileSystem to use for WAL archival
    * @param conf
    * @throws IOException
    */
   private static void archiveLogs(
       final List<Path> corruptedLogs,
       final List<Path> processedLogs, final Path oldLogDir,
-      final FileSystem fs, final Configuration conf) throws IOException {
+      final FileSystem walFS, final Configuration conf) throws IOException {
     final Path corruptDir = new Path(FSUtils.getWALRootDir(conf), conf.get(
         "hbase.regionserver.hlog.splitlog.corrupt.dir",  HConstants.CORRUPT_DIR_NAME));
 
-    if (!fs.mkdirs(corruptDir)) {
+    if (!walFS.mkdirs(corruptDir)) {
       LOG.info("Unable to mkdir " + corruptDir);
     }
-    fs.mkdirs(oldLogDir);
+    walFS.mkdirs(oldLogDir);
 
     // this method can get restarted or called multiple times for archiving
     // the same log files.
     for (Path corrupted : corruptedLogs) {
       Path p = new Path(corruptDir, corrupted.getName());
-      if (fs.exists(corrupted)) {
-        if (!fs.rename(corrupted, p)) {
+      if (walFS.exists(corrupted)) {
+        if (!walFS.rename(corrupted, p)) {
           LOG.warn("Unable to move corrupted log " + corrupted + " to " + p);
         } else {
           LOG.warn("Moved corrupted log " + corrupted + " to " + p);
@@ -521,8 +520,8 @@ public class WALSplitter {
 
     for (Path p : processedLogs) {
       Path newPath = FSHLog.getWALArchivePath(oldLogDir, p);
-      if (fs.exists(p)) {
-        if (!FSUtils.renameAndSetModifyTime(fs, p, newPath)) {
+      if (walFS.exists(p)) {
+        if (!FSUtils.renameAndSetModifyTime(walFS, p, newPath)) {
           LOG.warn("Unable to move  " + p + " to " + newPath);
         } else {
           LOG.info("Archived processed log " + p + " to " + newPath);
@@ -548,35 +547,28 @@ public class WALSplitter {
   @VisibleForTesting
   static Path getRegionSplitEditsPath(final Entry logEntry, String fileNameBeingSplit,
       String tmpDirName, Configuration conf) throws IOException {
-    FileSystem fs = FileSystem.get(conf);
-    Path rootDir = FSUtils.getRootDir(conf);
-    Path tableDir = FSUtils.getTableDir(rootDir, logEntry.getKey().getTablename());
+    FileSystem walFS = FSUtils.getWALFileSystem(conf);
+    Path tableDir = FSUtils.getWALTableDir(conf, logEntry.getKey().getTablename());
     String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
-    Path regiondir = HRegion.getRegionDir(tableDir, encodedRegionName);
-    Path dir = getRegionDirRecoveredEditsDir(regiondir);
+    Path regionDir = HRegion.getRegionDir(tableDir, encodedRegionName);
+    Path dir = getRegionDirRecoveredEditsDir(regionDir);
 
-    if (!fs.exists(regiondir)) {
-      LOG.info("This region's directory doesn't exist: "
-          + regiondir.toString() + ". It is very likely that it was" +
-          " already split so it's safe to discard those edits.");
-      return null;
-    }
-    if (fs.exists(dir) && fs.isFile(dir)) {
+    if (walFS.exists(dir) && walFS.isFile(dir)) {
       Path tmp = new Path(tmpDirName);
-      if (!fs.exists(tmp)) {
-        fs.mkdirs(tmp);
+      if (!walFS.exists(tmp)) {
+        walFS.mkdirs(tmp);
       }
       tmp = new Path(tmp,
         HConstants.RECOVERED_EDITS_DIR + "_" + encodedRegionName);
       LOG.warn("Found existing old file: " + dir + ". It could be some "
         + "leftover of an old installation. It should be a folder instead. "
         + "So moving it to " + tmp);
-      if (!fs.rename(dir, tmp)) {
+      if (!walFS.rename(dir, tmp)) {
         LOG.warn("Failed to sideline old file " + dir);
       }
     }
 
-    if (!fs.exists(dir) && !fs.mkdirs(dir)) {
+    if (!walFS.exists(dir) && !walFS.mkdirs(dir)) {
       LOG.warn("mkdir failed on " + dir);
     }
     // Append fileBeingSplit to prevent name conflict since we may have duplicate wal entries now.
@@ -614,31 +606,32 @@ public class WALSplitter {
   private static final String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp";
 
   /**
-   * @param regiondir
+   * @param regionDir
    *          This regions directory in the filesystem.
    * @return The directory that holds recovered edits files for the region
-   *         <code>regiondir</code>
+   *         <code>regionDir</code>
    */
-  public static Path getRegionDirRecoveredEditsDir(final Path regiondir) {
-    return new Path(regiondir, HConstants.RECOVERED_EDITS_DIR);
+  public static Path getRegionDirRecoveredEditsDir(final Path regionDir) {
+    return new Path(regionDir, HConstants.RECOVERED_EDITS_DIR);
   }
 
   /**
    * Returns sorted set of edit files made by splitter, excluding files
    * with '.temp' suffix.
    *
-   * @param fs
-   * @param regiondir
-   * @return Files in passed <code>regiondir</code> as a sorted set.
+   * @param walFS FileSystem to use for reading Recovered edits files
+   * @param regionDir Directory where Recovered edits should reside
+   * @return Files in passed <code>regionDir</code> as a sorted set.
    * @throws IOException
    */
-  public static NavigableSet<Path> getSplitEditFilesSorted(final FileSystem fs,
-      final Path regiondir) throws IOException {
+  public static NavigableSet<Path> getSplitEditFilesSorted(final FileSystem walFS,
+      final Path regionDir) throws IOException {
     NavigableSet<Path> filesSorted = new TreeSet<Path>();
-    Path editsdir = getRegionDirRecoveredEditsDir(regiondir);
-    if (!fs.exists(editsdir))
+    Path editsdir = getRegionDirRecoveredEditsDir(regionDir);
+    if (!walFS.exists(editsdir)) {
       return filesSorted;
-    FileStatus[] files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
+    }
+    FileStatus[] files = FSUtils.listStatus(walFS, editsdir, new PathFilter() {
       @Override
       public boolean accept(Path p) {
         boolean result = false;
@@ -648,7 +641,7 @@ public class WALSplitter {
           // In particular, on error, we'll move aside the bad edit file giving
           // it a timestamp suffix. See moveAsideBadEditsFile.
           Matcher m = EDITFILES_NAME_PATTERN.matcher(p.getName());
-          result = fs.isFile(p) && m.matches();
+          result = walFS.isFile(p) && m.matches();
           // Skip the file whose name ends with RECOVERED_LOG_TMPFILE_SUFFIX,
           // because it means splitwal thread is writting this file.
           if (p.getName().endsWith(RECOVERED_LOG_TMPFILE_SUFFIX)) {
@@ -676,17 +669,17 @@ public class WALSplitter {
   /**
    * Move aside a bad edits file.
    *
-   * @param fs
+   * @param walFS FileSystem to use for WAL operations
    * @param edits
    *          Edits file to move aside.
    * @return The name of the moved aside file.
    * @throws IOException
    */
-  public static Path moveAsideBadEditsFile(final FileSystem fs, final Path edits)
+  public static Path moveAsideBadEditsFile(final FileSystem walFS, final Path edits)
       throws IOException {
     Path moveAsideName = new Path(edits.getParent(), edits.getName() + "."
         + System.currentTimeMillis());
-    if (!fs.rename(edits, moveAsideName)) {
+    if (!walFS.rename(edits, moveAsideName)) {
       LOG.warn("Rename failed from " + edits + " to " + moveAsideName);
     }
     return moveAsideName;
@@ -707,21 +700,21 @@ public class WALSplitter {
 
   /**
    * Create a file with name as region open sequence id
-   * @param fs
-   * @param regiondir
+   * @param walFS FileSystem to write Sequence file to
+   * @param regionDir WALRegionDir used to determine where to write edits files
    * @param newSeqId
    * @param saftyBumper
    * @return long new sequence Id value
    * @throws IOException
    */
-  public static long writeRegionSequenceIdFile(final FileSystem fs, final Path regiondir,
+  public static long writeRegionSequenceIdFile(final FileSystem walFS, final Path regionDir,
       long newSeqId, long saftyBumper) throws IOException {
 
-    Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
+    Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(regionDir);
     long maxSeqId = 0;
     FileStatus[] files = null;
-    if (fs.exists(editsdir)) {
-      files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
+    if (walFS.exists(editsdir)) {
+      files = FSUtils.listStatus(walFS, editsdir, new PathFilter() {
         @Override
         public boolean accept(Path p) {
           return isSequenceIdFile(p);
@@ -749,7 +742,7 @@ public class WALSplitter {
     Path newSeqIdFile = new Path(editsdir, newSeqId + SEQUENCE_ID_FILE_SUFFIX);
     if (newSeqId != maxSeqId) {
       try {
-        if (!fs.createNewFile(newSeqIdFile) && !fs.exists(newSeqIdFile)) {
+        if (!walFS.createNewFile(newSeqIdFile) && !walFS.exists(newSeqIdFile)) {
           throw new IOException("Failed to create SeqId file:" + newSeqIdFile);
         }
         if (LOG.isDebugEnabled()) {
@@ -766,7 +759,7 @@ public class WALSplitter {
         if (newSeqIdFile.equals(status.getPath())) {
           continue;
         }
-        fs.delete(status.getPath(), false);
+        walFS.delete(status.getPath(), false);
       }
     }
     return newSeqId;
@@ -794,7 +787,7 @@ public class WALSplitter {
     }
 
     try {
-      FSUtils.getInstance(fs, conf).recoverFileLease(fs, path, conf, reporter);
+      FSUtils.getInstance(walFS, conf).recoverFileLease(walFS, path, conf, reporter);
       try {
         in = getReader(path, reporter);
       } catch (EOFException e) {
@@ -864,7 +857,7 @@ public class WALSplitter {
    */
   protected Writer createWriter(Path logfile)
       throws IOException {
-    return walFactory.createRecoveredEditsWriter(fs, logfile);
+    return walFactory.createRecoveredEditsWriter(walFS, logfile);
   }
 
   /**
@@ -872,7 +865,7 @@ public class WALSplitter {
    * @return new Reader instance, caller should close
    */
   protected Reader getReader(Path curLogFile, CancelableProgressable reporter) throws IOException {
-    return walFactory.createReader(fs, curLogFile, reporter);
+    return walFactory.createReader(walFS, curLogFile, reporter);
   }
 
   /**
@@ -1355,10 +1348,10 @@ public class WALSplitter {
     }
 
     // delete the one with fewer wal entries
-    void deleteOneWithFewerEntries(FileSystem rootFs, WriterAndPath wap, Path dst)
+    void deleteOneWithFewerEntries(WriterAndPath wap, Path dst)
         throws IOException {
       long dstMinLogSeqNum = -1L;
-      try (WAL.Reader reader = walFactory.createReader(fs, dst)) {
+      try (WAL.Reader reader = walFactory.createReader(walFS, dst)) {
         WAL.Entry entry = reader.next();
         if (entry != null) {
           dstMinLogSeqNum = entry.getKey().getLogSeqNum();
@@ -1373,15 +1366,15 @@ public class WALSplitter {
       if (wap.minLogSeqNum < dstMinLogSeqNum) {
         LOG.warn("Found existing old edits file. It could be the result of a previous failed"
             + " split attempt or we have duplicated wal entries. Deleting " + dst + ", length="
-            + fs.getFileStatus(dst).getLen());
-        if (!fs.delete(dst, false)) {
+            + walFS.getFileStatus(dst).getLen());
+        if (!walFS.delete(dst, false)) {
           LOG.warn("Failed deleting of old " + dst);
           throw new IOException("Failed deleting of old " + dst);
         }
       } else {
         LOG.warn("Found existing old edits file and we have less entries. Deleting " + wap.p
-            + ", length=" + rootFs.getFileStatus(wap.p).getLen());
-        if (!rootFs.delete(wap.p, false)) {
+            + ", length=" + walFS.getFileStatus(wap.p).getLen());
+        if (!walFS.delete(wap.p, false)) {
           LOG.warn("Failed deleting of " + wap.p);
           throw new IOException("Failed deleting of " + wap.p);
         }
@@ -1465,7 +1458,7 @@ public class WALSplitter {
       if (LOG.isTraceEnabled()) {
         LOG.trace("Closing " + wap.p);
       }
-      FileSystem rootFs = FileSystem.get(conf);
+
       try {
         wap.w.close();
       } catch (IOException ioe) {
@@ -1480,7 +1473,7 @@ public class WALSplitter {
       }
       if (wap.editsWritten == 0) {
         // just remove the empty recovered.edits file
-        if (rootFs.exists(wap.p) && !rootFs.delete(wap.p, false)) {
+        if (walFS.exists(wap.p) && !walFS.delete(wap.p, false)) {
           LOG.warn("Failed deleting empty " + wap.p);
           throw new IOException("Failed deleting empty  " + wap.p);
         }
@@ -1490,14 +1483,14 @@ public class WALSplitter {
       Path dst = getCompletedRecoveredEditsFilePath(wap.p,
           regionMaximumEditLogSeqNum.get(encodedRegionName));
       try {
-        if (!dst.equals(wap.p) && rootFs.exists(dst)) {
-          deleteOneWithFewerEntries(rootFs, wap, dst);
+        if (!dst.equals(wap.p) && walFS.exists(dst)) {
+          deleteOneWithFewerEntries(wap, dst);
         }
         // Skip the unit tests which create a splitter that reads and
         // writes the data without touching disk.
         // TestHLogSplit#testThreading is an example.
-        if (rootFs.exists(wap.p)) {
-          if (!rootFs.rename(wap.p, dst)) {
+        if (walFS.exists(wap.p)) {
+          if (!walFS.rename(wap.p, dst)) {
             throw new IOException("Failed renaming " + wap.p + " to " + dst);
           }
           LOG.info("Rename " + wap.p + " to " + dst);
@@ -1594,12 +1587,11 @@ public class WALSplitter {
       if (regionedits == null) {
         return null;
       }
-      FileSystem rootFs = FileSystem.get(conf);
-      if (rootFs.exists(regionedits)) {
+      if (walFS.exists(regionedits)) {
         LOG.warn("Found old edits file. It could be the "
             + "result of a previous failed split attempt. Deleting " + regionedits + ", length="
-            + rootFs.getFileStatus(regionedits).getLen());
-        if (!rootFs.delete(regionedits, false)) {
+            + walFS.getFileStatus(regionedits).getLen());
+        if (!walFS.delete(regionedits, false)) {
           LOG.warn("Failed delete of old " + regionedits);
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9675ad38/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 7ee009c..5644c3f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -692,7 +692,7 @@ public class TestHRegion {
       for (Store store : region.getStores()) {
         maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId - 1);
       }
-      long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
+      long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, status);
       assertEquals(maxSeqId, seqId);
       region.getMVCC().advanceTo(seqId);
       Get get = new Get(row);
@@ -746,7 +746,7 @@ public class TestHRegion {
       for (Store store : region.getStores()) {
         maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1);
       }
-      long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
+      long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, status);
       assertEquals(maxSeqId, seqId);
       region.getMVCC().advanceTo(seqId);
       Get get = new Get(row);
@@ -790,7 +790,7 @@ public class TestHRegion {
     for (Store store : region.getStores()) {
       maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId);
     }
-    long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, null);
+    long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, null);
     assertEquals(minSeqId, seqId);
   }
 
@@ -846,7 +846,7 @@ public class TestHRegion {
       for (Store store : region.getStores()) {
         maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1);
       }
-      long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
+      long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, status);
       assertEquals(maxSeqId, seqId);
 
       // assert that the files are flushed

http://git-wip-us.apache.org/repos/asf/hbase/blob/9675ad38/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
index bf72301..c3f21f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
@@ -112,7 +112,7 @@ public class TestRecoveredEdits {
     // There should be no store files.
     assertTrue(storeFiles.isEmpty());
     region.close();
-    Path regionDir = region.getRegionDir(hbaseRootDir, hri);
+    Path regionDir = region.getWALRegionDir();
     Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regionDir);
     // This is a little fragile getting this path to a file of 10M of edits.
     Path recoveredEditsFile = new Path(

http://git-wip-us.apache.org/repos/asf/hbase/blob/9675ad38/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
index a196f57..5240f2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
@@ -307,7 +307,7 @@ public class TestWALReplay {
 
     MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
+    Path basedir = FSUtils.getWALTableDir(conf, tableName);
     deleteDir(basedir);
 
     HTableDescriptor htd = createBasic3FamilyHTD(tableName);
@@ -934,7 +934,7 @@ public class TestWALReplay {
     WALSplitter.splitLogFile(hbaseWALRootDir, listStatus[0],
         this.fs, this.conf, null, null, null, mode, wals);
     FileStatus[] listStatus1 = this.fs.listStatus(
-      new Path(FSUtils.getTableDir(hbaseRootDir, tableName), new Path(hri.getEncodedName(),
+      new Path(FSUtils.getWALTableDir(this.conf, tableName), new Path(hri.getEncodedName(),
           "recovered.edits")), new PathFilter() {
         @Override
         public boolean accept(Path p) {
@@ -968,7 +968,7 @@ public class TestWALReplay {
     final int countPerFamily = 10;
     final HTableDescriptor htd = createBasic1FamilyHTD(tableName);
     HRegion region1 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.hbaseWALRootDir, this.conf, htd);
-    Path regionDir = region1.getRegionFileSystem().getRegionDir();
+    Path regionDir = region1.getWALRegionDir();
     HBaseTestingUtility.closeRegionAndWAL(region1);
 
     WAL wal = createWAL(this.conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/9675ad38/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
index 3c0bade..fb2db2b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
@@ -172,7 +172,7 @@ public class TestWALFactory {
     Path oldLogDir = new Path(hbaseWALDir, HConstants.HREGION_OLDLOGDIR_NAME);
     final int howmany = 3;
     HRegionInfo[] infos = new HRegionInfo[3];
-    Path tabledir = FSUtils.getTableDir(hbaseDir, tableName);
+    Path tabledir = FSUtils.getWALTableDir(conf, tableName);
     fs.mkdirs(tabledir);
     for(int i = 0; i < howmany; i++) {
       infos[i] = new HRegionInfo(tableName,

http://git-wip-us.apache.org/repos/asf/hbase/blob/9675ad38/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
index 9b9c23d..49f8534 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -247,9 +247,9 @@ public class TestWALSplit {
           }
           LOG.debug(ls);
           LOG.info("Splitting WALs out from under zombie. Expecting " + numWriters + " files.");
-          WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf2, wals);
+          WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf2, wals);
           LOG.info("Finished splitting out from under zombie.");
-          Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region);
+          Path[] logfiles = getLogForRegion(TABLE_NAME, region);
           assertEquals("wrong number of split files for region", numWriters, logfiles.length);
           int count = 0;
           for (Path logfile: logfiles) {
@@ -435,9 +435,9 @@ public class TestWALSplit {
 
     generateWALs(1, 10, -1, 0);
     useDifferentDFSClient();
-    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
     Path originalLog = (fs.listStatus(OLDLOGDIR))[0].getPath();
-    Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION);
+    Path[] splitLog = getLogForRegion(TABLE_NAME, REGION);
     assertEquals(1, splitLog.length);
 
     assertTrue("edits differ after split", logsAreEqual(originalLog, splitLog[0]));
@@ -451,9 +451,9 @@ public class TestWALSplit {
 
     generateWALs(1, 10, -1, 100);
     useDifferentDFSClient();
-    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
     Path originalLog = (fs.listStatus(OLDLOGDIR))[0].getPath();
-    Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION);
+    Path[] splitLog = getLogForRegion(TABLE_NAME, REGION);
     assertEquals(1, splitLog.length);
 
     assertFalse("edits differ after split", logsAreEqual(originalLog, splitLog[0]));
@@ -478,13 +478,13 @@ public class TestWALSplit {
     writer.close();
 
     useDifferentDFSClient();
-    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
 
     Path originalLog = (fs.listStatus(OLDLOGDIR))[0].getPath();
     // original log should have 10 test edits, 10 region markers, 1 compaction marker
     assertEquals(21, countWAL(originalLog));
 
-    Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, hri.getEncodedName());
+    Path[] splitLog = getLogForRegion(TABLE_NAME, hri.getEncodedName());
     assertEquals(1, splitLog.length);
 
     assertFalse("edits differ after split", logsAreEqual(originalLog, splitLog[0]));
@@ -499,10 +499,10 @@ public class TestWALSplit {
   private int splitAndCount(final int expectedFiles, final int expectedEntries)
       throws IOException {
     useDifferentDFSClient();
-    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
     int result = 0;
     for (String region : REGIONS) {
-      Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region);
+      Path[] logfiles = getLogForRegion(TABLE_NAME, region);
       assertEquals(expectedFiles, logfiles.length);
       int count = 0;
       for (Path logfile: logfiles) {
@@ -633,7 +633,7 @@ public class TestWALSplit {
         walDirContents.add(status.getPath().getName());
       }
       useDifferentDFSClient();
-      WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+      WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
       return walDirContents;
     } finally {
       conf.setClass("hbase.regionserver.hlog.reader.impl", backupClass,
@@ -674,9 +674,9 @@ public class TestWALSplit {
     corruptWAL(c1, corruption, true);
 
     useDifferentDFSClient();
-    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
 
-    Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION);
+    Path[] splitLog = getLogForRegion(TABLE_NAME, REGION);
     assertEquals(1, splitLog.length);
 
     int actualCount = 0;
@@ -710,7 +710,7 @@ public class TestWALSplit {
     conf.setBoolean(HBASE_SKIP_ERRORS, false);
     generateWALs(-1);
     useDifferentDFSClient();
-    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
     FileStatus[] archivedLogs = fs.listStatus(OLDLOGDIR);
     assertEquals("wrong number of files in the archive log", NUM_WRITERS, archivedLogs.length);
   }
@@ -726,7 +726,7 @@ public class TestWALSplit {
       throws IOException {
     generateWALs(-1);
     useDifferentDFSClient();
-    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
     FileStatus [] statuses = null;
     try {
       statuses = fs.listStatus(WALDIR);
@@ -756,7 +756,7 @@ public class TestWALSplit {
 
     try {
       InstrumentedLogWriter.activateFailure = true;
-      WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+      WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
     } catch (IOException e) {
       assertTrue(e.getMessage().
           contains("This exception is instrumented and should only be thrown for testing"));
@@ -777,7 +777,7 @@ public class TestWALSplit {
 
     Path regiondir = new Path(TABLEDIR, region);
     fs.delete(regiondir, true);
-    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
     assertFalse(fs.exists(regiondir));
   }
 
@@ -854,7 +854,7 @@ public class TestWALSplit {
     useDifferentDFSClient();
 
     try {
-      WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, spiedFs, conf, wals);
+      WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, spiedFs, conf, wals);
       assertEquals(NUM_WRITERS, fs.listStatus(OLDLOGDIR).length);
       assertFalse(fs.exists(WALDIR));
     } catch (IOException e) {
@@ -1077,7 +1077,7 @@ public class TestWALSplit {
     Path regiondir = new Path(TABLEDIR, REGION);
     LOG.info("Region directory is" + regiondir);
     fs.delete(regiondir, true);
-    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
     assertFalse(fs.exists(regiondir));
   }
 
@@ -1090,7 +1090,7 @@ public class TestWALSplit {
     injectEmptyFile(".empty", true);
     useDifferentDFSClient();
 
-    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
     Path tdir = FSUtils.getTableDir(HBASEDIR, TABLE_NAME);
     assertFalse(fs.exists(tdir));
 
@@ -1115,7 +1115,7 @@ public class TestWALSplit {
         Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true);
 
     useDifferentDFSClient();
-    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    WALSplitter.split(HBASELOGDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
 
     final Path corruptDir = new Path(FSUtils.getWALRootDir(conf), HConstants.CORRUPT_DIR_NAME);
     assertEquals(1, fs.listStatus(corruptDir).length);
@@ -1145,14 +1145,14 @@ public class TestWALSplit {
       @Override
       protected Writer createWriter(Path logfile)
           throws IOException {
-        Writer writer = wals.createRecoveredEditsWriter(this.fs, logfile);
+        Writer writer = wals.createRecoveredEditsWriter(this.walFS, logfile);
         // After creating writer, simulate region's
         // replayRecoveredEditsIfAny() which gets SplitEditFiles of this
         // region and delete them, excluding files with '.temp' suffix.
         NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regiondir);
         if (files != null && !files.isEmpty()) {
           for (Path file : files) {
-            if (!this.fs.delete(file, false)) {
+            if (!this.walFS.delete(file, false)) {
               LOG.error("Failed delete of " + file);
             } else {
               LOG.debug("Deleted recovered.edits file=" + file);
@@ -1231,9 +1231,9 @@ public class TestWALSplit {
 
 
 
-  private Path[] getLogForRegion(Path rootdir, TableName table, String region)
+  private Path[] getLogForRegion(TableName table, String region)
       throws IOException {
-    Path tdir = FSUtils.getTableDir(rootdir, table);
+    Path tdir = FSUtils.getWALTableDir(conf, table);
     @SuppressWarnings("deprecation")
     Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
         Bytes.toString(region.getBytes())));


[16/50] [abbrv] hbase git commit: HBASE-15134 Add visibility into Flush and Compaction queues

Posted by ap...@apache.org.
HBASE-15134 Add visibility into Flush and Compaction queues

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java


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

Branch: refs/heads/branch-1.3
Commit: c35fa2a3e115a41b1fcb7afc533dddeefa74163d
Parents: 3e0d693
Author: Abhishek Singh Chouhan <ac...@apache.org>
Authored: Fri Jul 28 13:17:32 2017 +0530
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:17 2018 -0800

----------------------------------------------------------------------
 .../hbase/regionserver/MetricsRegionSource.java |  8 +++++
 .../regionserver/MetricsRegionWrapper.java      | 24 +++++++++++++++
 .../regionserver/MetricsRegionSourceImpl.java   | 19 +++++++++++-
 .../TestMetricsRegionSourceImpl.java            | 20 ++++++++++++
 .../hbase/regionserver/CompactSplitThread.java  |  9 +++++-
 .../hadoop/hbase/regionserver/HRegion.java      | 20 +++++++++++-
 .../hbase/regionserver/MemStoreFlusher.java     |  2 ++
 .../regionserver/MetricsRegionWrapperImpl.java  | 32 ++++++++++++++++++++
 .../regionserver/MetricsRegionWrapperStub.java  | 20 ++++++++++++
 .../hbase/regionserver/TestMetricsRegion.java   | 12 ++++++++
 10 files changed, 163 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c35fa2a3/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
index decf841..d5738cf 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
@@ -30,11 +30,19 @@ public interface MetricsRegionSource extends Comparable<MetricsRegionSource> {
   String COMPACTIONS_COMPLETED_COUNT = "compactionsCompletedCount";
   String COMPACTIONS_FAILED_COUNT = "compactionsFailedCount";
   String LAST_MAJOR_COMPACTION_AGE = "lastMajorCompactionAge";
+  String COMPACTIONS_QUEUED_COUNT = "compactionsQueuedCount";
+  String MAX_COMPACTION_QUEUE_SIZE = "maxCompactionQueueSize";
   String NUM_BYTES_COMPACTED_COUNT = "numBytesCompactedCount";
   String NUM_FILES_COMPACTED_COUNT = "numFilesCompactedCount";
+  String FLUSHES_QUEUED_COUNT = "flushesQueuedCount";
+  String MAX_FLUSH_QUEUE_SIZE = "maxFlushQueueSize";
   String COMPACTIONS_COMPLETED_DESC = "Number of compactions that have completed.";
   String COMPACTIONS_FAILED_DESC = "Number of compactions that have failed.";
   String LAST_MAJOR_COMPACTION_DESC = "Age of the last major compaction in milliseconds.";
+  String COMPACTIONS_QUEUED_DESC = "Number of compactions that are queued/running for this region";
+  String MAX_COMPACTION_QUEUE_DESC = "Max number of compactions queued for this region";
+  String FLUSHES_QUEUED_DESC = "Number flushes requested/queued for this region";
+  String MAX_FLUSH_QUEUE_DESC = "Max number of flushes queued for this region";
   String  NUM_BYTES_COMPACTED_DESC =
       "Sum of filesize on all files entering a finished, successful or aborted, compaction";
   String NUM_FILES_COMPACTED_DESC =

http://git-wip-us.apache.org/repos/asf/hbase/blob/c35fa2a3/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
index 9b7acd3..9a725cd 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
@@ -112,6 +112,30 @@ public interface MetricsRegionWrapper {
    */
   long getNumCompactionsFailed();
 
+  /**
+   * @return the total number of compactions that are currently queued(or being executed) at point in
+   *  time
+   */
+  long getNumCompactionsQueued();
+
+  /**
+   * @return the total number of flushes currently queued(being executed) for this region at point in
+   *  time
+   */
+  long getNumFlushesQueued();
+
+  /**
+   * @return the max number of compactions queued for this region
+   * Note that this metric is updated periodically and hence might miss some data points
+   */
+  long getMaxCompactionQueueSize();
+
+  /**
+   * @return the max number of flushes queued for this region
+   * Note that this metric is updated periodically and hence might miss some data points
+   */
+  long getMaxFlushQueueSize();
+
   int getRegionHashCode();
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/c35fa2a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
index 924da8b..c50ffc9 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
@@ -265,9 +265,26 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
               regionNamePrefix + MetricsRegionServerSource.WRITE_REQUEST_COUNT,
               MetricsRegionServerSource.WRITE_REQUEST_COUNT_DESC),
           this.regionWrapper.getWriteRequestCount());
-      mrb.addCounter(Interns.info(regionNamePrefix + MetricsRegionSource.REPLICA_ID,
+      mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionSource.REPLICA_ID,
               MetricsRegionSource.REPLICA_ID_DESC),
           this.regionWrapper.getReplicaId());
+      mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionSource.COMPACTIONS_QUEUED_COUNT,
+              MetricsRegionSource.COMPACTIONS_QUEUED_DESC),
+          this.regionWrapper.getNumCompactionsQueued());
+      mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionSource.FLUSHES_QUEUED_COUNT,
+              MetricsRegionSource.FLUSHES_QUEUED_DESC),
+          this.regionWrapper.getNumFlushesQueued());
+      mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionSource.MAX_COMPACTION_QUEUE_SIZE,
+              MetricsRegionSource.MAX_COMPACTION_QUEUE_DESC),
+          this.regionWrapper.getMaxCompactionQueueSize());
+      mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionSource.MAX_FLUSH_QUEUE_SIZE,
+              MetricsRegionSource.MAX_FLUSH_QUEUE_DESC),
+          this.regionWrapper.getMaxFlushQueueSize());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c35fa2a3/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
index a0b7612..b39c467 100644
--- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
@@ -171,5 +171,25 @@ public class TestMetricsRegionSourceImpl {
     public int getReplicaId() {
       return 0;
     }
+
+    @Override
+    public long getNumCompactionsQueued() {
+      return 0;
+    }
+
+    @Override
+    public long getNumFlushesQueued() {
+      return 0;
+    }
+
+    @Override
+    public long getMaxCompactionQueueSize() {
+      return 0;
+    }
+
+    @Override
+    public long getMaxFlushQueueSize() {
+      return 0;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c35fa2a3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
index 71a8803..e6fe9cd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
@@ -356,6 +356,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
     ThreadPoolExecutor pool = (selectNow && s.throttleCompaction(compaction.getRequest().getSize()))
       ? longCompactions : shortCompactions;
     pool.execute(new CompactionRunner(s, r, compaction, pool, user));
+    ((HRegion)r).incrementCompactionsQueuedCount();
     if (LOG.isDebugEnabled()) {
       String type = (pool == shortCompactions) ? "Small " : "Large ";
       LOG.debug(type + "Compaction requested: " + (selectNow ? compaction.toString() : "system")
@@ -498,9 +499,13 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
         } catch (IOException ex) {
           LOG.error("Compaction selection failed " + this, ex);
           server.checkFileSystem();
+          region.decrementCompactionsQueuedCount();
           return;
         }
-        if (this.compaction == null) return; // nothing to do
+        if (this.compaction == null) {
+          region.decrementCompactionsQueuedCount();
+          return; // nothing to do
+        }
         // Now see if we are in correct pool for the size; if not, go to the correct one.
         // We might end up waiting for a while, so cancel the selection.
         assert this.compaction.hasSelection();
@@ -552,6 +557,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
         region.reportCompactionRequestFailure();
         server.checkFileSystem();
       } finally {
+        region.decrementCompactionsQueuedCount();
         LOG.debug("CompactSplitThread Status: " + CompactSplitThread.this);
       }
       this.compaction.getRequest().afterExecute();
@@ -562,6 +568,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
       Preconditions.checkNotNull(server);
       if (server.isStopped()
           || (region.getTableDesc() != null && !region.getTableDesc().isCompactionEnabled())) {
+        region.decrementCompactionsQueuedCount();
         return;
       }
       doCompaction(user);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c35fa2a3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
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 d9dbb72..74148ea 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
@@ -297,6 +297,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   final AtomicLong compactionsFailed = new AtomicLong(0L);
   final AtomicLong compactionNumFilesCompacted = new AtomicLong(0L);
   final AtomicLong compactionNumBytesCompacted = new AtomicLong(0L);
+  final AtomicLong compactionsQueued = new AtomicLong(0L);
+  final AtomicLong flushesQueued = new AtomicLong(0L);
 
   private final WAL wal;
   private final HRegionFileSystem fs;
@@ -2088,6 +2090,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           coprocessorHost.postFlush();
         }
 
+        if(fs.isFlushSucceeded()) {
+          flushesQueued.set(0L);
+        }
+
         status.markComplete("Flush successful");
         return fs;
       } finally {
@@ -8141,7 +8147,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT +
       ClassSize.ARRAY +
-      46 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
+      48 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
       (14 * Bytes.SIZEOF_LONG) +
       5 * Bytes.SIZEOF_BOOLEAN);
 
@@ -8723,6 +8729,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     compactionsFailed.incrementAndGet();
   }
 
+  public void incrementCompactionsQueuedCount() {
+    compactionsQueued.incrementAndGet();
+  }
+
+  public void decrementCompactionsQueuedCount() {
+    compactionsQueued.decrementAndGet();
+  }
+
+  public void incrementFlushesQueuedCount() {
+    flushesQueued.incrementAndGet();
+  }
+
   /**
    * Do not change this sequence id.
    * @return sequenceId

http://git-wip-us.apache.org/repos/asf/hbase/blob/c35fa2a3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
index b152958..b4adea6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
@@ -361,6 +361,7 @@ class MemStoreFlusher implements FlushRequester {
 
   @Override
   public void requestFlush(Region r, boolean forceFlushAllStores) {
+    ((HRegion)r).incrementFlushesQueuedCount();
     synchronized (regionsInQueue) {
       if (!regionsInQueue.containsKey(r)) {
         // This entry has no delay so it will be added at the top of the flush
@@ -374,6 +375,7 @@ class MemStoreFlusher implements FlushRequester {
 
   @Override
   public void requestDelayedFlush(Region r, long delay, boolean forceFlushAllStores) {
+    ((HRegion)r).incrementFlushesQueuedCount();
     synchronized (regionsInQueue) {
       if (!regionsInQueue.containsKey(r)) {
         // This entry has some delay

http://git-wip-us.apache.org/repos/asf/hbase/blob/c35fa2a3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
index 493ee18..60085f6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
@@ -52,6 +52,8 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
   private long minStoreFileAge;
   private long avgStoreFileAge;
   private long numReferenceFiles;
+  private long maxFlushQueueSize;
+  private long maxCompactionQueueSize;
 
   private ScheduledFuture<?> regionMetricsUpdateTask;
 
@@ -158,6 +160,26 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
   }
 
   @Override
+  public long getNumCompactionsQueued() {
+    return this.region.compactionsQueued.get();
+  }
+
+  @Override
+  public long getNumFlushesQueued() {
+    return this.region.flushesQueued.get();
+  }
+
+  @Override
+  public long getMaxCompactionQueueSize() {
+    return maxCompactionQueueSize;
+  }
+
+  @Override
+  public long getMaxFlushQueueSize() {
+    return maxFlushQueueSize;
+  }
+
+  @Override
   public long getMaxStoreFileAge() {
     return maxStoreFileAge;
   }
@@ -192,6 +214,8 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
       long tempMaxStoreFileAge = 0;
       long tempMinStoreFileAge = Long.MAX_VALUE;
       long tempNumReferenceFiles = 0;
+      long tempMaxCompactionQueueSize = 0;
+      long tempMaxFlushQueueSize = 0;
 
       long avgAgeNumerator = 0;
       long numHFiles = 0;
@@ -229,6 +253,14 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
       }
 
       numReferenceFiles = tempNumReferenceFiles;
+      tempMaxCompactionQueueSize = getNumCompactionsQueued();
+      tempMaxFlushQueueSize = getNumFlushesQueued();
+      if (tempMaxCompactionQueueSize > maxCompactionQueueSize) {
+        maxCompactionQueueSize = tempMaxCompactionQueueSize;
+      }
+      if (tempMaxFlushQueueSize > maxFlushQueueSize) {
+        maxFlushQueueSize = tempMaxFlushQueueSize;
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c35fa2a3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
index 8ae1180..4f18144 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
@@ -137,4 +137,24 @@ public class MetricsRegionWrapperStub implements MetricsRegionWrapper {
   public int getReplicaId() {
     return replicaid;
   }
+
+  @Override
+  public long getNumCompactionsQueued() {
+    return 4;
+  }
+
+  @Override
+  public long getNumFlushesQueued() {
+    return 6;
+  }
+
+  @Override
+  public long getMaxCompactionQueueSize() {
+    return 4;
+  }
+
+  @Override
+  public long getMaxFlushQueueSize() {
+    return 6;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c35fa2a3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
index febcd28..2009da9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
@@ -76,6 +76,18 @@ public class TestMetricsRegion {
     HELPER.assertCounter(
       "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_replicaid", 
       1, agg);
+    HELPER.assertCounter(
+      "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_compactionsQueuedCount",
+      4, agg);
+    HELPER.assertCounter(
+      "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_flushesQueuedCount",
+      6, agg);
+    HELPER.assertCounter(
+      "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_maxCompactionQueueSize",
+      4, agg);
+    HELPER.assertCounter(
+      "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_maxFlushQueueSize",
+      6, agg);
     mr.close();
   }
 }


[32/50] [abbrv] hbase git commit: HBASE-20808 Wrong shutdown order between Chores and ChoreService

Posted by ap...@apache.org.
HBASE-20808 Wrong shutdown order between Chores and ChoreService

Signed-off-by: Reid Chan <re...@apache.org>

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java


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

Branch: refs/heads/branch-1.3
Commit: 8e36761e89d0ecdd6a476f06293c5d1c8295eb06
Parents: 955264e
Author: Nihal Jain <ni...@gmail.com>
Authored: Fri Jul 6 00:42:58 2018 +0530
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:19 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java | 34 ++++++++------------
 .../hbase/regionserver/HRegionServer.java       | 21 ++++++------
 2 files changed, 23 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8e36761e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index b47fecb..aedb987 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -50,6 +50,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
@@ -1226,8 +1227,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
         LOG.error("Failed to stop master jetty server", e);
       }
     }
-    super.stopServiceThreads();
     stopChores();
+    super.stopServiceThreads();
     CleanerChore.shutDownChorePool();
 
     // Wait for all the remaining region servers to report in IFF we were
@@ -1240,9 +1241,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       LOG.debug("Stopping service threads");
     }
     // Clean up and close up shop
-    if (this.logCleaner != null) this.logCleaner.cancel(true);
-    if (this.hfileCleaner != null) this.hfileCleaner.cancel(true);
-    if (this.replicationZKLockCleanerChore != null) this.replicationZKLockCleanerChore.cancel(true);
     if (this.quotaManager != null) this.quotaManager.stop();
     if (this.activeMasterManager != null) this.activeMasterManager.stop();
     if (this.serverManager != null) this.serverManager.stop();
@@ -1283,23 +1281,17 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   }
 
   private void stopChores() {
-    if (this.balancerChore != null) {
-      this.balancerChore.cancel(true);
-    }
-    if (this.normalizerChore != null) {
-      this.normalizerChore.cancel(true);
-    }
-    if (this.clusterStatusChore != null) {
-      this.clusterStatusChore.cancel(true);
-    }
-    if (this.catalogJanitorChore != null) {
-      this.catalogJanitorChore.cancel(true);
-    }
-    if (this.clusterStatusPublisherChore != null){
-      clusterStatusPublisherChore.cancel(true);
-    }
-    if (this.periodicDoMetricsChore != null) {
-      periodicDoMetricsChore.cancel();
+    ChoreService choreService = getChoreService();
+    if (choreService != null) {
+      choreService.cancelChore(this.balancerChore);
+      choreService.cancelChore(this.normalizerChore);
+      choreService.cancelChore(this.clusterStatusChore);
+      choreService.cancelChore(this.catalogJanitorChore);
+      choreService.cancelChore(this.clusterStatusPublisherChore);
+      choreService.cancelChore(this.periodicDoMetricsChore);
+      choreService.cancelChore(this.logCleaner);
+      choreService.cancelChore(this.hfileCleaner);
+      choreService.cancelChore(this.replicationZKLockCleanerChore);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8e36761e/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 ec21599..545c926 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
@@ -1044,10 +1044,6 @@ public class HRegionServer extends HasThread implements
     if (this.hMemManager != null) this.hMemManager.stop();
     if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
     if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
-    if (this.compactionChecker != null) this.compactionChecker.cancel(true);
-    if (this.healthCheckChore != null) this.healthCheckChore.cancel(true);
-    if (this.nonceManagerChore != null) this.nonceManagerChore.cancel(true);
-    if (this.storefileRefresher != null) this.storefileRefresher.cancel(true);
     sendShutdownInterrupt();
 
     // Stop the quota manager
@@ -2210,13 +2206,16 @@ public class HRegionServer extends HasThread implements
    */
   protected void stopServiceThreads() {
     // clean up the scheduled chores
-    if (this.choreService != null) choreService.shutdown();
-    if (this.nonceManagerChore != null) nonceManagerChore.cancel(true);
-    if (this.compactionChecker != null) compactionChecker.cancel(true);
-    if (this.periodicFlusher != null) periodicFlusher.cancel(true);
-    if (this.healthCheckChore != null) healthCheckChore.cancel(true);
-    if (this.storefileRefresher != null) storefileRefresher.cancel(true);
-    if (this.movedRegionsCleaner != null) movedRegionsCleaner.cancel(true);
+    if (this.choreService != null) {
+      choreService.cancelChore(nonceManagerChore);
+      choreService.cancelChore(compactionChecker);
+      choreService.cancelChore(periodicFlusher);
+      choreService.cancelChore(healthCheckChore);
+      choreService.cancelChore(storefileRefresher);
+      choreService.cancelChore(movedRegionsCleaner);
+      // clean up the remaining scheduled chores (in case we missed out any)
+      choreService.shutdown();
+    }
 
     if (this.cacheFlusher != null) {
       this.cacheFlusher.join();


[19/50] [abbrv] hbase git commit: HBASE-19391 Calling HRegion#initializeRegionInternals from a region replica can still re-create a region directory

Posted by ap...@apache.org.
HBASE-19391 Calling HRegion#initializeRegionInternals from a region replica can still re-create a region directory

HBASE-19391 Calling HRegion#initializeRegionInternals from a region replica can still re-create a region directory; ADDEDNDUM to fix TestRegionOpen failure

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 453935f31d3cd56e0db54fce34fdd3078d0b7981
Parents: 8a4b2b5
Author: Esteban Gutierrez <es...@apache.org>
Authored: Wed Feb 21 15:28:50 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:18 2018 -0800

----------------------------------------------------------------------
 .../hbase/regionserver/HRegionFileSystem.java   | 29 ++++++++++----------
 .../hbase/regionserver/TestRegionOpen.java      |  4 ++-
 2 files changed, 18 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/453935f3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 619358c..7672204 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -889,28 +889,29 @@ public class HRegionFileSystem {
   public static HRegionFileSystem createRegionOnFileSystem(final Configuration conf,
       final FileSystem fs, final Path tableDir, final HRegionInfo regionInfo) throws IOException {
     HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, regionInfo);
-    Path regionDir = regionFs.getRegionDir();
 
-    if (fs.exists(regionDir)) {
-      LOG.warn("Trying to create a region that already exists on disk: " + regionDir);
-      throw new IOException("The specified region already exists on disk: " + regionDir);
-    }
+    // We only create a .regioninfo and the region directory if this is the default region replica
+    if (regionInfo.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+      Path regionDir = regionFs.getRegionDir();
+      if (fs.exists(regionDir)) {
+        LOG.warn("Trying to create a region that already exists on disk: " + regionDir);
+        throw new IOException("The specified region already exists on disk: " + regionDir);
+      }
 
-    // Create the region directory
-    if (!createDirOnFileSystem(fs, conf, regionDir)) {
-      LOG.warn("Unable to create the region directory: " + regionDir);
-      throw new IOException("Unable to create region directory: " + regionDir);
-    }
+      // Create the region directory
+      if (!createDirOnFileSystem(fs, conf, regionDir)) {
+        LOG.warn("Unable to create the region directory: " + regionDir);
+        throw new IOException("Unable to create region directory: " + regionDir);
+      }
 
-    // Write HRI to a file in case we need to recover hbase:meta
-    // Only primary replicas should write region info
-    if (regionInfo.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
+      // Write HRI to a file in case we need to recover hbase:meta
       regionFs.writeRegionInfoOnFilesystem(false);
+      return regionFs;
     } else {
       if (LOG.isDebugEnabled())
         LOG.debug("Skipping creation of .regioninfo file for " + regionInfo);
     }
-    return regionFs;
+    return null;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/453935f3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
index edb1d52..62aabce 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -97,7 +98,8 @@ public class TestRegionOpen {
     assertEquals(2, exec.getCompletedTaskCount());
   }
 
-  @Test(timeout = 60000)
+  @Ignore // Needs rewrite since HBASE-19391 which returns null out of createRegionOnFileSystem
+  @Test
   public void testNonExistentRegionReplica() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final byte[] FAMILYNAME = Bytes.toBytes("fam");


[41/50] [abbrv] hbase git commit: HBASE-20469 Directory used for sidelining old recovered edits files should be made configurable

Posted by ap...@apache.org.
HBASE-20469 Directory used for sidelining old recovered edits files should be made configurable

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 1d362f6de3e5c21c9f124656a928a555e4853a2b
Parents: 2702f23
Author: Nihal Jain <ni...@gmail.com>
Authored: Sat Apr 21 16:40:21 2018 +0530
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 19:18:54 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/wal/WALSplitter.java     | 10 ++++++----
 .../java/org/apache/hadoop/hbase/wal/TestWALSplit.java    |  7 +++++--
 2 files changed, 11 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1d362f6d/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index c88e6d3..45dbb11 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -539,6 +539,7 @@ public class WALSplitter {
    * creating it if necessary.
    * @param logEntry
    * @param fileNameBeingSplit the file being split currently. Used to generate tmp file name.
+   * @param tmpDirName of the directory used to sideline old recovered edits file
    * @param conf
    * @return Path to file into which to dump split log edits.
    * @throws IOException
@@ -546,8 +547,7 @@ public class WALSplitter {
   @SuppressWarnings("deprecation")
   @VisibleForTesting
   static Path getRegionSplitEditsPath(final Entry logEntry, String fileNameBeingSplit,
-      Configuration conf)
-  throws IOException {
+      String tmpDirName, Configuration conf) throws IOException {
     FileSystem fs = FileSystem.get(conf);
     Path rootDir = FSUtils.getRootDir(conf);
     Path tableDir = FSUtils.getTableDir(rootDir, logEntry.getKey().getTablename());
@@ -562,7 +562,7 @@ public class WALSplitter {
       return null;
     }
     if (fs.exists(dir) && fs.isFile(dir)) {
-      Path tmp = new Path("/tmp");
+      Path tmp = new Path(tmpDirName);
       if (!fs.exists(tmp)) {
         fs.mkdirs(tmp);
       }
@@ -1587,8 +1587,10 @@ public class WALSplitter {
      * @return a path with a write for that path. caller should close.
      */
     WriterAndPath createWAP(byte[] region, Entry entry) throws IOException {
+      String tmpDirName = conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY,
+        HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY);
       Path regionedits = getRegionSplitEditsPath(entry,
-          fileBeingSplit.getPath().getName(), conf);
+          fileBeingSplit.getPath().getName(), tmpDirName, conf);
       if (regionedits == null) {
         return null;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1d362f6d/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
index 0d0bbec..9b9c23d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -123,6 +123,7 @@ public class TestWALSplit {
   private Path OLDLOGDIR;
   private Path CORRUPTDIR;
   private Path TABLEDIR;
+  private String TMPDIRNAME;
 
   private static final int NUM_WRITERS = 10;
   private static final int ENTRIES = 10; // entries per writer per region
@@ -185,6 +186,8 @@ public class TestWALSplit {
     OLDLOGDIR = new Path(HBASELOGDIR, HConstants.HREGION_OLDLOGDIR_NAME);
     CORRUPTDIR = new Path(HBASELOGDIR, HConstants.CORRUPT_DIR_NAME);
     TABLEDIR = FSUtils.getTableDir(HBASEDIR, TABLE_NAME);
+    TMPDIRNAME = conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY,
+      HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY);
     REGIONS.clear();
     Collections.addAll(REGIONS, "bbb", "ccc");
     InstrumentedLogWriter.activateFailure = false;
@@ -386,7 +389,7 @@ public class TestWALSplit {
             TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
             new WALEdit());
     Path p = WALSplitter.getRegionSplitEditsPath(entry,
-        FILENAME_BEING_SPLIT, conf);
+        FILENAME_BEING_SPLIT, TMPDIRNAME, conf);
     String parentOfParent = p.getParent().getParent().getName();
     assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
   }
@@ -412,7 +415,7 @@ public class TestWALSplit {
     fs.createNewFile(parent); // create a recovered.edits file
 
     Path p = WALSplitter.getRegionSplitEditsPath(entry,
-        FILENAME_BEING_SPLIT, conf);
+        FILENAME_BEING_SPLIT, TMPDIRNAME, conf);
     String parentOfParent = p.getParent().getParent().getName();
     assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
     WALFactory.createRecoveredEditsWriter(fs, p, conf).close();


[33/50] [abbrv] hbase git commit: HBASE-20732 Shutdown scan pool when master is stopped

Posted by ap...@apache.org.
HBASE-20732 Shutdown scan pool when master is stopped

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java


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

Branch: refs/heads/branch-1.3
Commit: 955264ed437165fbb26f04172f617fb44835ff6b
Parents: 0824695
Author: Reid Chan <re...@apache.org>
Authored: Wed Jun 27 18:55:21 2018 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:19 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java |  1 +
 .../hbase/master/cleaner/CleanerChore.java      | 74 ++++++++++++--------
 .../hbase/master/cleaner/TestCleanerChore.java  | 16 +++--
 3 files changed, 55 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/955264ed/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 67c7787..b47fecb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -1228,6 +1228,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     }
     super.stopServiceThreads();
     stopChores();
+    CleanerChore.shutDownChorePool();
 
     // Wait for all the remaining region servers to report in IFF we were
     // running a cluster shutdown AND we were NOT aborting.

http://git-wip-us.apache.org/repos/asf/hbase/blob/955264ed/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
index b3f1f0a..7d38ddb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.master.cleaner;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -118,7 +119,7 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
           break;
         }
       }
-      pool.shutdownNow();
+      shutDownNow();
       LOG.info("Update chore's pool size from " + pool.getParallelism() + " to " + size);
       pool = new ForkJoinPool(size);
     }
@@ -136,6 +137,13 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
     synchronized void submit(ForkJoinTask task) {
       pool.submit(task);
     }
+
+    synchronized void shutDownNow() {
+      if (pool == null || pool.isShutdown()) {
+        return;
+      }
+      pool.shutdownNow();
+    }
   }
   // It may be waste resources for each cleaner chore own its pool,
   // so let's make pool for all cleaner chores.
@@ -148,17 +156,24 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
   protected Map<String, Object> params;
   private AtomicBoolean enabled = new AtomicBoolean(true);
 
-  public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Configuration conf,
-                      FileSystem fs, Path oldFileDir, String confKey) {
-    this(name, sleepPeriod, s, conf, fs, oldFileDir, confKey, null);
-  }
-
   public static void initChorePool(Configuration conf) {
     if (POOL == null) {
       POOL = new DirScanPool(conf);
     }
   }
 
+  public static void shutDownChorePool() {
+    if (POOL != null) {
+      POOL.shutDownNow();
+      POOL = null;
+    }
+  }
+
+  public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Configuration conf,
+                      FileSystem fs, Path oldFileDir, String confKey) {
+    this(name, sleepPeriod, s, conf, fs, oldFileDir, confKey, null);
+  }
+
   /**
    * @param name name of the chore being run
    * @param sleepPeriod the period of time to sleep between each run
@@ -432,6 +447,7 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
     protected Boolean compute() {
       LOG.trace("Cleaning under " + dir);
       List<FileStatus> subDirs;
+      List<FileStatus> tmpFiles;
       final List<FileStatus> files;
       try {
         // if dir doesn't exist, we'll get null back for both of these
@@ -442,48 +458,48 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
             return f.isDirectory();
           }
         });
-        files = FSUtils.listStatusWithStatusFilter(fs, dir, new FileStatusFilter() {
+        if (subDirs == null) {
+          subDirs = Collections.emptyList();
+        }
+        tmpFiles = FSUtils.listStatusWithStatusFilter(fs, dir, new FileStatusFilter() {
           @Override
           public boolean accept(FileStatus f) {
             return f.isFile();
           }
         });
+        files = tmpFiles == null ? Collections.<FileStatus>emptyList() : tmpFiles;
       } catch (IOException ioe) {
         LOG.warn("failed to get FileStatus for contents of '" + dir + "'", ioe);
         return false;
       }
 
-      boolean nullSubDirs = subDirs == null;
-      if (nullSubDirs) {
-        LOG.trace("There is no subdir under " + dir);
-      }
-      if (files == null) {
-        LOG.trace("There is no file under " + dir);
+      boolean allFilesDeleted = true;
+      if (!files.isEmpty()) {
+        allFilesDeleted = deleteAction(new Action<Boolean>() {
+          @Override
+          public Boolean act() throws IOException {
+            return checkAndDeleteFiles(files);
+          }
+        }, "files");
       }
 
-      int capacity = nullSubDirs ? 0 : subDirs.size();
-      final List<CleanerTask> tasks = Lists.newArrayListWithCapacity(capacity);
-      if (!nullSubDirs) {
+      boolean allSubdirsDeleted = true;
+      if (!subDirs.isEmpty()) {
+        final List<CleanerTask> tasks = Lists.newArrayListWithCapacity(subDirs.size());
         for (FileStatus subdir : subDirs) {
           CleanerTask task = new CleanerTask(subdir, false);
           tasks.add(task);
           task.fork();
         }
+        allSubdirsDeleted = deleteAction(new Action<Boolean>() {
+          @Override
+          public Boolean act() throws IOException {
+            return getCleanResult(tasks);
+          }
+        }, "subdirs");
       }
 
-      boolean result = true;
-      result &= deleteAction(new Action<Boolean>() {
-        @Override
-        public Boolean act() throws IOException {
-          return checkAndDeleteFiles(files);
-        }
-      }, "files");
-      result &= deleteAction(new Action<Boolean>() {
-        @Override
-        public Boolean act() throws IOException {
-          return getCleanResult(tasks);
-        }
-      }, "subdirs");
+      boolean result = allFilesDeleted && allSubdirsDeleted;
       // if and only if files and subdirs under current dir are deleted successfully, and
       // it is not the root dir, then task will try to delete it.
       if (result && !root) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/955264ed/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
index 7711354..12f78be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
@@ -38,8 +38,8 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.StoppableImplementation;
-import org.junit.After;
-import org.junit.Before;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
@@ -52,16 +52,17 @@ public class TestCleanerChore {
   private static final Log LOG = LogFactory.getLog(TestCleanerChore.class);
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
-  @Before
-  public void setup() throws Exception {
+  @BeforeClass
+  public static void setup() {
     CleanerChore.initChorePool(UTIL.getConfiguration());
   }
 
-  @After
-  public void cleanup() throws Exception {
+  @AfterClass
+  public static void cleanup() throws Exception {
     // delete and recreate the test directory, ensuring a clean test dir between tests
     UTIL.cleanupTestDir();
-}
+    CleanerChore.shutDownChorePool();
+  }
 
 
   @Test
@@ -296,6 +297,7 @@ public class TestCleanerChore {
    */
   @Test
   public void testNoExceptionFromDirectoryWithRacyChildren() throws Exception {
+    UTIL.cleanupTestDir();
     Stoppable stop = new StoppableImplementation();
     // need to use a localutil to not break the rest of the test that runs on the local FS, which
     // gets hosed when we start to use a minicluster.


[24/50] [abbrv] hbase git commit: HBASE-19553 Old replica regions should be cleared from AM memory after primary region split or merge

Posted by ap...@apache.org.
HBASE-19553 Old replica regions should be cleared from AM memory after primary region split or merge

Signed-off-by: Ted Yu <te...@apache.org>
Signed-off-by: Huaxiang Sun <hu...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 5c799c18f684121352b1f071676a442a8efdc8a1
Parents: b3f911c
Author: Pankaj Kumar <pa...@huawei.com>
Authored: Thu Jan 25 13:25:22 2018 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:18 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/AssignmentManager.java  |  17 ++++
 .../apache/hadoop/hbase/client/TestAdmin1.java  | 102 +++++++++++++++++--
 2 files changed, 111 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5c799c18/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index 63ef5ff..79fe596 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -4256,6 +4256,8 @@ public class AssignmentManager extends ZooKeeperListener {
       LOG.warn("Couldn't assign all replica(s) of region " + mergedHri+ " because of " +
                 ie.getMessage());
     }
+    // Remove merged region's replica from AM's memory
+    clearReplicaRegions(c);
   }
 
   private void doSplittingOfReplicas(final HRegionInfo parentHri, final HRegionInfo hri_a,
@@ -4299,6 +4301,21 @@ public class AssignmentManager extends ZooKeeperListener {
     } catch (InterruptedException e) {
       LOG.warn("Caught exception " + e + " while trying to assign replica(s) of daughter(s)");
     }
+    // Remove parent region's replica from AM's memory
+    clearReplicaRegions(c);
+  }
+
+  /*
+   * Clear the replica regions after region split or merge.
+   */
+  private void clearReplicaRegions(Collection<List<HRegionInfo>> regionInfos) {
+    for (List<HRegionInfo> regionInfoList : regionInfos) {
+      for (HRegionInfo regionInfo : regionInfoList) {
+        if (!RegionReplicaUtil.isDefaultReplica(regionInfo)) {
+          regionStates.deleteRegion(regionInfo);
+        }
+      }
+    }
   }
 
   private void prepareDaughterReplicaForAssignment(HRegionInfo daughterHri, HRegionInfo parentHri,

http://git-wip-us.apache.org/repos/asf/hbase/blob/5c799c18/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index d153a85..e9fa3e3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -56,7 +57,9 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
+import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
@@ -1168,15 +1171,8 @@ public class TestAdmin1 {
     // regions.
     // Set up a table with 3 regions and replication set to 3
     TableName tableName = TableName.valueOf("testSplitAndMergeWithReplicaTable");
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.setRegionReplication(3);
     byte[] cf = "f".getBytes();
-    HColumnDescriptor hcd = new HColumnDescriptor(cf);
-    desc.addFamily(hcd);
-    byte[][] splitRows = new byte[2][];
-    splitRows[0] = new byte[]{(byte)'4'};
-    splitRows[1] = new byte[]{(byte)'7'};
-    TEST_UTIL.getHBaseAdmin().createTable(desc, splitRows);
+    createReplicaTable(tableName, cf);
     List<HRegion> oldRegions;
     do {
       oldRegions = TEST_UTIL.getHBaseCluster().getRegions(tableName);
@@ -1284,6 +1280,96 @@ public class TestAdmin1 {
   }
 
   /**
+   * Test case to validate whether parent's replica regions are cleared from AM's memory after
+   * SPLIT/MERGE.
+   */
+  @Test
+  public void testRegionStateCleanupFromAMMemoryAfterRegionSplitAndMerge() throws Exception {
+    final TableName tableName =
+        TableName.valueOf("testRegionStateCleanupFromAMMemoryAfterRegionSplitAndMerge");
+    createReplicaTable(tableName, "f".getBytes());
+    final int regionReplication = admin.getTableDescriptor(tableName).getRegionReplication();
+
+    List<Pair<HRegionInfo, ServerName>> regions = MetaTableAccessor.getTableRegionsAndLocations(
+      TEST_UTIL.getZooKeeperWatcher(), TEST_UTIL.getConnection(), tableName);
+    assertEquals(9, regions.size());
+    final int primaryRegionCount = regions.size() / regionReplication;
+
+    final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
+    List<HRegionInfo> splitRegions =
+        am.getRegionStates().getRegionByStateOfTable(tableName).get(RegionState.State.SPLIT);
+    assertEquals(0, splitRegions.size());
+
+    // Validate region split
+    byte[] regionName = regions.get(0).getFirst().getRegionName();
+    try {
+      TEST_UTIL.getHBaseAdmin().split(regionName, Bytes.toBytes('2'));
+    } catch (IllegalArgumentException ex) {
+      fail("Exception occured during region split" + ex);
+    }
+
+    // Wait for replica region to become online
+    TEST_UTIL.waitFor(60000, 500, new Waiter.Predicate<IOException>() {
+      @Override
+      public boolean evaluate() throws IOException {
+        return am.getRegionStates().getRegionByStateOfTable(tableName).get(RegionState.State.OPEN)
+            .size() == (primaryRegionCount + 1) * regionReplication;
+      }
+    });
+
+    regions = MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getZooKeeperWatcher(),
+      TEST_UTIL.getConnection(), tableName);
+    assertEquals(12, regions.size());
+    final int primaryRegionCountAfterSplit = regions.size() / regionReplication;
+
+    // Split region after region split
+    splitRegions =
+        am.getRegionStates().getRegionByStateOfTable(tableName).get(RegionState.State.SPLIT);
+    // Parent's replica region should be removed from AM's memory.
+    assertEquals(1, splitRegions.size());
+
+    // Validate region merge
+    HRegionInfo regionA = regions.get(3).getFirst();
+    HRegionInfo regionB = regions.get(6).getFirst();
+    try {
+      TEST_UTIL.getHBaseAdmin().mergeRegions(regionA.getRegionName(), regionB.getRegionName(),
+        true);
+    } catch (IllegalArgumentException ex) {
+      fail("Exception occured during region merge" + ex);
+    }
+
+    // Wait for replica regions to become online
+    TEST_UTIL.waitFor(60000, 500, new Waiter.Predicate<IOException>() {
+      @Override
+      public boolean evaluate() throws IOException {
+        return am.getRegionStates().getRegionByStateOfTable(tableName).get(RegionState.State.OPEN)
+            .size() == (primaryRegionCountAfterSplit - 1) * regionReplication;
+      }
+    });
+
+    regions = MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getZooKeeperWatcher(),
+      TEST_UTIL.getConnection(), tableName);
+    assertEquals(9, regions.size());
+    // Offline region after region merge
+    List<HRegionInfo> offlineRegions =
+        am.getRegionStates().getRegionByStateOfTable(tableName).get(RegionState.State.OFFLINE);
+    // Parent's replica region should be removed from AM's memory.
+    assertEquals(0, offlineRegions.size());
+  }
+
+  private byte[] createReplicaTable(TableName tableName, byte[] cf) throws IOException {
+    HTableDescriptor desc = new HTableDescriptor(tableName);
+    desc.setRegionReplication(3);
+    HColumnDescriptor hcd = new HColumnDescriptor(cf);
+    desc.addFamily(hcd);
+    byte[][] splitRows = new byte[2][];
+    splitRows[0] = new byte[] { (byte) '4' };
+    splitRows[1] = new byte[] { (byte) '7' };
+    TEST_UTIL.getHBaseAdmin().createTable(desc, splitRows);
+    return cf;
+  }
+
+  /**
    * HADOOP-2156
    * @throws IOException
    */


[18/50] [abbrv] hbase git commit: HBASE-20141 Fix TooManyFiles exception when RefreshingChannels

Posted by ap...@apache.org.
HBASE-20141 Fix TooManyFiles exception when RefreshingChannels

HBASE-19435 implements a fix for reopening file channels when they are unnexpected closed
to avoid disabling the BucketCache. However, it was missed that the the channels might not
actually be completely closed (the write or read channel might still be open
(see https://docs.oracle.com/javase/7/docs/api/java/nio/channels/ClosedChannelException.html)
This commit closes any open channels before creating a new channel.


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

Branch: refs/heads/branch-1.3
Commit: ecfa9a8a7997a188c1f3ec2d7776245664dc7dbe
Parents: 8bc84fd
Author: Zach York <zy...@amazon.com>
Authored: Wed Feb 28 10:40:38 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Dec 12 18:08:18 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/io/hfile/bucket/FileIOEngine.java     | 13 +++++++++++--
 .../hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java | 11 +++++++++++
 2 files changed, 22 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ecfa9a8a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
index cb454d4..7b773bd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.io.hfile.bucket;
 
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
@@ -234,7 +233,17 @@ public class FileIOEngine implements IOEngine {
     return fileNum;
   }
 
-  private void refreshFileConnection(int accessFileNum) throws FileNotFoundException {
+  @VisibleForTesting
+  FileChannel[] getFileChannels() {
+    return fileChannels;
+  }
+
+  @VisibleForTesting
+  void refreshFileConnection(int accessFileNum) throws IOException {
+    FileChannel fileChannel = fileChannels[accessFileNum];
+    if (fileChannel != null) {
+      fileChannel.close();
+    }
     rafs[accessFileNum] = new RandomAccessFile(filePaths[accessFileNum], "rw");
     fileChannels[accessFileNum] = rafs[accessFileNum].getChannel();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecfa9a8a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
index adf7fd0..8c2bc6e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileIOEngine.java
@@ -19,10 +19,13 @@
 package org.apache.hadoop.hbase.io.hfile.bucket;
 
 import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 
 import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -129,4 +132,12 @@ public class TestFileIOEngine {
     fileIOEngine.read(ByteBuffer.wrap(data2), offset);
     assertArrayEquals(data1, data2);
   }
+
+  @Test
+  public void testRefreshFileConnectionClosesConnections() throws IOException {
+    FileChannel fileChannel = fileIOEngine.getFileChannels()[0];
+    assertNotNull(fileChannel);
+    fileIOEngine.refreshFileConnection(0);
+    assertFalse(fileChannel.isOpen());
+  }
 }