saintstack commented on a change in pull request #1753:
URL: https://github.com/apache/hbase/pull/1753#discussion_r429033369



##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/BaseTimeToLiveFileCleaner.java
##########
@@ -0,0 +1,92 @@
+/**
+ * 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.cleaner;
+
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base class for time to live file cleaner.
+ */
+@InterfaceAudience.Private
+public abstract class BaseTimeToLiveFileCleaner extends BaseLogCleanerDelegate 
{
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(BaseTimeToLiveFileCleaner.class.getName());
+
+  private static final DateTimeFormatter FORMATTER =
+    DateTimeFormatter.ISO_DATE_TIME.withZone(ZoneOffset.systemDefault());
+
+  // Configured time a log can be kept after it was closed
+  private long ttlMs;
+
+  private boolean stopped = false;
+
+  @Override
+  public final void setConf(Configuration conf) {
+    super.setConf(conf);
+    this.ttlMs = getTtlMs(conf);
+  }
+
+  @Override
+  public boolean isFileDeletable(FileStatus status) {
+    // Files are validated for the second time here,
+    // if it causes a bottleneck this logic needs refactored
+    if (!valiateFilename(status.getPath())) {
+      return true;
+    }
+    long currentTime = EnvironmentEdgeManager.currentTime();
+    long time = status.getModificationTime();
+    long life = currentTime - time;
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("File life:{}ms, ttl:{}ms, current:{}, from{}", life, ttlMs,
+        FORMATTER.format(Instant.ofEpochMilli(currentTime)),
+        FORMATTER.format(Instant.ofEpochMilli(time)));
+    }
+    if (life < 0) {
+      LOG.warn("Found a file ({}) newer than current time ({} < {}), probably 
a clock skew",
+        status.getPath(), FORMATTER.format(Instant.ofEpochMilli(currentTime)),
+        FORMATTER.format(Instant.ofEpochMilli(time)));
+      return false;
+    }
+    return life > ttlMs;
+  }
+
+  @Override
+  public void stop(String why) {
+    this.stopped = true;

Review comment:
       Someone logs the 'why'?

##########
File path: hbase-common/src/main/resources/hbase-default.xml
##########
@@ -125,7 +125,7 @@ possible configurations would overwhelm and obscure the 
important.
   </property>
   <property>
     <name>hbase.master.logcleaner.plugins</name>
-    
<value>org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner,org.apache.hadoop.hbase.master.cleaner.TimeToLiveProcedureWALCleaner</value>
+    
<value>org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner,org.apache.hadoop.hbase.master.cleaner.TimeToLiveProcedureWALCleaner,org.apache.hadoop.hbase.master.cleaner.TimeToLiveMasterLocalStoreWALCleaner</value>

Review comment:
       We want the procedureWALCleaner here still ? Different from the general 
master local store cleaner?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
##########
@@ -86,8 +87,9 @@ public LogCleaner(final int period, final Stoppable stopper, 
Configuration conf,
 
   @Override
   protected boolean validate(Path file) {
-    return AbstractFSWALProvider.validateWALFilename(file.getName())
-        || MasterProcedureUtil.validateProcedureWALFilename(file.getName());
+    return AbstractFSWALProvider.validateWALFilename(file.getName()) ||
+      MasterProcedureUtil.validateProcedureWALFilename(file.getName()) ||
+      file.getName().endsWith(LocalStore.ARCHIVED_WAL_SUFFIX);
   }
 
   @Override

Review comment:
       Man, we should have called this file the WALCleaner, not LogCleaner. Not 
your fault.

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/BaseTimeToLiveFileCleaner.java
##########
@@ -0,0 +1,92 @@
+/**
+ * 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.cleaner;
+
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base class for time to live file cleaner.
+ */
+@InterfaceAudience.Private
+public abstract class BaseTimeToLiveFileCleaner extends BaseLogCleanerDelegate 
{
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(BaseTimeToLiveFileCleaner.class.getName());
+
+  private static final DateTimeFormatter FORMATTER =
+    DateTimeFormatter.ISO_DATE_TIME.withZone(ZoneOffset.systemDefault());
+
+  // Configured time a log can be kept after it was closed
+  private long ttlMs;
+
+  private boolean stopped = false;
+
+  @Override
+  public final void setConf(Configuration conf) {
+    super.setConf(conf);
+    this.ttlMs = getTtlMs(conf);
+  }
+
+  @Override
+  public boolean isFileDeletable(FileStatus status) {
+    // Files are validated for the second time here,
+    // if it causes a bottleneck this logic needs refactored
+    if (!valiateFilename(status.getPath())) {

Review comment:
       Why a second time? Just-in-case? Can you say why we do it twice in 
comment? What you are afraid of?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/store/LocalRegion.java
##########
@@ -0,0 +1,325 @@
+/**
+ * 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.store;
+
+import static org.apache.hadoop.hbase.HConstants.HREGION_LOGDIR_NAME;
+
+import java.io.IOException;
+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.HBaseIOException;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.apache.hadoop.hbase.util.RecoverLeaseFSUtils;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.math.IntMath;
+
+/**
+ * A region that stores data in a separated directory.
+ * <p/>
+ * FileSystem layout:
+ *
+ * <pre>
+ * hbase
+ *   |
+ *   --&lt;region dir&gt;
+ *       |
+ *       --data
+ *       |  |
+ *       |  --/&lt;ns&gt/&lt;table&gt/&lt;encoded-region-name&gt; <---- The 
region data
+ *       |      |
+ *       |      --replay <---- The edits to replay
+ *       |
+ *       --WALs
+ *          |
+ *          --&lt;master-server-name&gt; <---- The WAL dir for active master
+ *          |
+ *          --&lt;master-server-name&gt;-dead <---- The WAL dir for dead master
+ * </pre>
+ *
+ * Notice that, you can use different root file system and WAL file system. 
Then the above directory
+ * will be on two file systems, the root file system will have the data 
directory while the WAL
+ * filesystem will have the WALs directory. The archived HFile will be moved 
to the global HFile
+ * archived directory with the {@link LocalRegionParams#archivedWalSuffix()} 
suffix. The archived
+ * WAL will be moved to the global WAL archived directory with the
+ * {@link LocalRegionParams#archivedHFileSuffix()} suffix.
+ */
+@InterfaceAudience.Private
+public final class LocalRegion {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LocalRegion.class);
+
+  private static final String REPLAY_EDITS_DIR = "recovered.wals";

Review comment:
       What goes here?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/store/LocalStore.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.master.store;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import 
org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Used for storing data at master side. The data will be stored in a {@link 
LocalRegion}.
+ */
+@InterfaceAudience.Private
+public final class LocalStore {
+
+  // Use the character $ to let the log cleaner know that this is not the 
normal wal file.
+  public static final String ARCHIVED_WAL_SUFFIX = "$masterlocalwal$";
+
+  public static final String ARCHIVED_HFILE_SUFFIX = "$masterlocalhfile$";
+
+  private static final String MAX_WALS_KEY = 
"hbase.master.store.region.maxwals";
+
+  private static final int DEFAULT_MAX_WALS = 10;
+
+  public static final String USE_HSYNC_KEY = 
"hbase.master.store.region.wal.hsync";
+
+  public static final String MASTER_STORE_DIR = "MasterData";
+
+  private static final String FLUSH_SIZE_KEY = 
"hbase.master.store.region.flush.size";
+
+  private static final long DEFAULT_FLUSH_SIZE = 
TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE;
+
+  private static final String FLUSH_PER_CHANGES_KEY = 
"hbase.master.store.region.flush.per.changes";
+
+  private static final long DEFAULT_FLUSH_PER_CHANGES = 1_000_000;
+
+  private static final String FLUSH_INTERVAL_MS_KEY = 
"hbase.master.store.region.flush.interval.ms";
+
+  // default to flush every 15 minutes, for safety
+  private static final long DEFAULT_FLUSH_INTERVAL_MS = 
TimeUnit.MINUTES.toMillis(15);
+
+  private static final String COMPACT_MIN_KEY = 
"hbase.master.store.region.compact.min";
+
+  private static final int DEFAULT_COMPACT_MIN = 4;
+
+  private static final String ROLL_PERIOD_MS_KEY = 
"hbase.master.store.region.walroll.period.ms";
+
+  private static final long DEFAULT_ROLL_PERIOD_MS = 
TimeUnit.MINUTES.toMillis(15);
+
+  private static final String RING_BUFFER_SLOT_COUNT = 
"hbase.master.store.ringbuffer.slot.count";
+
+  private static final int DEFAULT_RING_BUFFER_SLOT_COUNT = 128;
+
+  public static final TableName TABLE_NAME = TableName.valueOf("master:store");
+
+  public static final byte[] PROC_FAMILY = Bytes.toBytes("proc");
+
+  private static final TableDescriptor TABLE_DESC = 
TableDescriptorBuilder.newBuilder(TABLE_NAME)
+    .setColumnFamily(ColumnFamilyDescriptorBuilder.of(PROC_FAMILY)).build();

Review comment:
       oh, the second family will come in here if we decide to store root table 
in here?  Not till then. Ok. I think that makes sense. If table already exists, 
will have to alter it when we add the root table CF?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/BaseTimeToLiveFileCleaner.java
##########
@@ -0,0 +1,92 @@
+/**
+ * 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.cleaner;
+
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base class for time to live file cleaner.
+ */
+@InterfaceAudience.Private
+public abstract class BaseTimeToLiveFileCleaner extends BaseLogCleanerDelegate 
{
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(BaseTimeToLiveFileCleaner.class.getName());
+
+  private static final DateTimeFormatter FORMATTER =
+    DateTimeFormatter.ISO_DATE_TIME.withZone(ZoneOffset.systemDefault());
+
+  // Configured time a log can be kept after it was closed
+  private long ttlMs;
+
+  private boolean stopped = false;

Review comment:
       Need to be volatile?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/BaseTimeToLiveFileCleaner.java
##########
@@ -0,0 +1,92 @@
+/**
+ * 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.cleaner;
+
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base class for time to live file cleaner.
+ */
+@InterfaceAudience.Private
+public abstract class BaseTimeToLiveFileCleaner extends BaseLogCleanerDelegate 
{
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(BaseTimeToLiveFileCleaner.class.getName());
+
+  private static final DateTimeFormatter FORMATTER =
+    DateTimeFormatter.ISO_DATE_TIME.withZone(ZoneOffset.systemDefault());
+
+  // Configured time a log can be kept after it was closed
+  private long ttlMs;
+
+  private boolean stopped = false;
+
+  @Override
+  public final void setConf(Configuration conf) {
+    super.setConf(conf);
+    this.ttlMs = getTtlMs(conf);
+  }
+
+  @Override
+  public boolean isFileDeletable(FileStatus status) {
+    // Files are validated for the second time here,
+    // if it causes a bottleneck this logic needs refactored
+    if (!valiateFilename(status.getPath())) {

Review comment:
       s/valiate/validate/
   

##########
File path: hbase-common/src/main/resources/hbase-default.xml
##########
@@ -139,16 +139,9 @@ possible configurations would overwhelm and obscure the 
important.
     <description>How long a WAL remain in the archive 
({hbase.rootdir}/oldWALs) directory,
     after which it will be cleaned by a Master thread. The value is in 
milliseconds.</description>
   </property>
-  <property>
-    <name>hbase.master.procedurewalcleaner.ttl</name>
-    <value>604800000</value>
-    <description>How long a Procedure WAL will remain in the
-    archive directory, after which it will be cleaned
-    by a Master thread. The value is in milliseconds.</description>
-  </property>
   <property>
     <name>hbase.master.hfilecleaner.plugins</name>
-    
<value>org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner</value>
+    
<value>org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner,org.apache.hadoop.hbase.master.cleaner.TimeToLiveMasterLocalStoreHFileCleaner</value>

Review comment:
       Great

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/BaseTimeToLiveFileCleaner.java
##########
@@ -0,0 +1,92 @@
+/**
+ * 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.cleaner;
+
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base class for time to live file cleaner.
+ */
+@InterfaceAudience.Private
+public abstract class BaseTimeToLiveFileCleaner extends BaseLogCleanerDelegate 
{
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(BaseTimeToLiveFileCleaner.class.getName());
+
+  private static final DateTimeFormatter FORMATTER =
+    DateTimeFormatter.ISO_DATE_TIME.withZone(ZoneOffset.systemDefault());
+
+  // Configured time a log can be kept after it was closed
+  private long ttlMs;
+
+  private boolean stopped = false;
+
+  @Override
+  public final void setConf(Configuration conf) {
+    super.setConf(conf);
+    this.ttlMs = getTtlMs(conf);
+  }
+
+  @Override
+  public boolean isFileDeletable(FileStatus status) {
+    // Files are validated for the second time here,
+    // if it causes a bottleneck this logic needs refactored
+    if (!valiateFilename(status.getPath())) {
+      return true;
+    }
+    long currentTime = EnvironmentEdgeManager.currentTime();
+    long time = status.getModificationTime();
+    long life = currentTime - time;
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("File life:{}ms, ttl:{}ms, current:{}, from{}", life, ttlMs,
+        FORMATTER.format(Instant.ofEpochMilli(currentTime)),
+        FORMATTER.format(Instant.ofEpochMilli(time)));
+    }
+    if (life < 0) {
+      LOG.warn("Found a file ({}) newer than current time ({} < {}), probably 
a clock skew",
+        status.getPath(), FORMATTER.format(Instant.ofEpochMilli(currentTime)),
+        FORMATTER.format(Instant.ofEpochMilli(time)));
+      return false;
+    }
+    return life > ttlMs;
+  }
+
+  @Override
+  public void stop(String why) {
+    this.stopped = true;

Review comment:
       Maybe it is not important enough to log.

##########
File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/master/store/TestLocalRegionOnTwoFileSystems.java
##########
@@ -0,0 +1,170 @@
+/**
+ * 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.store;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+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.ChoreService;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.cleaner.DirScanPool;
+import org.apache.hadoop.hbase.regionserver.MemStoreLAB;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+
+@Category({ MasterTests.class, MediumTests.class })
+public class TestLocalRegionOnTwoFileSystems {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestLocalRegionOnTwoFileSystems.class);
+
+  private static final HBaseCommonTestingUtility HFILE_UTIL = new 
HBaseCommonTestingUtility();
+
+  private static final HBaseTestingUtility WAL_UTIL = new 
HBaseTestingUtility();
+
+  private static ChoreService CHORE_SERVICE;
+
+  private static DirScanPool CLEANER_POOL;
+
+  private static LocalRegion REGION;
+
+  private static byte[] CF = Bytes.toBytes("f");
+
+  private static byte[] QUALIFIER = Bytes.toBytes("q");
+
+  private static String REGION_DIR_NAME = "local";
+
+  private static TableDescriptor TD =
+    TableDescriptorBuilder.newBuilder(TableName.valueOf("test:local"))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build();
+
+  private static int COMPACT_MIN = 4;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    WAL_UTIL.startMiniCluster(3);
+    Configuration conf = HFILE_UTIL.getConfiguration();
+    conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, false);
+    CHORE_SERVICE = new ChoreService("TestLocalRegionOnTwoFileSystems");
+    CLEANER_POOL = new DirScanPool(conf);
+    Server server = mock(Server.class);
+    when(server.getConfiguration()).thenReturn(conf);
+    when(server.getServerName())
+      .thenReturn(ServerName.valueOf("localhost", 12345, 
System.currentTimeMillis()));
+    when(server.getChoreService()).thenReturn(CHORE_SERVICE);
+    Path rootDir = HFILE_UTIL.getDataTestDir();
+    CommonFSUtils.setRootDir(conf, rootDir);
+    Path walRootDir = WAL_UTIL.getDataTestDirOnTestFS();
+    FileSystem walFs = WAL_UTIL.getTestFileSystem();
+    CommonFSUtils.setWALRootDir(conf,
+      walRootDir.makeQualified(walFs.getUri(), walFs.getWorkingDirectory()));
+    LocalRegionParams params = new LocalRegionParams();
+    params.server(server).regionDirName(REGION_DIR_NAME).tableDescriptor(TD)
+      
.flushSize(TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE).flushPerChanges(1_000_000)
+      
.flushIntervalMs(TimeUnit.MINUTES.toMillis(15)).compactMin(COMPACT_MIN).maxWals(32)
+      
.useHsync(false).ringBufferSlotCount(16).rollPeriodMs(TimeUnit.MINUTES.toMillis(15))
+      .archivedWalSuffix(LocalStore.ARCHIVED_WAL_SUFFIX)
+      .archivedHFileSuffix(LocalStore.ARCHIVED_HFILE_SUFFIX);
+    REGION = LocalRegion.create(params);

Review comment:
       Good

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/store/LocalRegion.java
##########
@@ -0,0 +1,325 @@
+/**
+ * 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.store;
+
+import static org.apache.hadoop.hbase.HConstants.HREGION_LOGDIR_NAME;
+
+import java.io.IOException;
+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.HBaseIOException;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.apache.hadoop.hbase.util.RecoverLeaseFSUtils;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.math.IntMath;
+
+/**
+ * A region that stores data in a separated directory.
+ * <p/>
+ * FileSystem layout:
+ *
+ * <pre>
+ * hbase
+ *   |
+ *   --&lt;region dir&gt;
+ *       |
+ *       --data
+ *       |  |
+ *       |  --/&lt;ns&gt/&lt;table&gt/&lt;encoded-region-name&gt; <---- The 
region data
+ *       |      |
+ *       |      --replay <---- The edits to replay
+ *       |
+ *       --WALs
+ *          |
+ *          --&lt;master-server-name&gt; <---- The WAL dir for active master
+ *          |
+ *          --&lt;master-server-name&gt;-dead <---- The WAL dir for dead master
+ * </pre>
+ *
+ * Notice that, you can use different root file system and WAL file system. 
Then the above directory
+ * will be on two file systems, the root file system will have the data 
directory while the WAL
+ * filesystem will have the WALs directory. The archived HFile will be moved 
to the global HFile
+ * archived directory with the {@link LocalRegionParams#archivedWalSuffix()} 
suffix. The archived
+ * WAL will be moved to the global WAL archived directory with the
+ * {@link LocalRegionParams#archivedHFileSuffix()} suffix.
+ */
+@InterfaceAudience.Private
+public final class LocalRegion {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LocalRegion.class);
+
+  private static final String REPLAY_EDITS_DIR = "recovered.wals";
+
+  private static final String DEAD_WAL_DIR_SUFFIX = "-dead";

Review comment:
       Whats this?

##########
File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/master/store/TestLocalRegionOnTwoFileSystems.java
##########
@@ -0,0 +1,170 @@
+/**
+ * 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.store;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+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.ChoreService;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.cleaner.DirScanPool;
+import org.apache.hadoop.hbase.regionserver.MemStoreLAB;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+
+@Category({ MasterTests.class, MediumTests.class })
+public class TestLocalRegionOnTwoFileSystems {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestLocalRegionOnTwoFileSystems.class);
+
+  private static final HBaseCommonTestingUtility HFILE_UTIL = new 
HBaseCommonTestingUtility();
+
+  private static final HBaseTestingUtility WAL_UTIL = new 
HBaseTestingUtility();
+
+  private static ChoreService CHORE_SERVICE;
+
+  private static DirScanPool CLEANER_POOL;
+
+  private static LocalRegion REGION;
+
+  private static byte[] CF = Bytes.toBytes("f");
+
+  private static byte[] QUALIFIER = Bytes.toBytes("q");
+
+  private static String REGION_DIR_NAME = "local";
+
+  private static TableDescriptor TD =
+    TableDescriptorBuilder.newBuilder(TableName.valueOf("test:local"))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build();
+
+  private static int COMPACT_MIN = 4;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    WAL_UTIL.startMiniCluster(3);
+    Configuration conf = HFILE_UTIL.getConfiguration();
+    conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, false);
+    CHORE_SERVICE = new ChoreService("TestLocalRegionOnTwoFileSystems");
+    CLEANER_POOL = new DirScanPool(conf);
+    Server server = mock(Server.class);
+    when(server.getConfiguration()).thenReturn(conf);
+    when(server.getServerName())
+      .thenReturn(ServerName.valueOf("localhost", 12345, 
System.currentTimeMillis()));
+    when(server.getChoreService()).thenReturn(CHORE_SERVICE);
+    Path rootDir = HFILE_UTIL.getDataTestDir();
+    CommonFSUtils.setRootDir(conf, rootDir);
+    Path walRootDir = WAL_UTIL.getDataTestDirOnTestFS();
+    FileSystem walFs = WAL_UTIL.getTestFileSystem();
+    CommonFSUtils.setWALRootDir(conf,
+      walRootDir.makeQualified(walFs.getUri(), walFs.getWorkingDirectory()));
+    LocalRegionParams params = new LocalRegionParams();
+    params.server(server).regionDirName(REGION_DIR_NAME).tableDescriptor(TD)
+      
.flushSize(TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE).flushPerChanges(1_000_000)
+      
.flushIntervalMs(TimeUnit.MINUTES.toMillis(15)).compactMin(COMPACT_MIN).maxWals(32)
+      
.useHsync(false).ringBufferSlotCount(16).rollPeriodMs(TimeUnit.MINUTES.toMillis(15))
+      .archivedWalSuffix(LocalStore.ARCHIVED_WAL_SUFFIX)
+      .archivedHFileSuffix(LocalStore.ARCHIVED_HFILE_SUFFIX);
+    REGION = LocalRegion.create(params);

Review comment:
       Are there params to set for local regions walfs and hfilefs?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/RegionProcedureStore.java
##########
@@ -183,51 +106,14 @@ public void start(int numThreads) throws IOException {
     this.numThreads = numThreads;
   }
 
-  private void shutdownWAL() {
-    if (walFactory != null) {
-      try {
-        walFactory.shutdown();
-      } catch (IOException e) {
-        LOG.warn("Failed to shutdown WAL", e);
-      }
-    }
-  }
-
-  private void closeRegion(boolean abort) {
-    if (region != null) {
-      try {
-        region.close(abort);
-      } catch (IOException e) {
-        LOG.warn("Failed to close region", e);
-      }
-    }
-
-  }
-
   @Override
   public void stop(boolean abort) {
     if (!setRunning(false)) {
       return;
     }
     LOG.info("Stopping the Region Procedure Store, isAbort={}", abort);
-    if (cleaner != null) {
-      cleaner.cancel(abort);
-    }
-    if (flusherAndCompactor != null) {
-      flusherAndCompactor.close();
-    }
-    // if abort, we shutdown wal first to fail the ongoing updates to the 
region, and then close the
-    // region, otherwise there will be dead lock.
-    if (abort) {
-      shutdownWAL();
-      closeRegion(true);
-    } else {
-      closeRegion(false);
-      shutdownWAL();
-    }
-
-    if (walRoller != null) {
-      walRoller.close();
+    if (localStore != null) {

Review comment:
       Is this the right place to stop this? Should the localStore be made by 
the Master, owned and shutdown by the Master? It passes it in here to the RPS 
to use? Master shuts it down. Makes sure it doesn't shut it down before other 
users are done with it?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to