You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mm...@apache.org on 2018/12/27 19:22:57 UTC

[accumulo] branch master updated: Create builder to clean up CachableBlockFile (#858)

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

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/master by this push:
     new a6fa30f  Create builder to clean up CachableBlockFile (#858)
a6fa30f is described below

commit a6fa30fa8716c5a0f9010948616d40015e1a1860
Author: Mike Miller <mm...@apache.org>
AuthorDate: Thu Dec 27 14:22:53 2018 -0500

    Create builder to clean up CachableBlockFile (#858)
---
 .../accumulo/core/client/rfile/RFileScanner.java   |   8 +-
 .../file/blockfile/impl/CachableBlockFile.java     | 112 ++++++++++++++-------
 .../apache/accumulo/core/file/rfile/PrintInfo.java |   8 +-
 .../org/apache/accumulo/core/file/rfile/RFile.java |   4 +
 .../accumulo/core/file/rfile/RFileOperations.java  |  13 +--
 .../accumulo/core/file/rfile/SplitLarge.java       |   8 +-
 .../accumulo/core/summary/SummaryReader.java       |  16 ++-
 .../core/file/rfile/MultiLevelIndexTest.java       |   8 +-
 .../core/file/rfile/MultiThreadedRFileTest.java    |   7 +-
 .../apache/accumulo/core/file/rfile/RFileTest.java |  18 ++--
 10 files changed, 123 insertions(+), 79 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
index 53f78df..d6db861 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
@@ -48,7 +48,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
 import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheManagerFactory;
-import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
 import org.apache.accumulo.core.iterators.IteratorAdapter;
@@ -339,8 +339,10 @@ class RFileScanner extends ScannerOptions implements Scanner {
       for (int i = 0; i < sources.length; i++) {
         // TODO may have been a bug with multiple files and caching in older version...
         FSDataInputStream inputStream = (FSDataInputStream) sources[i].getInputStream();
-        readers.add(new RFile.Reader(new CachableBlockFile.Reader("source-" + i, inputStream,
-            sources[i].getLength(), opts.in.getConf(), dataCache, indexCache, cryptoService)));
+        CachableBuilder cb = new CachableBuilder().cacheId("source-" + i).input(inputStream)
+            .length(sources[i].getLength()).conf(opts.in.getConf()).data(dataCache)
+            .index(indexCache).cryptoService(cryptoService);
+        readers.add(new RFile.Reader(cb));
       }
 
       if (getSamplerConfiguration() != null) {
diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java
index 97deb4a..44699b2 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.fs.Seekable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
 import com.google.common.cache.Cache;
 
 /**
@@ -66,6 +65,70 @@ public class CachableBlockFile {
     return p.toString();
   }
 
+  public static class CachableBuilder {
+    String cacheId = null;
+    IoeSupplier<InputStream> inputSupplier = null;
+    IoeSupplier<Long> lengthSupplier = null;
+    Cache<String,Long> fileLenCache = null;
+    BlockCache dCache = null;
+    BlockCache iCache = null;
+    RateLimiter readLimiter = null;
+    Configuration hadoopConf = null;
+    CryptoService cryptoService = null;
+
+    public CachableBuilder cacheId(String id) {
+      this.cacheId = id;
+      return this;
+    }
+
+    public CachableBuilder conf(Configuration hadoopConf) {
+      this.hadoopConf = hadoopConf;
+      return this;
+    }
+
+    public CachableBuilder fsPath(FileSystem fs, Path dataFile) {
+      this.cacheId = pathToCacheId(dataFile);
+      this.inputSupplier = () -> fs.open(dataFile);
+      this.lengthSupplier = () -> fs.getFileStatus(dataFile).getLen();
+      return this;
+    }
+
+    public CachableBuilder input(InputStream is) {
+      this.inputSupplier = () -> is;
+      return this;
+    }
+
+    public CachableBuilder length(long len) {
+      this.lengthSupplier = () -> len;
+      return this;
+    }
+
+    public CachableBuilder fileLen(Cache<String,Long> cache) {
+      this.fileLenCache = cache;
+      return this;
+    }
+
+    public CachableBuilder data(BlockCache dCache) {
+      this.dCache = dCache;
+      return this;
+    }
+
+    public CachableBuilder index(BlockCache iCache) {
+      this.iCache = iCache;
+      return this;
+    }
+
+    public CachableBuilder readLimiter(RateLimiter readLimiter) {
+      this.readLimiter = readLimiter;
+      return this;
+    }
+
+    public CachableBuilder cryptoService(CryptoService cryptoService) {
+      this.cryptoService = cryptoService;
+      return this;
+    }
+  }
+
   /**
    * Class wraps the BCFile reader.
    */
@@ -299,43 +362,16 @@ public class CachableBlockFile {
       }
     }
 
-    private Reader(String cacheId, IoeSupplier<InputStream> inputSupplier,
-        IoeSupplier<Long> lenghtSupplier, Cache<String,Long> fileLenCache, BlockCache data,
-        BlockCache index, RateLimiter readLimiter, Configuration conf,
-        CryptoService cryptoService) {
-      Preconditions.checkArgument(cacheId != null || (data == null && index == null));
-      this.cacheId = cacheId;
-      this.inputSupplier = inputSupplier;
-      this.lengthSupplier = lenghtSupplier;
-      this.fileLenCache = fileLenCache;
-      this._dCache = data;
-      this._iCache = index;
-      this.readLimiter = readLimiter;
-      this.conf = conf;
-      this.cryptoService = Objects.requireNonNull(cryptoService);
-    }
-
-    public Reader(FileSystem fs, Path dataFile, Configuration conf, BlockCache data,
-        BlockCache index, CryptoService cryptoService) {
-      this(fs, dataFile, conf, null, data, index, null, cryptoService);
-    }
-
-    public Reader(FileSystem fs, Path dataFile, Configuration conf, Cache<String,Long> fileLenCache,
-        BlockCache data, BlockCache index, RateLimiter readLimiter, CryptoService cryptoService) {
-      this(pathToCacheId(dataFile), () -> fs.open(dataFile),
-          () -> fs.getFileStatus(dataFile).getLen(), fileLenCache, data, index, readLimiter, conf,
-          cryptoService);
-    }
-
-    public <InputStreamType extends InputStream & Seekable> Reader(String cacheId,
-        InputStreamType fsin, long len, Configuration conf, BlockCache data, BlockCache index,
-        CryptoService cryptoService) {
-      this(cacheId, () -> fsin, () -> len, null, data, index, null, conf, cryptoService);
-    }
-
-    public <InputStreamType extends InputStream & Seekable> Reader(InputStreamType fsin, long len,
-        Configuration conf, CryptoService cryptoService) {
-      this(null, () -> fsin, () -> len, null, null, null, null, conf, cryptoService);
+    public Reader(CachableBuilder b) {
+      this.cacheId = b.cacheId;
+      this.inputSupplier = b.inputSupplier;
+      this.lengthSupplier = b.lengthSupplier;
+      this.fileLenCache = b.fileLenCache;
+      this._dCache = b.dCache;
+      this._iCache = b.iCache;
+      this.readLimiter = b.readLimiter;
+      this.conf = b.hadoopConf;
+      this.cryptoService = Objects.requireNonNull(b.cryptoService);
     }
 
     /**
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
index 7278cb8..a36595d 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
@@ -33,7 +33,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
 import org.apache.accumulo.core.file.rfile.bcfile.Utils;
 import org.apache.accumulo.core.summary.SummaryReader;
@@ -178,9 +178,9 @@ public class PrintInfo implements KeywordExecutable {
 
       printCryptoParams(path, fs);
 
-      CachableBlockFile.Reader _rdr = new CachableBlockFile.Reader(fs, path, conf, null, null,
-          CryptoServiceFactory.newInstance(siteConfig, ClassloaderType.ACCUMULO));
-      Reader iter = new RFile.Reader(_rdr);
+      CachableBuilder cb = new CachableBuilder().fsPath(fs, path).conf(conf)
+          .cryptoService(CryptoServiceFactory.newInstance(siteConfig, ClassloaderType.JAVA));
+      Reader iter = new RFile.Reader(cb);
       MetricsGatherer<Map<String,ArrayList<VisibilityMetric>>> vmg = new VisMetricsGatherer();
 
       if (opts.vis || opts.hash)
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
index f6bbd82..09dabed 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
@@ -1247,6 +1247,10 @@ public class RFile {
       this.lgContext = new LocalityGroupContext(currentReaders);
     }
 
+    public Reader(CachableBlockFile.CachableBuilder b) throws IOException {
+      this(new CachableBlockFile.Reader(b));
+    }
+
     private void closeLocalityGroupReaders() {
       for (LocalityGroupReader lgr : currentReaders) {
         try {
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
index 0be4fa6..ad2e26e 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
@@ -29,7 +29,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
 import org.apache.accumulo.core.file.rfile.bcfile.BCFile;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.sample.impl.SamplerFactory;
@@ -45,11 +45,12 @@ public class RFileOperations extends FileOperations {
   private static final Collection<ByteSequence> EMPTY_CF_SET = Collections.emptySet();
 
   private static RFile.Reader getReader(FileOptions options) throws IOException {
-    CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(options.getFileSystem(),
-        new Path(options.getFilename()), options.getConfiguration(), options.getFileLenCache(),
-        options.getDataCache(), options.getIndexCache(), options.getRateLimiter(),
-        options.getCryptoService());
-    return new RFile.Reader(_cbr);
+    CachableBuilder cb = new CachableBuilder()
+        .fsPath(options.getFileSystem(), new Path(options.getFilename()))
+        .conf(options.getConfiguration()).fileLen(options.getFileLenCache())
+        .data(options.getDataCache()).index(options.getIndexCache())
+        .readLimiter(options.getRateLimiter()).cryptoService(options.getCryptoService());
+    return new RFile.Reader(cb);
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/SplitLarge.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/SplitLarge.java
index 69af66a..6f330d4 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/SplitLarge.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/SplitLarge.java
@@ -28,7 +28,7 @@ import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
 import org.apache.accumulo.core.file.rfile.RFile.Writer;
 import org.apache.accumulo.core.file.rfile.bcfile.BCFile;
@@ -66,9 +66,9 @@ public class SplitLarge {
       CryptoService cryptoService = ConfigurationTypeHelper.getClassInstance(null, opts.cryptoClass,
           CryptoService.class, CryptoServiceFactory.newDefaultInstance());
       Path path = new Path(file);
-      CachableBlockFile.Reader rdr = new CachableBlockFile.Reader(fs, path, conf, null, null,
-          cryptoService);
-      try (Reader iter = new RFile.Reader(rdr)) {
+      CachableBuilder cb = new CachableBuilder().fsPath(fs, path).conf(conf)
+          .cryptoService(cryptoService);
+      try (Reader iter = new RFile.Reader(cb)) {
 
         if (!file.endsWith(".rf")) {
           throw new IllegalArgumentException("File must end with .rf");
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java b/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
index 5730e74..b36fb13 100644
--- a/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
@@ -33,6 +33,7 @@ import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
 import org.apache.accumulo.core.file.rfile.bcfile.MetaBlockDoesNotExist;
 import org.apache.accumulo.core.spi.cache.BlockCache;
@@ -42,7 +43,6 @@ import org.apache.accumulo.core.summary.Gatherer.RowRange;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.Seekable;
 import org.apache.hadoop.io.WritableUtils;
 
 import com.google.common.cache.Cache;
@@ -177,12 +177,9 @@ public class SummaryReader {
   public static SummaryReader load(Configuration conf, AccumuloConfiguration aConf,
       InputStream inputStream, long length, Predicate<SummarizerConfiguration> summarySelector,
       SummarizerFactory factory, CryptoService cryptoService) throws IOException {
-    // @formatter:off
-    org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.Reader bcReader =
-      new CachableBlockFile.Reader((InputStream & Seekable) inputStream, length, conf,
-              cryptoService);
-    // @formatter:on
-    return load(bcReader, summarySelector, factory);
+    CachableBuilder cb = new CachableBuilder().input(inputStream).length(length).conf(conf)
+        .cryptoService(cryptoService);
+    return load(new CachableBlockFile.Reader(cb), summarySelector, factory);
   }
 
   public static SummaryReader load(FileSystem fs, Configuration conf, AccumuloConfiguration aConf,
@@ -195,8 +192,9 @@ public class SummaryReader {
       // the reason BCFile is used instead of RFile is to avoid reading in the RFile meta block when
       // only summary data is wanted.
       CompositeCache compositeCache = new CompositeCache(summaryCache, indexCache);
-      bcReader = new CachableBlockFile.Reader(fs, file, conf, fileLenCache, null, compositeCache,
-          null, cryptoService);
+      CachableBuilder cb = new CachableBuilder().fsPath(fs, file).conf(conf).fileLen(fileLenCache)
+          .index(compositeCache).cryptoService(cryptoService);
+      bcReader = new CachableBlockFile.Reader(cb);
       return load(bcReader, summarySelector, factory);
     } catch (FileNotFoundException fne) {
       SummaryReader sr = new SummaryReader();
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
index 134c185..9418176 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
 import org.apache.accumulo.core.file.rfile.MultiLevelIndex.BufferedWriter;
 import org.apache.accumulo.core.file.rfile.MultiLevelIndex.IndexEntry;
 import org.apache.accumulo.core.file.rfile.MultiLevelIndex.Reader;
@@ -82,9 +83,10 @@ public class MultiLevelIndexTest {
     byte[] data = baos.toByteArray();
     SeekableByteArrayInputStream bais = new SeekableByteArrayInputStream(data);
     FSDataInputStream in = new FSDataInputStream(bais);
-    CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in, data.length,
-        CachedConfiguration.getInstance(),
-        CryptoServiceFactory.newInstance(aconf, ClassloaderType.JAVA));
+    CachableBuilder cb = new CachableBuilder().input(in).length(data.length)
+        .conf(CachedConfiguration.getInstance())
+        .cryptoService(CryptoServiceFactory.newInstance(aconf, ClassloaderType.JAVA));
+    CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(cb);
 
     Reader reader = new Reader(_cbr, RFile.RINDEX_VER_8);
     CachableBlockFile.CachedBlockRead rootIn = _cbr.getMetaBlock("root");
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java
index 9eca5f3..405524b 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java
@@ -48,6 +48,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
 import org.apache.accumulo.core.file.rfile.bcfile.BCFile;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -186,9 +187,9 @@ public class MultiThreadedRFileTest {
       AccumuloConfiguration defaultConf = DefaultConfiguration.getInstance();
 
       // the caches used to obfuscate the multithreaded issues
-      CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(fs, path, conf, null, null,
-          CryptoServiceFactory.newInstance(defaultConf, ClassloaderType.JAVA));
-      reader = new RFile.Reader(_cbr);
+      CachableBuilder b = new CachableBuilder().fsPath(fs, path).conf(conf)
+          .cryptoService(CryptoServiceFactory.newInstance(defaultConf, ClassloaderType.JAVA));
+      reader = new RFile.Reader(new CachableBlockFile.Reader(b));
       iter = new ColumnFamilySkippingIterator(reader);
 
       checkIndex(reader);
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
index 82affa5..864f6d6 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
@@ -70,7 +70,7 @@ import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguratio
 import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheManagerFactory;
 import org.apache.accumulo.core.file.blockfile.cache.lru.LruBlockCache;
 import org.apache.accumulo.core.file.blockfile.cache.lru.LruBlockCacheManager;
-import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
 import org.apache.accumulo.core.file.rfile.bcfile.BCFile;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -304,10 +304,10 @@ public class RFileTest {
       LruBlockCache indexCache = (LruBlockCache) manager.getBlockCache(CacheType.INDEX);
       LruBlockCache dataCache = (LruBlockCache) manager.getBlockCache(CacheType.DATA);
 
-      CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader("source-1", in, fileLength, conf,
-          dataCache, indexCache,
-          CryptoServiceFactory.newInstance(accumuloConfiguration, ClassloaderType.JAVA));
-      reader = new RFile.Reader(_cbr);
+      CachableBuilder cb = new CachableBuilder().cacheId("source-1").input(in).length(fileLength)
+          .conf(conf).data(dataCache).index(indexCache).cryptoService(
+              CryptoServiceFactory.newInstance(accumuloConfiguration, ClassloaderType.JAVA));
+      reader = new RFile.Reader(cb);
       if (cfsi)
         iter = new ColumnFamilySkippingIterator(reader);
 
@@ -1735,10 +1735,10 @@ public class RFileTest {
     byte data[] = baos.toByteArray();
     SeekableByteArrayInputStream bais = new SeekableByteArrayInputStream(data);
     FSDataInputStream in2 = new FSDataInputStream(bais);
-    CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in2, data.length,
-        CachedConfiguration.getInstance(),
-        CryptoServiceFactory.newInstance(aconf, ClassloaderType.JAVA));
-    Reader reader = new RFile.Reader(_cbr);
+    CachableBuilder cb = new CachableBuilder().input(in2).length(data.length)
+        .conf(CachedConfiguration.getInstance())
+        .cryptoService(CryptoServiceFactory.newInstance(aconf, ClassloaderType.JAVA));
+    Reader reader = new RFile.Reader(cb);
     checkIndex(reader);
 
     ColumnFamilySkippingIterator iter = new ColumnFamilySkippingIterator(reader);