You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ta...@apache.org on 2020/12/21 19:08:35 UTC

[hbase] branch HBASE-25249 created (now e5e21fa)

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

taklwu pushed a change to branch HBASE-25249
in repository https://gitbox.apache.org/repos/asf/hbase.git.


      at e5e21fa  HBASE-25249 Adding HStoreContext

This branch includes the following new commits:

     new e5e21fa  HBASE-25249 Adding HStoreContext

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[hbase] 01/01: HBASE-25249 Adding HStoreContext

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e5e21fa09a6869fc3de3179ec1dee077e0019a9a
Author: Abhishek Khanna <ak...@amazon.com>
AuthorDate: Mon Aug 3 23:47:07 2020 -0700

    HBASE-25249 Adding HStoreContext
    
    Adding HStoreContext which contains the metadata about the HStore. This
    meta data can be used across the HFileWriter/Readers and other HStore
    consumers without the need of passing around the complete store and
    exposing its internals.
---
 .../hadoop/hbase/mapreduce/HFileOutputFormat2.java |   5 +-
 .../java/org/apache/hadoop/hbase/mob/MobUtils.java |   5 +-
 .../hadoop/hbase/regionserver/HMobStore.java       |  31 +--
 .../apache/hadoop/hbase/regionserver/HStore.java   | 241 +++++++++++----------
 .../hadoop/hbase/regionserver/HStoreContext.java   | 174 +++++++++++++++
 .../hadoop/hbase/regionserver/StoreUtils.java      |  26 +++
 .../hadoop/hbase/tool/BulkLoadHFilesTool.java      |   5 +-
 .../hadoop/hbase/tool/LoadIncrementalHFiles.java   |   0
 .../wal/BoundedRecoveredHFilesOutputSink.java      |   6 +-
 .../hadoop/hbase/regionserver/TestHRegion.java     |   2 +-
 .../regionserver/TestSecureBulkLoadManager.java    |   4 +-
 11 files changed, 358 insertions(+), 141 deletions(-)

diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
index 75b5246..ee6d533 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
+import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
@@ -369,8 +370,8 @@ public class HFileOutputFormat2
         encoding = encoding == null ? datablockEncodingMap.get(tableAndFamily) : encoding;
         encoding = encoding == null ? DataBlockEncoding.NONE : encoding;
         HFileContextBuilder contextBuilder = new HFileContextBuilder().withCompression(compression)
-            .withDataBlockEncoding(encoding).withChecksumType(HStore.getChecksumType(conf))
-            .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf)).withBlockSize(blockSize)
+            .withDataBlockEncoding(encoding).withChecksumType(StoreUtils.getChecksumType(conf))
+            .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf)).withBlockSize(blockSize)
             .withColumnFamily(family).withTableName(tableName);
 
         if (HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
index acc8f74..4ef1b90 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
+import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
@@ -540,8 +541,8 @@ public final class MobUtils {
       Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext,
       boolean isCompaction) throws IOException {
     return createWriter(conf, fs, family, new Path(basePath, mobFileName.getFileName()),
-      maxKeyCount, compression, cacheConfig, cryptoContext, HStore.getChecksumType(conf),
-      HStore.getBytesPerChecksum(conf), family.getBlocksize(), BloomType.NONE, isCompaction);
+      maxKeyCount, compression, cacheConfig, cryptoContext, StoreUtils.getChecksumType(conf),
+      StoreUtils.getBytesPerChecksum(conf), family.getBlocksize(), BloomType.NONE, isCompaction);
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
index 5960b80..0f2b148 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
@@ -93,7 +93,6 @@ public class HMobStore extends HStore {
   private AtomicLong mobFlushedCellsSize = new AtomicLong();
   private AtomicLong mobScanCellsCount = new AtomicLong();
   private AtomicLong mobScanCellsSize = new AtomicLong();
-  private ColumnFamilyDescriptor family;
   private Map<TableName, List<Path>> map = new ConcurrentHashMap<>();
   private final IdLock keyLock = new IdLock();
   // When we add a MOB reference cell to the HFile, we will add 2 tags along with it
@@ -107,16 +106,15 @@ public class HMobStore extends HStore {
   public HMobStore(final HRegion region, final ColumnFamilyDescriptor family,
       final Configuration confParam, boolean warmup) throws IOException {
     super(region, family, confParam, warmup);
-    this.family = family;
     this.mobFileCache = region.getMobFileCache();
     this.homePath = MobUtils.getMobHome(conf);
     this.mobFamilyPath = MobUtils.getMobFamilyPath(conf, this.getTableName(),
-        family.getNameAsString());
+        getColumnFamilyName());
     List<Path> locations = new ArrayList<>(2);
     locations.add(mobFamilyPath);
     TableName tn = region.getTableDescriptor().getTableName();
     locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils.getMobRegionInfo(tn)
-        .getEncodedName(), family.getNameAsString()));
+        .getEncodedName(), getColumnFamilyName()));
     map.put(tn, locations);
     List<Tag> tags = new ArrayList<>(2);
     tags.add(MobConstants.MOB_REF_TAG);
@@ -209,7 +207,7 @@ public class HMobStore extends HStore {
       Compression.Algorithm compression, byte[] startKey,
       boolean isCompaction) throws IOException {
     MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID()
-        .toString().replaceAll("-", ""),  region.getRegionInfo().getEncodedName());
+        .toString().replaceAll("-", ""),  getHRegion().getRegionInfo().getEncodedName());
     return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression, isCompaction);
   }
 
@@ -226,9 +224,10 @@ public class HMobStore extends HStore {
   public StoreFileWriter createWriterInTmp(MobFileName mobFileName, Path basePath,
       long maxKeyCount, Compression.Algorithm compression,
       boolean isCompaction) throws IOException {
-    return MobUtils.createWriter(conf, region.getFilesystem(), family,
-      new Path(basePath, mobFileName.getFileName()), maxKeyCount, compression, cacheConf,
-      cryptoContext, checksumType, bytesPerChecksum, blocksize, BloomType.NONE, isCompaction);
+    return MobUtils.createWriter(conf, getFileSystem(), getColumnFamilyDescriptor(),
+      new Path(basePath, mobFileName.getFileName()), maxKeyCount, compression, getCacheConfig(),
+        getCryptoContext(), StoreUtils.getChecksumType(conf), StoreUtils.getBytesPerChecksum(conf),
+        blocksize, BloomType.NONE, isCompaction);
   }
 
   /**
@@ -245,10 +244,10 @@ public class HMobStore extends HStore {
     validateMobFile(sourceFile);
     LOG.info(" FLUSH Renaming flushed file from {} to {}", sourceFile, dstPath);
     Path parent = dstPath.getParent();
-    if (!region.getFilesystem().exists(parent)) {
-      region.getFilesystem().mkdirs(parent);
+    if (!getFileSystem().exists(parent)) {
+      getFileSystem().mkdirs(parent);
     }
-    if (!region.getFilesystem().rename(sourceFile, dstPath)) {
+    if (!getFileSystem().rename(sourceFile, dstPath)) {
       throw new IOException("Failed rename of " + sourceFile + " to " + dstPath);
     }
   }
@@ -261,7 +260,7 @@ public class HMobStore extends HStore {
   private void validateMobFile(Path path) throws IOException {
     HStoreFile storeFile = null;
     try {
-      storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.cacheConf,
+      storeFile = new HStoreFile(getFileSystem(), path, conf, getCacheConfig(),
           BloomType.NONE, isPrimaryReplicaStore());
       storeFile.initReader();
     } catch (IOException e) {
@@ -352,9 +351,11 @@ public class HMobStore extends HStore {
         locations = map.get(tableName);
         if (locations == null) {
           locations = new ArrayList<>(2);
-          locations.add(MobUtils.getMobFamilyPath(conf, tableName, family.getNameAsString()));
+          locations.add(MobUtils.getMobFamilyPath(conf, tableName, getColumnFamilyDescriptor()
+            .getNameAsString()));
           locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tableName,
-            MobUtils.getMobRegionInfo(tableName).getEncodedName(), family.getNameAsString()));
+            MobUtils.getMobRegionInfo(tableName).getEncodedName(), getColumnFamilyDescriptor()
+              .getNameAsString()));
           map.put(tableName, locations);
         }
       } finally {
@@ -388,7 +389,7 @@ public class HMobStore extends HStore {
       MobFile file = null;
       Path path = new Path(location, fileName);
       try {
-        file = mobFileCache.openFile(fs, path, cacheConf);
+        file = mobFileCache.openFile(fs, path, getCacheConfig());
         return readPt != -1 ? file.readCell(search, cacheMobBlocks, readPt)
             : file.readCell(search, cacheMobBlocks);
       } catch (IOException e) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 3a71c23..1cd5032 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -97,7 +97,6 @@ import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -157,11 +156,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
   protected final MemStore memstore;
   // This stores directory in the filesystem.
-  protected final HRegion region;
-  private final ColumnFamilyDescriptor family;
-  private final HRegionFileSystem fs;
+  private final HRegion region;
   protected Configuration conf;
-  protected CacheConfig cacheConf;
   private long lastCompactSize = 0;
   volatile boolean forceMajor = false;
   private AtomicLong storeSize = new AtomicLong();
@@ -218,13 +214,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   protected final int blocksize;
   private HFileDataBlockEncoder dataBlockEncoder;
 
-  /** Checksum configuration */
-  protected ChecksumType checksumType;
-  protected int bytesPerChecksum;
-
-  // Comparing KeyValues
-  protected final CellComparator comparator;
-
   final StoreEngine<?, ?, ?, ?> storeEngine;
 
   private static final AtomicBoolean offPeakCompactionTracker = new AtomicBoolean();
@@ -236,7 +225,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
   private long blockingFileCount;
   private int compactionCheckMultiplier;
-  protected Encryption.Context cryptoContext = Encryption.Context.NONE;
 
   private AtomicLong flushedCellsCount = new AtomicLong();
   private AtomicLong compactedCellsCount = new AtomicLong();
@@ -246,6 +234,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   private AtomicLong compactedCellsSize = new AtomicLong();
   private AtomicLong majorCompactedCellsSize = new AtomicLong();
 
+  private HStoreContext storeContext;
+
   /**
    * Constructor
    * @param family HColumnDescriptor for this column
@@ -254,48 +244,47 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   protected HStore(final HRegion region, final ColumnFamilyDescriptor family,
       final Configuration confParam, boolean warmup) throws IOException {
 
-    this.fs = region.getRegionFileSystem();
-
-    // Assemble the store's home directory and Ensure it exists.
-    fs.createStoreDir(family.getNameAsString());
-    this.region = region;
-    this.family = family;
     // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
     // CompoundConfiguration will look for keys in reverse order of addition, so we'd
     // add global config first, then table and cf overrides, then cf metadata.
     this.conf = new CompoundConfiguration()
-      .add(confParam)
-      .addBytesMap(region.getTableDescriptor().getValues())
-      .addStringMap(family.getConfiguration())
-      .addBytesMap(family.getValues());
-    this.blocksize = family.getBlocksize();
+        .add(confParam)
+        .addBytesMap(region.getTableDescriptor().getValues())
+        .addStringMap(family.getConfiguration())
+        .addBytesMap(family.getValues());
+
+    this.region = region;
+    this.storeContext = initializeStoreContext(family);
+
+    // Assemble the store's home directory and Ensure it exists.
+    getRegionFileSystem().createStoreDir(getColumnFamilyName());
+
+    this.blocksize = getColumnFamilyDescriptor().getBlocksize();
 
     // set block storage policy for store directory
-    String policyName = family.getStoragePolicy();
+    String policyName = getColumnFamilyDescriptor().getStoragePolicy();
     if (null == policyName) {
       policyName = this.conf.get(BLOCK_STORAGE_POLICY_KEY, DEFAULT_BLOCK_STORAGE_POLICY);
     }
-    this.fs.setStoragePolicy(family.getNameAsString(), policyName.trim());
+    getRegionFileSystem().setStoragePolicy(getColumnFamilyName(), policyName.trim());
 
-    this.dataBlockEncoder = new HFileDataBlockEncoderImpl(family.getDataBlockEncoding());
+    this.dataBlockEncoder = new HFileDataBlockEncoderImpl(getColumnFamilyDescriptor()
+        .getDataBlockEncoding());
 
-    this.comparator = region.getCellComparator();
     // used by ScanQueryMatcher
     long timeToPurgeDeletes =
         Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
     LOG.trace("Time to purge deletes set to {}ms in {}", timeToPurgeDeletes, this);
     // Get TTL
-    long ttl = determineTTLFromFamily(family);
+    long ttl = determineTTLFromFamily(getColumnFamilyDescriptor());
     // Why not just pass a HColumnDescriptor in here altogether?  Even if have
     // to clone it?
-    scanInfo = new ScanInfo(conf, family, ttl, timeToPurgeDeletes, this.comparator);
+    scanInfo = new ScanInfo(conf, getColumnFamilyDescriptor(), ttl, timeToPurgeDeletes,
+        getComparator());
     this.memstore = getMemstore();
 
     this.offPeakHours = OffPeakHours.getInstance(conf);
 
-    // Setting up cache configuration for this family
-    createCacheConf(family);
-
     this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false);
 
     this.blockingFileCount =
@@ -308,7 +297,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       this.compactionCheckMultiplier = DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER;
     }
 
-    this.storeEngine = createStoreEngine(this, this.conf, this.comparator);
+    this.storeEngine = createStoreEngine(this, this.conf, getComparator());
     List<HStoreFile> hStoreFiles = loadStoreFiles(warmup);
     // Move the storeSize calculation out of loadStoreFiles() method, because the secondary read
     // replica's refreshStoreFiles() will also use loadStoreFiles() to refresh its store files and
@@ -318,10 +307,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     this.totalUncompressedBytes.addAndGet(getTotalUncompressedBytes(hStoreFiles));
     this.storeEngine.getStoreFileManager().loadFiles(hStoreFiles);
 
-    // Initialize checksum type from name. The names are CRC32, CRC32C, etc.
-    this.checksumType = getChecksumType(conf);
-    // Initialize bytes per checksum
-    this.bytesPerChecksum = getBytesPerChecksum(conf);
     flushRetriesNumber = conf.getInt(
         "hbase.hstore.flush.retries.number", DEFAULT_FLUSH_RETRIES_NUMBER);
     pauseTime = conf.getInt(HConstants.HBASE_SERVER_PAUSE, HConstants.DEFAULT_HBASE_SERVER_PAUSE);
@@ -330,7 +315,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
           "hbase.hstore.flush.retries.number must be > 0, not "
               + flushRetriesNumber);
     }
-    cryptoContext = EncryptionUtil.createEncryptionContext(conf, family);
 
     int confPrintThreshold =
         this.conf.getInt("hbase.region.store.parallel.put.print.threshold", 50);
@@ -347,6 +331,48 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     cacheOnWriteLogged = false;
   }
 
+  private HStoreContext initializeStoreContext(ColumnFamilyDescriptor family) throws IOException {
+    return new HStoreContext.Builder()
+        .withBloomType(family.getBloomFilterType())
+        .withCacheConfig(createCacheConf(family))
+        .withCellComparator(region.getCellComparator())
+        .withColumnFamilyDescriptor(family)
+        .withCompactedFilesSupplier(this::getCompactedFiles)
+        .withRegionFileSystem(region.getRegionFileSystem())
+        .withDefaultHFileContext(getDefaultHFileContext(family))
+        .withFavoredNodesSupplier(this::getFavoredNodes)
+        .withFamilyStoreDirectoryPath(region.getRegionFileSystem()
+            .getStoreDir(family.getNameAsString()))
+        .withRegionCoprocessorHost(region.getCoprocessorHost())
+        .build();
+  }
+
+  private InetSocketAddress[] getFavoredNodes() {
+    InetSocketAddress[] favoredNodes = null;
+    if (region.getRegionServerServices() != null) {
+      favoredNodes = region.getRegionServerServices().getFavoredNodesForRegion(
+          region.getRegionInfo().getEncodedName());
+    }
+    return favoredNodes;
+  }
+
+  private HFileContext getDefaultHFileContext(ColumnFamilyDescriptor family) throws IOException {
+    HFileContext hFileContext = new HFileContextBuilder()
+        .withCompression(HFile.DEFAULT_COMPRESSION_ALGORITHM)
+        .withCompressTags(family.isCompressTags())
+        .withChecksumType(StoreUtils.getChecksumType(conf))
+        .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf))
+        .withBlockSize(family.getBlocksize())
+        .withHBaseCheckSum(true)
+        .withDataBlockEncoding(family.getDataBlockEncoding())
+        .withEncryptionContext(EncryptionUtil.createEncryptionContext(conf, family))
+        .withColumnFamily(family.getName())
+        .withTableName(region.getTableDescriptor().getTableName().getName())
+        .withCellComparator(region.getCellComparator())
+        .build();
+    return hFileContext;
+  }
+
   /**
    * @return MemStore Instance to use in this store.
    */
@@ -358,7 +384,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       inMemoryCompaction = MemoryCompactionPolicy.valueOf(
           conf.get("hbase.systemtables.compacting.memstore.type", "NONE"));
     } else {
-      inMemoryCompaction = family.getInMemoryCompaction();
+      inMemoryCompaction = getColumnFamilyDescriptor().getInMemoryCompaction();
     }
     if (inMemoryCompaction == null) {
       inMemoryCompaction =
@@ -368,13 +394,13 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     switch (inMemoryCompaction) {
       case NONE:
         ms = ReflectionUtils.newInstance(DefaultMemStore.class,
-            new Object[] { conf, this.comparator,
+            new Object[] { conf, getComparator(),
                 this.getHRegion().getRegionServicesForStores()});
         break;
       default:
         Class<? extends CompactingMemStore> clz = conf.getClass(MEMSTORE_CLASS_NAME,
             CompactingMemStore.class, CompactingMemStore.class);
-        ms = ReflectionUtils.newInstance(clz, new Object[]{conf, this.comparator, this,
+        ms = ReflectionUtils.newInstance(clz, new Object[]{conf, getComparator(), this,
             this.getHRegion().getRegionServicesForStores(), inMemoryCompaction});
     }
     return ms;
@@ -384,10 +410,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    * Creates the cache config.
    * @param family The current column family.
    */
-  protected void createCacheConf(final ColumnFamilyDescriptor family) {
-    this.cacheConf = new CacheConfig(conf, family, region.getBlockCache(),
+  protected CacheConfig createCacheConf(final ColumnFamilyDescriptor family) {
+    CacheConfig cacheConf = new CacheConfig(conf, family, region.getBlockCache(),
         region.getRegionServicesForStores().getByteBuffAllocator());
-    LOG.info("Created cacheConfig: " + this.getCacheConfig() + " for " + this);
+    LOG.info("Created cacheConfig: " + cacheConf + " for " + this);
+    return cacheConf;
   }
 
   /**
@@ -400,7 +427,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    */
   protected StoreEngine<?, ?, ?, ?> createStoreEngine(HStore store, Configuration conf,
       CellComparator kvComparator) throws IOException {
-    return StoreEngine.create(store, conf, comparator);
+    return StoreEngine.create(store, conf, getComparator());
   }
 
   /**
@@ -423,7 +450,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
   @Override
   public String getColumnFamilyName() {
-    return this.family.getNameAsString();
+    return this.storeContext.getFamily().getNameAsString();
   }
 
   @Override
@@ -433,11 +460,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
   @Override
   public FileSystem getFileSystem() {
-    return this.fs.getFileSystem();
+    return storeContext.getRegionFileSystem().getFileSystem();
   }
 
   public HRegionFileSystem getRegionFileSystem() {
-    return this.fs;
+    return storeContext.getRegionFileSystem();
   }
 
   /* Implementation of StoreConfigInformation */
@@ -474,33 +501,14 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   }
   /* End implementation of StoreConfigInformation */
 
-  /**
-   * Returns the configured bytesPerChecksum value.
-   * @param conf The configuration
-   * @return The bytesPerChecksum that is set in the configuration
-   */
-  public static int getBytesPerChecksum(Configuration conf) {
-    return conf.getInt(HConstants.BYTES_PER_CHECKSUM,
-                       HFile.DEFAULT_BYTES_PER_CHECKSUM);
-  }
-
-  /**
-   * Returns the configured checksum algorithm.
-   * @param conf The configuration
-   * @return The checksum algorithm that is set in the configuration
-   */
-  public static ChecksumType getChecksumType(Configuration conf) {
-    String checksumName = conf.get(HConstants.CHECKSUM_TYPE_NAME);
-    if (checksumName == null) {
-      return ChecksumType.getDefaultChecksumType();
-    } else {
-      return ChecksumType.nameToType(checksumName);
-    }
-  }
 
   @Override
   public ColumnFamilyDescriptor getColumnFamilyDescriptor() {
-    return this.family;
+    return this.storeContext.getFamily();
+  }
+
+  public Encryption.Context getCryptoContext() {
+    return storeContext.getDefaultFileContext().getEncryptionContext();
   }
 
   @Override
@@ -559,7 +567,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    * from the given directory.
    */
   private List<HStoreFile> loadStoreFiles(boolean warmup) throws IOException {
-    Collection<StoreFileInfo> files = fs.getStoreFiles(getColumnFamilyName());
+    Collection<StoreFileInfo> files = getRegionFileSystem().getStoreFiles(getColumnFamilyName());
     return openStoreFiles(files, warmup);
   }
 
@@ -610,7 +618,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     if (ioe != null) {
       // close StoreFile readers
       boolean evictOnClose =
-          cacheConf != null? cacheConf.shouldEvictOnClose(): true;
+          getCacheConfig() != null? getCacheConfig().shouldEvictOnClose(): true;
       for (HStoreFile file : results) {
         try {
           if (file != null) {
@@ -638,7 +646,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       results.removeAll(filesToRemove);
       if (!filesToRemove.isEmpty() && this.isPrimaryReplicaStore()) {
         LOG.debug("Moving the files {} to archive", filesToRemove);
-        this.fs.removeStoreFiles(this.getColumnFamilyDescriptor().getNameAsString(), filesToRemove);
+        getRegionFileSystem().removeStoreFiles(this.getColumnFamilyDescriptor().getNameAsString(),
+            filesToRemove);
       }
     }
 
@@ -647,7 +656,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
   @Override
   public void refreshStoreFiles() throws IOException {
-    Collection<StoreFileInfo> newFiles = fs.getStoreFiles(getColumnFamilyName());
+    Collection<StoreFileInfo> newFiles = getRegionFileSystem().getStoreFiles(getColumnFamilyName());
     refreshStoreFilesInternal(newFiles);
   }
 
@@ -658,7 +667,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   public void refreshStoreFiles(Collection<String> newFiles) throws IOException {
     List<StoreFileInfo> storeFiles = new ArrayList<>(newFiles.size());
     for (String file : newFiles) {
-      storeFiles.add(fs.getStoreFileInfo(getColumnFamilyName(), file));
+      storeFiles.add(getRegionFileSystem().getStoreFileInfo(getColumnFamilyName(), file));
     }
     refreshStoreFilesInternal(storeFiles);
   }
@@ -735,7 +744,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
   private HStoreFile createStoreFileAndReader(StoreFileInfo info) throws IOException {
     info.setRegionCoprocessorHost(this.region.getCoprocessorHost());
-    HStoreFile storeFile = new HStoreFile(info, this.family.getBloomFilterType(), this.cacheConf);
+    HStoreFile storeFile = new HStoreFile(info, getColumnFamilyDescriptor().getBloomFilterType(),
+            getCacheConfig());
     storeFile.initReader();
     return storeFile;
   }
@@ -818,7 +828,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       LOG.info("Validating hfile at " + srcPath + " for inclusion in " + this);
       FileSystem srcFs = srcPath.getFileSystem(conf);
       srcFs.access(srcPath, FsAction.READ_WRITE);
-      reader = HFile.createReader(srcFs, srcPath, cacheConf, isPrimaryReplicaStore(), conf);
+      reader = HFile.createReader(srcFs, srcPath, getCacheConfig(), isPrimaryReplicaStore(), conf);
 
       Optional<byte[]> firstKey = reader.getFirstRowKey();
       Preconditions.checkState(firstKey.isPresent(), "First key can not be null");
@@ -855,7 +865,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         do {
           Cell cell = scanner.getCell();
           if (prevCell != null) {
-            if (comparator.compareRows(prevCell, cell) > 0) {
+            if (getComparator().compareRows(prevCell, cell) > 0) {
               throw new InvalidHFileException("Previous row is greater than"
                   + " current row: path=" + srcPath + " previous="
                   + CellUtil.getCellKeyAsString(prevCell) + " current="
@@ -892,13 +902,13 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    */
   public Pair<Path, Path> preBulkLoadHFile(String srcPathStr, long seqNum) throws IOException {
     Path srcPath = new Path(srcPathStr);
-    return fs.bulkLoadStoreFile(getColumnFamilyName(), srcPath, seqNum);
+    return getRegionFileSystem().bulkLoadStoreFile(getColumnFamilyName(), srcPath, seqNum);
   }
 
   public Path bulkLoadHFile(byte[] family, String srcPathStr, Path dstPath) throws IOException {
     Path srcPath = new Path(srcPathStr);
     try {
-      fs.commitStoreFile(srcPath, dstPath);
+      getRegionFileSystem().commitStoreFile(srcPath, dstPath);
     } finally {
       if (this.getCoprocessorHost() != null) {
         this.getCoprocessorHost().postCommitStoreFile(family, srcPath, dstPath);
@@ -964,8 +974,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
           storeEngine.getStoreFileManager().clearCompactedFiles();
       // clear the compacted files
       if (CollectionUtils.isNotEmpty(compactedfiles)) {
-        removeCompactedfiles(compactedfiles, cacheConf != null ?
-          cacheConf.shouldEvictOnClose() : true);
+        removeCompactedfiles(compactedfiles, getCacheConfig() != null ?
+            getCacheConfig().shouldEvictOnClose() : true);
       }
       if (!result.isEmpty()) {
         // initialize the thread pool for closing store files in parallel.
@@ -981,7 +991,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
             @Override
             public Void call() throws IOException {
               boolean evictOnClose =
-                  cacheConf != null? cacheConf.shouldEvictOnClose(): true;
+                  getCacheConfig() != null? getCacheConfig().shouldEvictOnClose(): true;
               f.closeStoreFile(evictOnClose);
               return null;
             }
@@ -1092,7 +1102,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     FileSystem srcFs = path.getFileSystem(conf);
     srcFs.access(path, FsAction.READ_WRITE);
     try (HFile.Reader reader =
-        HFile.createReader(srcFs, path, cacheConf, isPrimaryReplicaStore(), conf)) {
+        HFile.createReader(srcFs, path, getCacheConfig(), isPrimaryReplicaStore(), conf)) {
       Optional<byte[]> firstKey = reader.getFirstRowKey();
       Preconditions.checkState(firstKey.isPresent(), "First key can not be null");
       Optional<Cell> lk = reader.getLastKey();
@@ -1104,7 +1114,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       }
     }
 
-    Path dstPath = fs.commitStoreFile(getColumnFamilyName(), path);
+    Path dstPath = getRegionFileSystem().commitStoreFile(getColumnFamilyName(), path);
     HStoreFile sf = createStoreFileAndReader(dstPath);
     StoreFileReader r = sf.getReader();
     this.storeSize.addAndGet(r.length());
@@ -1129,7 +1139,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   private HStoreFile commitFile(Path path, long logCacheFlushId, MonitoredTask status)
       throws IOException {
     // Write-out finished successfully, move into the right spot
-    Path dstPath = fs.commitStoreFile(getColumnFamilyName(), path);
+    Path dstPath = getRegionFileSystem().commitStoreFile(getColumnFamilyName(), path);
 
     status.setStatus("Flushing " + this + ": reopening flushed file");
     HStoreFile sf = createStoreFileAndReader(dstPath);
@@ -1167,12 +1177,13 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       boolean shouldDropBehind, long totalCompactedFilesSize, String fileStoragePolicy)
         throws IOException {
     // creating new cache config for each new writer
+    final CacheConfig cacheConf = getCacheConfig();
     final CacheConfig writerCacheConf = new CacheConfig(cacheConf);
     if (isCompaction) {
       // Don't cache data on write on compactions, unless specifically configured to do so
       // Cache only when total file size remains lower than configured threshold
       final boolean cacheCompactedBlocksOnWrite =
-        cacheConf.shouldCacheCompactedBlocksOnWrite();
+        getCacheConfig().shouldCacheCompactedBlocksOnWrite();
       // if data blocks are to be cached on write
       // during compaction, we should forcefully
       // cache index and bloom blocks as well
@@ -1212,12 +1223,12 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
           region.getRegionInfo().getEncodedName());
     }
     HFileContext hFileContext = createFileContext(compression, includeMVCCReadpoint, includesTag,
-      cryptoContext);
-    Path familyTempDir = new Path(fs.getTempDir(), family.getNameAsString());
+      getCryptoContext());
+    Path familyTempDir = new Path(getRegionFileSystem().getTempDir(), getColumnFamilyName());
     StoreFileWriter.Builder builder = new StoreFileWriter.Builder(conf, writerCacheConf,
         this.getFileSystem())
             .withOutputDir(familyTempDir)
-            .withBloomType(family.getBloomFilterType())
+            .withBloomType(storeContext.getBloomFilterType())
             .withMaxKeyCount(maxKeyCount)
             .withFavoredNodes(favoredNodes)
             .withFileContext(hFileContext)
@@ -1236,18 +1247,19 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
                                 .withIncludesMvcc(includeMVCCReadpoint)
                                 .withIncludesTags(includesTag)
                                 .withCompression(compression)
-                                .withCompressTags(family.isCompressTags())
-                                .withChecksumType(checksumType)
-                                .withBytesPerCheckSum(bytesPerChecksum)
+                                .withCompressTags(getColumnFamilyDescriptor().isCompressTags())
+                                .withChecksumType(StoreUtils.getChecksumType(conf))
+                                .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf))
                                 .withBlockSize(blocksize)
                                 .withHBaseCheckSum(true)
-                                .withDataBlockEncoding(family.getDataBlockEncoding())
+                                .withDataBlockEncoding(getColumnFamilyDescriptor()
+                                      .getDataBlockEncoding())
                                 .withEncryptionContext(cryptoContext)
                                 .withCreateTime(EnvironmentEdgeManager.currentTime())
-                                .withColumnFamily(family.getName())
+                                .withColumnFamily(getColumnFamilyDescriptor().getName())
                                 .withTableName(region.getTableDescriptor()
                                     .getTableName().getName())
-                                .withCellComparator(this.comparator)
+                                .withCellComparator(getComparator())
                                 .build();
     return hFileContext;
   }
@@ -1529,7 +1541,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
       // Ready to go. Have list of files to compact.
       LOG.info("Starting compaction of " + filesToCompact +
-        " into tmpdir=" + fs.getTempDir() + ", totalSize=" +
+        " into tmpdir=" + getRegionFileSystem().getTempDir() + ", totalSize=" +
           TraditionalBinaryPrefix.long2String(cr.getSize(), "", 1));
 
       return doCompaction(cr, filesToCompact, user, compactionStartTime,
@@ -1579,7 +1591,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     String prefix = HConstants.STORAGE_POLICY_PREFIX;
     for (Path newFile : newFiles) {
       if (newFile.getParent().getName().startsWith(prefix)) {
-        CommonFSUtils.setStoragePolicy(fs.getFileSystem(), newFile,
+        CommonFSUtils.setStoragePolicy(getRegionFileSystem().getFileSystem(), newFile,
             newFile.getParent().getName().substring(prefix.length()));
       }
     }
@@ -1604,7 +1616,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   HStoreFile moveFileIntoPlace(Path newFile) throws IOException {
     validateStoreFile(newFile);
     // Move the file into the right spot
-    Path destPath = fs.commitStoreFile(getColumnFamilyName(), newFile);
+    Path destPath = getRegionFileSystem().commitStoreFile(getColumnFamilyName(), newFile);
     return createStoreFileAndReader(destPath);
   }
 
@@ -1624,8 +1636,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         newFiles.stream().map(HStoreFile::getPath).collect(Collectors.toList());
     RegionInfo info = this.region.getRegionInfo();
     CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info,
-        family.getName(), inputPaths, outputPaths,
-      fs.getStoreDir(getColumnFamilyDescriptor().getNameAsString()));
+        getColumnFamilyDescriptor().getName(), inputPaths, outputPaths,
+        getRegionFileSystem().getStoreDir(getColumnFamilyDescriptor().getNameAsString()));
     // Fix reaching into Region to get the maxWaitForSeqId.
     // Does this method belong in Region altogether given it is making so many references up there?
     // Could be Region#writeCompactionMarker(compactionDescriptor);
@@ -1752,7 +1764,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     String familyName = this.getColumnFamilyName();
     Set<String> inputFiles = new HashSet<>();
     for (String compactionInput : compactionInputs) {
-      Path inputPath = fs.getStoreFilePath(familyName, compactionInput);
+      Path inputPath = getRegionFileSystem().getStoreFilePath(familyName, compactionInput);
       inputFiles.add(inputPath.getName());
     }
 
@@ -1772,7 +1784,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         compactionOutputs.remove(sf.getPath().getName());
       }
       for (String compactionOutput : compactionOutputs) {
-        StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), compactionOutput);
+        StoreFileInfo storeFileInfo =
+            getRegionFileSystem().getStoreFileInfo(getColumnFamilyName(), compactionOutput);
         HStoreFile storeFile = createStoreFileAndReader(storeFileInfo);
         outputStoreFiles.add(storeFile);
       }
@@ -2092,7 +2105,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       throw new IllegalArgumentException("Number of versions must be > 0");
     }
     // Make sure we do not return more than maximum versions for this store.
-    int maxVersions = this.family.getMaxVersions();
+    int maxVersions = getColumnFamilyDescriptor().getMaxVersions();
     return wantedVersions > maxVersions ? maxVersions: wantedVersions;
   }
 
@@ -2367,7 +2380,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
   @Override
   public RegionInfo getRegionInfo() {
-    return this.fs.getRegionInfo();
+    return getRegionFileSystem().getRegionInfo();
   }
 
   @Override
@@ -2509,7 +2522,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       List<HStoreFile> storeFiles = new ArrayList<>(fileNames.size());
       for (String file : fileNames) {
         // open the file as a store file (hfile link, etc)
-        StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), file);
+        StoreFileInfo storeFileInfo = getRegionFileSystem().getStoreFileInfo(getColumnFamilyName(), file);
         HStoreFile storeFile = createStoreFileAndReader(storeFileInfo);
         storeFiles.add(storeFile);
         HStore.this.storeSize.addAndGet(storeFile.getReader().length());
@@ -2559,7 +2572,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    * @return cache configuration for this Store.
    */
   public CacheConfig getCacheConfig() {
-    return this.cacheConf;
+    return storeContext.getCacheConf();
   }
 
   public static final long FIXED_OVERHEAD = ClassSize.estimateBase(HStore.class, false);
@@ -2578,7 +2591,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
   @Override
   public CellComparator getComparator() {
-    return comparator;
+    return storeContext.getComparator();
   }
 
   public ScanInfo getScanInfo() {
@@ -2652,7 +2665,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   public void onConfigurationChange(Configuration conf) {
     this.conf = new CompoundConfiguration()
             .add(conf)
-            .addBytesMap(family.getValues());
+            .addBytesMap(getColumnFamilyDescriptor().getValues());
     this.storeEngine.compactionPolicy.setConf(conf);
     this.offPeakHours = OffPeakHours.getInstance(conf);
   }
@@ -2784,8 +2797,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         LOG.debug("Moving the files {} to archive", filesToRemove);
         // Only if this is successful it has to be removed
         try {
-          this.fs.removeStoreFiles(this.getColumnFamilyDescriptor().getNameAsString(),
-            filesToRemove);
+          getRegionFileSystem().removeStoreFiles(this.getColumnFamilyDescriptor().getNameAsString(),
+  filesToRemove);
         } catch (FailedArchiveException fae) {
           // Even if archiving some files failed, we still need to clear out any of the
           // files which were successfully archived.  Otherwise we will receive a
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreContext.java
new file mode 100644
index 0000000..4a0c7af
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreContext.java
@@ -0,0 +1,174 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFileContext;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.function.Supplier;
+
+/**
+ * This carries the information on some of the meta data about the HStore. This
+ * meta data can be used across the HFileWriter/Readers and other HStore consumers without the
+ * need of passing around the complete store.
+ */
+@InterfaceAudience.Private
+public class HStoreContext {
+  private final HFileContext defaultFileContext;
+  private final CacheConfig cacheConf;
+  private final HRegionFileSystem regionFileSystem;
+  private final CellComparator comparator;
+  private final BloomType bloomFilterType;
+  private final Supplier<Collection<HStoreFile>> compactedFilesSupplier;
+  private final Supplier<InetSocketAddress[]> favoredNodesSupplier;
+  private final ColumnFamilyDescriptor family;
+  private final Path familyStoreDirectoryPath;
+  private final RegionCoprocessorHost coprocessorHost;
+
+  private HStoreContext(Builder builder) {
+    this.defaultFileContext = builder.defaultFileContext;
+    this.cacheConf = builder.cacheConf;
+    this.regionFileSystem = builder.regionFileSystem;
+    this.comparator = builder.comparator;
+    this.bloomFilterType = builder.bloomFilterType;
+    this.compactedFilesSupplier = builder.compactedFilesSupplier;
+    this.favoredNodesSupplier = builder.favoredNodesSupplier;
+    this.family = builder.family;
+    this.familyStoreDirectoryPath = builder.familyStoreDirectoryPath;
+    this.coprocessorHost = builder.coprocessorHost;
+  }
+
+  public HFileContext getDefaultFileContext() {
+    return defaultFileContext;
+  }
+
+  public CacheConfig getCacheConf() {
+    return cacheConf;
+  }
+
+  public HRegionFileSystem getRegionFileSystem() {
+    return regionFileSystem;
+  }
+
+  public CellComparator getComparator() {
+    return comparator;
+  }
+
+  public BloomType getBloomFilterType() {
+    return bloomFilterType;
+  }
+
+  public Supplier<Collection<HStoreFile>> getCompactedFilesSupplier() {
+    return compactedFilesSupplier;
+  }
+
+  public Supplier<InetSocketAddress[]> getFavoredNodesSupplier() {
+    return favoredNodesSupplier;
+  }
+
+  public ColumnFamilyDescriptor getFamily() {
+    return family;
+  }
+
+  public Path getFamilyStoreDirectoryPath() {
+    return familyStoreDirectoryPath;
+  }
+
+  public RegionCoprocessorHost getCoprocessorHost() {
+    return coprocessorHost;
+  }
+
+  public static Builder getBuilder() {
+    return new Builder();
+  }
+
+  public static class Builder {
+    private HFileContext defaultFileContext;
+    private CacheConfig cacheConf;
+    private HRegionFileSystem regionFileSystem;
+    private CellComparator comparator;
+    private BloomType bloomFilterType;
+    private Supplier<Collection<HStoreFile>> compactedFilesSupplier;
+    private Supplier<InetSocketAddress[]> favoredNodesSupplier;
+    private ColumnFamilyDescriptor family;
+    private Path familyStoreDirectoryPath;
+    private RegionCoprocessorHost coprocessorHost;
+
+    public Builder withDefaultHFileContext(HFileContext defaultFileContext) {
+      this.defaultFileContext = defaultFileContext;
+      return this;
+    }
+
+    public Builder withCacheConfig(CacheConfig cacheConf) {
+      this.cacheConf = cacheConf;
+      return this;
+    }
+
+    public Builder withRegionFileSystem(HRegionFileSystem regionFileSystem) {
+      this.regionFileSystem = regionFileSystem;
+      return this;
+    }
+
+    public Builder withCellComparator(CellComparator comparator) {
+      this.comparator = comparator;
+      return this;
+    }
+
+    public Builder withBloomType(BloomType bloomFilterType) {
+      this.bloomFilterType = bloomFilterType;
+      return this;
+    }
+
+    public Builder withCompactedFilesSupplier(Supplier<Collection<HStoreFile>>
+        compactedFilesSupplier) {
+      this.compactedFilesSupplier = compactedFilesSupplier;
+      return this;
+    }
+
+    public Builder withFavoredNodesSupplier(Supplier<InetSocketAddress[]> favoredNodesSupplier) {
+      this.favoredNodesSupplier = favoredNodesSupplier;
+      return this;
+    }
+
+    public Builder withColumnFamilyDescriptor(ColumnFamilyDescriptor family) {
+      this.family = family;
+      return this;
+    }
+
+    public Builder withFamilyStoreDirectoryPath(Path familyStoreDirectoryPath) {
+      this.familyStoreDirectoryPath = familyStoreDirectoryPath;
+      return this;
+    }
+
+    public Builder withRegionCoprocessorHost(RegionCoprocessorHost coprocessorHost) {
+      this.coprocessorHost = coprocessorHost;
+      return this;
+    }
+
+    public HStoreContext build () {
+      return new HStoreContext(this);
+    }
+  }
+
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
index 0e4f6c2..6d383e8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
@@ -24,9 +24,13 @@ import java.util.Optional;
 import java.util.OptionalInt;
 import java.util.OptionalLong;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -136,4 +140,26 @@ public class StoreUtils {
     return largestFile.isPresent() ? StoreUtils.getFileSplitPoint(largestFile.get(), comparator)
         : Optional.empty();
   }
+
+  /**
+   * Returns the configured checksum algorithm.
+   * @param conf The configuration
+   * @return The checksum algorithm that is set in the configuration
+   */
+  public static ChecksumType getChecksumType(Configuration conf) {
+    String checksumName = conf.get(HConstants.CHECKSUM_TYPE_NAME);
+    return checksumName == null ? ChecksumType.getDefaultChecksumType() :
+        ChecksumType.nameToType(checksumName);
+  }
+
+  /**
+   * Returns the configured bytesPerChecksum value.
+   * @param conf The configuration
+   * @return The bytesPerChecksum that is set in the configuration
+   */
+  public static int getBytesPerChecksum(Configuration conf) {
+    return conf.getInt(HConstants.BYTES_PER_CHECKSUM,
+        HFile.DEFAULT_BYTES_PER_CHECKSUM);
+  }
+
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
index ec9a59c..342565d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
@@ -86,6 +86,7 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
+import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -749,8 +750,8 @@ public class BulkLoadHFilesTool extends Configured implements BulkLoadHFiles, To
       Algorithm compression = familyDescriptor.getCompressionType();
       BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
       HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
-        .withChecksumType(HStore.getChecksumType(conf))
-        .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf)).withBlockSize(blocksize)
+        .withChecksumType(StoreUtils.getChecksumType(conf))
+        .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf)).withBlockSize(blocksize)
         .withDataBlockEncoding(familyDescriptor.getDataBlockEncoding()).withIncludesTags(true)
         .build();
       halfWriter = new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(outFile)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
new file mode 100644
index 0000000..e69de29
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
index 9623bd1..50bc5fe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
@@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.regionserver.CellSet;
-import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
+import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.EntryBuffers.RegionEntryBuffer;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
@@ -199,8 +199,8 @@ public class BoundedRecoveredHFilesOutputSink extends OutputSink {
         new StoreFileWriter.Builder(walSplitter.conf, CacheConfig.DISABLED, walSplitter.rootFS)
             .withOutputDir(outputDir);
     HFileContext hFileContext = new HFileContextBuilder().
-      withChecksumType(HStore.getChecksumType(walSplitter.conf)).
-      withBytesPerCheckSum(HStore.getBytesPerChecksum(walSplitter.conf)).
+      withChecksumType(StoreUtils.getChecksumType(walSplitter.conf)).
+      withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(walSplitter.conf)).
       withCellComparator(isMetaTable?
         MetaCellComparator.META_COMPARATOR: CellComparatorImpl.COMPARATOR).build();
     return writerBuilder.withFileContext(hFileContext).build();
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 fcbc718..bd1715b 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
@@ -7657,7 +7657,7 @@ public class TestHRegion {
         LOG.warn("hbase.hstore.compaction.complete is set to false");
         List<HStoreFile> sfs = new ArrayList<>(newFiles.size());
         final boolean evictOnClose =
-            cacheConf != null? cacheConf.shouldEvictOnClose(): true;
+            getCacheConfig() != null? getCacheConfig().shouldEvictOnClose(): true;
         for (Path newFile : newFiles) {
           // Create storefile around what we wrote with a reader on it.
           HStoreFile sf = createStoreFileAndReader(newFile);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
index 12cf576..88f201e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
@@ -221,8 +221,8 @@ public class TestSecureBulkLoadManager {
         .withIncludesTags(true)
         .withCompression(compression)
         .withCompressTags(family.isCompressTags())
-        .withChecksumType(HStore.getChecksumType(conf))
-        .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf))
+        .withChecksumType(StoreUtils.getChecksumType(conf))
+        .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf))
         .withBlockSize(family.getBlocksize())
         .withHBaseCheckSum(true)
         .withDataBlockEncoding(family.getDataBlockEncoding())