z-york commented on a change in pull request #2931:
URL: https://github.com/apache/hbase/pull/2931#discussion_r593400626



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
##########
@@ -1646,6 +1647,18 @@
    */
   public static final int BATCH_ROWS_THRESHOLD_DEFAULT = 5000;
 
+  /**
+   * Configuration for storefile tracking feature
+   */
+  public static final String STOREFILE_TRACKING_PERSIST_ENABLED =
+    "hbase.storefile.tracking.persist.enabled";
+  public static final boolean DEFAULT_STOREFILE_TRACKING_PERSIST_ENABLED = 
false;
+
+  public static final String STOREFILE_TRACKING_INIT_TIMEOUT =
+    "hbase.storefile.tracking.init.timeout";
+  public static final long DEFAULT_STOREFILE_TRACKING_INIT_TIMEOUT =
+    TimeUnit.MINUTES.toMillis(5);
+

Review comment:
       I think this is a bit of an anti-pattern now to put constants in 
HConstants, but not a major issue IMO.

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HTableStoreFilePathAccessor.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
+/**
+ * Helper class to interact with the hbase:storefile system table
+ *
+ * <pre>
+ *   ROW-KEY              FAMILY:QUALIFIER      DATA VALUE
+ *   region-store-table   included:files        List&lt;Path&gt; 
filesIncludedInRead
+ * </pre>
+ *
+ * The region encoded name is set as prefix for region split loading balance, 
and we use the
+ * target table name as suffix such that operator can identify the records per 
table.
+ *
+ * included:files is used for persisting storefiles of StoreFileManager in the 
cases of store
+ * opens and store closes. Meanwhile compactedFiles of StoreFileManager isn't 
being tracked
+ * off-memory, because the updated included:files contains compactedFiles and 
the leftover
+ * compactedFiles are either archived when a store closes or opens.
+ *
+ * TODO we will need a followup change to introduce in-memory temporarily 
file, such that further
+ *      we can introduce a non-tracking temporarily storefiles left from a 
flush or compaction when
+ *      a regionserver crashes without closing the store properly
+ */

Review comment:
       The in-memory tmp tracking won't touch the Accessor as nothing will be 
persisted. I think you can remove this.

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/PersistedStoreFileManager.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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 java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import 
org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
+import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+import 
org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * A Storefile manager that is used by {@link PersistedStoreEngine} that 
persists the in-memory
+ * storefile tracking to a persistent table hbase:storefile.
+ *
+ * We don't override the {@link #clearFiles()} from {@link 
DefaultStoreFileManager} and persist
+ * in-memory storefiles tracking, it will be reused when region reassigns on a 
different
+ * region server.
+ */
+@InterfaceAudience.Private
+public class PersistedStoreFileManager extends DefaultStoreFileManager {
+  private static final Logger LOG = 
LoggerFactory.getLogger(PersistedStoreFileManager.class);
+  private final RegionInfo regionInfo;
+  private final String tableName;
+  private final String regionName;
+  private final String storeName;
+  private final StoreFilePathAccessor accessor;
+  private final Configuration conf;
+  // only uses for warmupHRegion
+  private final boolean readOnly;
+
+  public PersistedStoreFileManager(CellComparator cellComparator,
+    Comparator<HStoreFile> storeFileComparator, Configuration conf,
+    CompactionConfiguration compactionConfiguration, HRegionFileSystem 
regionFs,
+    RegionInfo regionInfo, String familyName, StoreFilePathAccessor accessor, 
boolean readOnly) {
+    super(cellComparator, storeFileComparator, conf, compactionConfiguration, 
regionFs, familyName);
+    this.conf = conf;
+    this.regionInfo = regionInfo;
+    this.tableName = regionInfo.getTable().getNameAsString();
+    this.regionName = regionInfo.getEncodedName();
+    this.storeName = familyName;
+    this.accessor = accessor;
+    this.readOnly = readOnly;
+  }
+
+  public PersistedStoreFileManager(CellComparator cellComparator,
+    Comparator<HStoreFile> storeFileComparator, Configuration conf,
+    CompactionConfiguration compactionConfiguration, HRegionFileSystem 
regionFs,
+    RegionInfo regionInfo, String familyName, StoreFilePathAccessor accessor) {
+    this(cellComparator, storeFileComparator, conf, compactionConfiguration, 
regionFs, regionInfo,
+      familyName, accessor, false);
+  }
+
+  @Override
+  public void loadFiles(List<HStoreFile> storeFiles) throws IOException {
+    // update with a sorted store files
+    super.loadFiles(storeFiles);
+    Preconditions.checkArgument(storeFiles != null, "store files cannot be "
+      + "null when loading");
+    if (storeFiles.isEmpty()) {
+      LOG.warn("Other than fresh region with no store files, store files 
should not be empty");
+      return;
+    }
+    
updatePathListToTracker(StoreFilePathUpdate.builder().withStoreFiles(getStorefiles()).build());
+  }
+
+  @Override
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    // concatenate the new store files
+    super.insertNewFiles(sfs);
+    // return in case of empty store files as it is a No-op, here empty files 
are expected
+    // during region close
+    if (CollectionUtils.isEmpty(getStorefiles())) {
+      return;
+    }
+    
updatePathListToTracker(StoreFilePathUpdate.builder().withStoreFiles(getStorefiles()).build());
+  }
+
+  @Override
+  protected void addCompactionResultsHook(ImmutableList<HStoreFile> storeFiles)
+    throws IOException {
+    Preconditions.checkNotNull(storeFiles, "storeFiles cannot be null");
+    
updatePathListToTracker(StoreFilePathUpdate.builder().withStoreFiles(storeFiles).build());
+  }
+
+  @Override
+  public Collection<StoreFileInfo> loadInitialFiles() throws IOException {
+    // this logic is totally different from the default implementation in 
DefaultStoreFileManager
+
+    List<Path> pathList = accessor.getIncludedStoreFilePaths(tableName, 
regionName, storeName);
+    boolean isEmptyInPersistedFilePaths = CollectionUtils.isEmpty(pathList);
+    if (isEmptyInPersistedFilePaths) {
+      // When the path accessor is returning empty result, we scan the
+      // the file storage and see if there is any existing HFiles should be 
loaded.
+      // the scan is a one time process when store open during region 
assignment.
+      //
+      // this is especially used for region and store open
+      // 1. First time migration from a filesystem based e.g. 
DefaultStoreFileEngine
+      // 2. After region split and merge
+      // 3. After table clone and create new HFiles directly into data 
directory
+      //
+      // Also we don't handle the inconsistency between storefile tracking and 
file system, which
+      // will be handled by a HBCK command

Review comment:
       JIRA?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/PersistedStoreFileManager.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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 java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import 
org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
+import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+import 
org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * A Storefile manager that is used by {@link PersistedStoreEngine} that 
persists the in-memory
+ * storefile tracking to a persistent table hbase:storefile.
+ *
+ * We don't override the {@link #clearFiles()} from {@link 
DefaultStoreFileManager} and persist
+ * in-memory storefiles tracking, it will be reused when region reassigns on a 
different
+ * region server.
+ */
+@InterfaceAudience.Private
+public class PersistedStoreFileManager extends DefaultStoreFileManager {
+  private static final Logger LOG = 
LoggerFactory.getLogger(PersistedStoreFileManager.class);
+  private final RegionInfo regionInfo;
+  private final String tableName;
+  private final String regionName;
+  private final String storeName;
+  private final StoreFilePathAccessor accessor;
+  private final Configuration conf;
+  // only uses for warmupHRegion
+  private final boolean readOnly;
+
+  public PersistedStoreFileManager(CellComparator cellComparator,
+    Comparator<HStoreFile> storeFileComparator, Configuration conf,
+    CompactionConfiguration compactionConfiguration, HRegionFileSystem 
regionFs,
+    RegionInfo regionInfo, String familyName, StoreFilePathAccessor accessor, 
boolean readOnly) {
+    super(cellComparator, storeFileComparator, conf, compactionConfiguration, 
regionFs, familyName);
+    this.conf = conf;
+    this.regionInfo = regionInfo;
+    this.tableName = regionInfo.getTable().getNameAsString();
+    this.regionName = regionInfo.getEncodedName();
+    this.storeName = familyName;
+    this.accessor = accessor;
+    this.readOnly = readOnly;
+  }
+
+  public PersistedStoreFileManager(CellComparator cellComparator,
+    Comparator<HStoreFile> storeFileComparator, Configuration conf,
+    CompactionConfiguration compactionConfiguration, HRegionFileSystem 
regionFs,
+    RegionInfo regionInfo, String familyName, StoreFilePathAccessor accessor) {
+    this(cellComparator, storeFileComparator, conf, compactionConfiguration, 
regionFs, regionInfo,
+      familyName, accessor, false);
+  }
+
+  @Override
+  public void loadFiles(List<HStoreFile> storeFiles) throws IOException {
+    // update with a sorted store files
+    super.loadFiles(storeFiles);
+    Preconditions.checkArgument(storeFiles != null, "store files cannot be "
+      + "null when loading");
+    if (storeFiles.isEmpty()) {
+      LOG.warn("Other than fresh region with no store files, store files 
should not be empty");
+      return;
+    }
+    
updatePathListToTracker(StoreFilePathUpdate.builder().withStoreFiles(getStorefiles()).build());
+  }
+
+  @Override
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    // concatenate the new store files
+    super.insertNewFiles(sfs);
+    // return in case of empty store files as it is a No-op, here empty files 
are expected
+    // during region close
+    if (CollectionUtils.isEmpty(getStorefiles())) {
+      return;
+    }
+    
updatePathListToTracker(StoreFilePathUpdate.builder().withStoreFiles(getStorefiles()).build());
+  }
+
+  @Override
+  protected void addCompactionResultsHook(ImmutableList<HStoreFile> storeFiles)
+    throws IOException {
+    Preconditions.checkNotNull(storeFiles, "storeFiles cannot be null");
+    
updatePathListToTracker(StoreFilePathUpdate.builder().withStoreFiles(storeFiles).build());
+  }
+
+  @Override
+  public Collection<StoreFileInfo> loadInitialFiles() throws IOException {
+    // this logic is totally different from the default implementation in 
DefaultStoreFileManager
+
+    List<Path> pathList = accessor.getIncludedStoreFilePaths(tableName, 
regionName, storeName);
+    boolean isEmptyInPersistedFilePaths = CollectionUtils.isEmpty(pathList);
+    if (isEmptyInPersistedFilePaths) {
+      // When the path accessor is returning empty result, we scan the
+      // the file storage and see if there is any existing HFiles should be 
loaded.
+      // the scan is a one time process when store open during region 
assignment.
+      //
+      // this is especially used for region and store open
+      // 1. First time migration from a filesystem based e.g. 
DefaultStoreFileEngine
+      // 2. After region split and merge
+      // 3. After table clone and create new HFiles directly into data 
directory
+      //
+      // Also we don't handle the inconsistency between storefile tracking and 
file system, which
+      // will be handled by a HBCK command
+      LOG.info("Cannot find tracking paths ({}) for store {} in region {} of "
+          + "table {}, fall back to scan the storage to get a list of 
storefiles to be opened"
+        , isEmptyInPersistedFilePaths, storeName, regionName,
+        tableName);
+      return getRegionFs().getStoreFiles(getFamilyName());
+    }
+    ArrayList<StoreFileInfo> storeFiles = new ArrayList<>();
+    for (Path storeFilePath : pathList) {
+      if 
(!StoreFileInfo.isValid(getRegionFs().getFileSystem().getFileStatus(storeFilePath)))
 {
+        LOG.warn("Invalid StoreFile: {}, and archiving it", storeFilePath);

Review comment:
       nit: remove 'and'

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HTableStoreFilePathAccessor.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
+/**
+ * Helper class to interact with the hbase:storefile system table
+ *
+ * <pre>
+ *   ROW-KEY              FAMILY:QUALIFIER      DATA VALUE
+ *   region-store-table   included:files        List&lt;Path&gt; 
filesIncludedInRead
+ * </pre>
+ *
+ * The region encoded name is set as prefix for region split loading balance, 
and we use the
+ * target table name as suffix such that operator can identify the records per 
table.
+ *
+ * included:files is used for persisting storefiles of StoreFileManager in the 
cases of store
+ * opens and store closes. Meanwhile compactedFiles of StoreFileManager isn't 
being tracked
+ * off-memory, because the updated included:files contains compactedFiles and 
the leftover
+ * compactedFiles are either archived when a store closes or opens.
+ *
+ * TODO we will need a followup change to introduce in-memory temporarily 
file, such that further
+ *      we can introduce a non-tracking temporarily storefiles left from a 
flush or compaction when
+ *      a regionserver crashes without closing the store properly
+ */
+
+@InterfaceAudience.Private
+public class HTableStoreFilePathAccessor extends AbstractStoreFilePathAccessor 
{
+
+  public static final byte[] STOREFILE_FAMILY_INCLUDED = 
Bytes.toBytes(STOREFILE_INCLUDED_STR);
+
+  private static final String DASH_SEPARATOR = "-";
+  private static final String STOREFILE_QUALIFIER_STR = "filepaths";
+  private static final byte[] STOREFILE_QUALIFIER = 
Bytes.toBytes(STOREFILE_QUALIFIER_STR);
+  private static final int STOREFILE_TABLE_VERSIONS = 3;
+
+  // TODO find a way for system table to support region split at table 
creation or remove this

Review comment:
       link to a JIRA?




----------------------------------------------------------------
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