You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2018/11/30 15:44:11 UTC

[GitHub] keith-turner closed pull request #786: fixes #785 avoid using system classloader in client rfile code

keith-turner closed pull request #786: fixes #785 avoid using system classloader in client rfile code
URL: https://github.com/apache/accumulo/pull/786
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 d08bd846ff..9efd4e857e 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
@@ -40,6 +40,7 @@
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.Key;
@@ -217,7 +218,7 @@ public void indexWeightChanged() {}
     if (null == this.dataCache) {
       this.dataCache = new NoopCache();
     }
-    this.cryptoService = CryptoServiceFactory.newInstance(tableConf);
+    this.cryptoService = CryptoServiceFactory.newInstance(tableConf, ClassloaderType.JAVA);
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSummariesRetriever.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSummariesRetriever.java
index e1de4fd0fb..c02af8f953 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSummariesRetriever.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSummariesRetriever.java
@@ -34,6 +34,8 @@
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
+import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.summary.Gatherer;
 import org.apache.accumulo.core.summary.SummarizerFactory;
 import org.apache.accumulo.core.summary.SummaryCollection;
@@ -89,10 +91,10 @@ public SummaryOptions endRow(Text endRow) {
     RFileSource[] sources = in.getSources();
     try {
       SummaryCollection all = new SummaryCollection();
+      CryptoService cservice = CryptoServiceFactory.newInstance(acuconf, ClassloaderType.JAVA);
       for (RFileSource source : sources) {
         SummaryReader fileSummary = SummaryReader.load(in.getFileSystem().getConf(), acuconf,
-            source.getInputStream(), source.getLength(), summarySelector, factory,
-            CryptoServiceFactory.newInstance(acuconf));
+            source.getInputStream(), source.getLength(), summarySelector, factory, cservice);
         SummaryCollection sc = fileSummary
             .getSummaries(Collections.singletonList(new Gatherer.RowRange(startRow, endRow)));
         all.merge(sc, factory);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
index 5a73c1aff7..0e87d697ee 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
@@ -34,8 +34,11 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
+import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -97,6 +100,8 @@ public RFileWriter build() throws IOException {
       acuconf = new ConfigurationCopy(Iterables.concat(acuconf, userProps.entrySet()));
     }
 
+    CryptoService cs = CryptoServiceFactory.newInstance(acuconf, ClassloaderType.JAVA);
+
     if (out.getOutputStream() != null) {
       FSDataOutputStream fsdo;
       if (out.getOutputStream() instanceof FSDataOutputStream) {
@@ -104,11 +109,13 @@ public RFileWriter build() throws IOException {
       } else {
         fsdo = new FSDataOutputStream(out.getOutputStream(), new FileSystem.Statistics("foo"));
       }
-      return new RFileWriter(fileops.newWriterBuilder().forOutputStream(".rf", fsdo, out.getConf())
-          .withTableConfiguration(acuconf).withStartDisabled().build(), visCacheSize);
+      return new RFileWriter(
+          fileops.newWriterBuilder().forOutputStream(".rf", fsdo, out.getConf(), cs)
+              .withTableConfiguration(acuconf).withStartDisabled().build(),
+          visCacheSize);
     } else {
       return new RFileWriter(fileops.newWriterBuilder()
-          .forFile(out.path.toString(), out.getFileSystem(), out.getConf())
+          .forFile(out.path.toString(), out.getFileSystem(), out.getConf(), cs)
           .withTableConfiguration(acuconf).withStartDisabled().build(), visCacheSize);
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/BulkImport.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/BulkImport.java
index 3d8da28b42..5652358756 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/BulkImport.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/BulkImport.java
@@ -59,6 +59,7 @@
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.LoadPlan;
@@ -70,6 +71,7 @@
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
 import org.apache.accumulo.core.master.thrift.FateOperation;
+import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.hadoop.fs.FileStatus;
@@ -206,8 +208,8 @@ public MLong(long i) {
   }
 
   public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path mapFile,
-      long fileSize, Collection<KeyExtent> extents, FileSystem ns, Cache<String,Long> fileLenCache)
-      throws IOException {
+      long fileSize, Collection<KeyExtent> extents, FileSystem ns, Cache<String,Long> fileLenCache,
+      CryptoService cs) throws IOException {
 
     if (extents.size() == 1) {
       return Collections.singletonMap(extents.iterator().next(), fileSize);
@@ -221,7 +223,7 @@ public MLong(long i) {
     Text row = new Text();
 
     FileSKVIterator index = FileOperations.getInstance().newIndexReaderBuilder()
-        .forFile(mapFile.toString(), ns, ns.getConf()).withTableConfiguration(acuConf)
+        .forFile(mapFile.toString(), ns, ns.getConf(), cs).withTableConfiguration(acuConf)
         .withFileLenCache(fileLenCache).build();
 
     try {
@@ -302,10 +304,11 @@ private static Text nextRow(Text row) {
   }
 
   public static List<KeyExtent> findOverlappingTablets(ClientContext context,
-      KeyExtentCache extentCache, Path file, FileSystem fs, Cache<String,Long> fileLenCache)
+      KeyExtentCache extentCache, Path file, FileSystem fs, Cache<String,Long> fileLenCache,
+      CryptoService cs)
       throws IOException, AccumuloException, AccumuloSecurityException, TableNotFoundException {
     try (FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-        .forFile(file.toString(), fs, fs.getConf())
+        .forFile(file.toString(), fs, fs.getConf(), cs)
         .withTableConfiguration(context.getConfiguration()).withFileLenCache(fileLenCache)
         .seekToBeginning().build()) {
       return findOverlappingTablets(context, extentCache, reader);
@@ -495,14 +498,16 @@ private Text toText(byte[] row) {
 
     List<CompletableFuture<Map<KeyExtent,Bulk.FileInfo>>> futures = new ArrayList<>();
 
+    CryptoService cs = CryptoServiceFactory.newDefaultInstance();
+
     for (FileStatus fileStatus : files) {
       CompletableFuture<Map<KeyExtent,Bulk.FileInfo>> future = CompletableFuture.supplyAsync(() -> {
         try {
           long t1 = System.currentTimeMillis();
           List<KeyExtent> extents = findOverlappingTablets(context, extentCache,
-              fileStatus.getPath(), fs, fileLensCache);
+              fileStatus.getPath(), fs, fileLensCache, cs);
           Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(),
-              fileStatus.getPath(), fileStatus.getLen(), extents, fs, fileLensCache);
+              fileStatus.getPath(), fileStatus.getLen(), extents, fs, fileLensCache, cs);
           Map<KeyExtent,Bulk.FileInfo> pathLocations = new HashMap<>();
           for (KeyExtent ke : extents) {
             pathLocations.put(ke,
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
index ba3efbcab8..afd9393e3e 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
@@ -38,6 +38,7 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;
 import org.apache.accumulo.core.data.PartialKey;
@@ -362,7 +363,8 @@ private void nextTablet() throws TableNotFoundException, AccumuloException, IOEx
     for (String file : absFiles) {
       FileSystem fs = VolumeConfiguration.getVolume(file, conf, config).getFileSystem();
       FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-          .forFile(file, fs, conf).withTableConfiguration(acuTableConf).build();
+          .forFile(file, fs, conf, CryptoServiceFactory.newDefaultInstance())
+          .withTableConfiguration(acuTableConf).build();
       if (scannerSamplerConfigImpl != null) {
         reader = reader.getSample(scannerSamplerConfigImpl);
         if (reader == null)
diff --git a/core/src/main/java/org/apache/accumulo/core/crypto/CryptoServiceFactory.java b/core/src/main/java/org/apache/accumulo/core/crypto/CryptoServiceFactory.java
index d5155f3f35..aefacaf501 100644
--- a/core/src/main/java/org/apache/accumulo/core/crypto/CryptoServiceFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/crypto/CryptoServiceFactory.java
@@ -17,19 +17,48 @@
 package org.apache.accumulo.core.crypto;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.cryptoImpl.NoCryptoService;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
 
 public class CryptoServiceFactory {
 
-  /**
-   * Create a new crypto service configured in {@link Property#INSTANCE_CRYPTO_SERVICE}.
-   */
-  public static CryptoService newInstance(AccumuloConfiguration conf) {
-    CryptoService newCryptoService = Property.createInstanceFromPropertyName(conf,
-        Property.INSTANCE_CRYPTO_SERVICE, CryptoService.class, new NoCryptoService());
+  public enum ClassloaderType {
+    // Use the Accumulo custom classloader. Should only be used by Accumulo server side code.
+    ACCUMULO,
+    // Use basic Java classloading mechanism. Should be use by Accumulo client code.
+    JAVA
+  }
+
+  public static CryptoService newInstance(AccumuloConfiguration conf, ClassloaderType ct) {
+
+    CryptoService newCryptoService;
+
+    if (ct == ClassloaderType.ACCUMULO) {
+      newCryptoService = Property.createInstanceFromPropertyName(conf,
+          Property.INSTANCE_CRYPTO_SERVICE, CryptoService.class, new NoCryptoService());
+    } else if (ct == ClassloaderType.JAVA) {
+      String clazzName = conf.get(Property.INSTANCE_CRYPTO_SERVICE);
+      if (clazzName == null || clazzName.trim().isEmpty()) {
+        newCryptoService = new NoCryptoService();
+      } else {
+        try {
+          newCryptoService = CryptoServiceFactory.class.getClassLoader().loadClass(clazzName)
+              .asSubclass(CryptoService.class).newInstance();
+        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    } else {
+      throw new IllegalArgumentException();
+    }
+
     newCryptoService.init(conf.getAllPropertiesWithPrefix(Property.INSTANCE_CRYPTO_PREFIX));
     return newCryptoService;
   }
+
+  public static CryptoService newDefaultInstance() {
+    return newInstance(DefaultConfiguration.getInstance(), ClassloaderType.JAVA);
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
index 1e1d3fe638..9c9d417431 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
@@ -43,6 +43,7 @@
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
@@ -477,7 +478,8 @@ public static void main(String[] args) throws IOException {
 
     String suffix = FileOperations.getNewFileExtension(acuconf);
     String fname = "/tmp/test." + suffix;
-    FileSKVWriter bmfw = FileOperations.getInstance().newWriterBuilder().forFile(fname, fs, conf)
+    FileSKVWriter bmfw = FileOperations.getInstance().newWriterBuilder()
+        .forFile(fname, fs, conf, CryptoServiceFactory.newDefaultInstance())
         .withTableConfiguration(acuconf).build();
 
     long t1 = System.currentTimeMillis();
@@ -499,7 +501,8 @@ public static void main(String[] args) throws IOException {
     bmfw.close();
 
     t1 = System.currentTimeMillis();
-    FileSKVIterator bmfr = FileOperations.getInstance().newReaderBuilder().forFile(fname, fs, conf)
+    FileSKVIterator bmfr = FileOperations.getInstance().newReaderBuilder()
+        .forFile(fname, fs, conf, CryptoServiceFactory.newDefaultInstance())
         .withTableConfiguration(acuconf).build();
     t2 = System.currentTimeMillis();
     out.println("Opened " + fname + " in " + (t2 - t1));
diff --git a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
index 1ff0e225b9..42077cc858 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
@@ -188,7 +188,7 @@ public FileOptions(AccumuloConfiguration tableConfiguration, String filename, Fi
       this.indexCache = indexCache;
       this.fileLenCache = fileLenCache;
       this.seekToBeginning = seekToBeginning;
-      this.cryptoService = cryptoService;
+      this.cryptoService = Objects.requireNonNull(cryptoService);
       this.range = range;
       this.columnFamilies = columnFamilies;
       this.inclusive = inclusive;
@@ -263,34 +263,30 @@ public boolean isRangeInclusive() {
    * Helper class extended by both writers and readers.
    */
   public class FileHelper {
-    protected AccumuloConfiguration tableConfiguration;
-    protected String filename;
-    protected FileSystem fs;
-    protected Configuration fsConf;
-    protected RateLimiter rateLimiter;
-    protected CryptoService cryptoService;
+    private AccumuloConfiguration tableConfiguration;
+    private String filename;
+    private FileSystem fs;
+    private Configuration fsConf;
+    private RateLimiter rateLimiter;
+    private CryptoService cryptoService;
 
     protected FileHelper fs(FileSystem fs) {
-      Objects.requireNonNull(fs);
-      this.fs = fs;
+      this.fs = Objects.requireNonNull(fs);
       return this;
     }
 
     protected FileHelper fsConf(Configuration fsConf) {
-      Objects.requireNonNull(fsConf);
-      this.fsConf = fsConf;
+      this.fsConf = Objects.requireNonNull(fsConf);
       return this;
     }
 
     protected FileHelper filename(String filename) {
-      Objects.requireNonNull(filename);
-      this.filename = filename;
+      this.filename = Objects.requireNonNull(filename);
       return this;
     }
 
     protected FileHelper tableConfiguration(AccumuloConfiguration tableConfiguration) {
-      Objects.requireNonNull(tableConfiguration);
-      this.tableConfiguration = tableConfiguration;
+      this.tableConfiguration = Objects.requireNonNull(tableConfiguration);
       return this;
     }
 
@@ -300,7 +296,7 @@ protected FileHelper rateLimiter(RateLimiter rateLimiter) {
     }
 
     protected FileHelper cryptoService(CryptoService cs) {
-      this.cryptoService = cs;
+      this.cryptoService = Objects.requireNonNull(cs);
       return this;
     }
 
@@ -327,6 +323,10 @@ protected FileOptions toScanReaderBuilderOptions(Range range, Set<ByteSequence>
       return new FileOptions(tableConfiguration, filename, fs, fsConf, rateLimiter, null, null,
           false, null, null, null, false, cryptoService, range, columnFamilies, inclusive);
     }
+
+    protected AccumuloConfiguration getTableConfiguration() {
+      return tableConfiguration;
+    }
   }
 
   /**
@@ -338,14 +338,15 @@ protected FileOptions toScanReaderBuilderOptions(Range range, Set<ByteSequence>
     private boolean enableAccumuloStart = true;
 
     public WriterTableConfiguration forOutputStream(String extension,
-        FSDataOutputStream outputStream, Configuration fsConf) {
+        FSDataOutputStream outputStream, Configuration fsConf, CryptoService cs) {
       this.outputStream = outputStream;
-      filename("foo" + extension).fsConf(fsConf);
+      filename("foo" + extension).fsConf(fsConf).cryptoService(cs);
       return this;
     }
 
-    public WriterTableConfiguration forFile(String filename, FileSystem fs, Configuration fsConf) {
-      filename(filename).fs(fs).fsConf(fsConf);
+    public WriterTableConfiguration forFile(String filename, FileSystem fs, Configuration fsConf,
+        CryptoService cs) {
+      filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
       return this;
     }
 
@@ -370,11 +371,6 @@ public WriterBuilder withRateLimiter(RateLimiter rateLimiter) {
       return this;
     }
 
-    public WriterBuilder withCryptoService(CryptoService cs) {
-      cryptoService(cs);
-      return this;
-    }
-
     public FileSKVWriter build() throws IOException {
       return openWriter(toWriterBuilderOptions(compression, outputStream, enableAccumuloStart));
     }
@@ -393,8 +389,9 @@ public FileSKVWriter build() throws IOException {
     private Cache<String,Long> fileLenCache;
     private boolean seekToBeginning = false;
 
-    public ReaderTableConfiguration forFile(String filename, FileSystem fs, Configuration fsConf) {
-      filename(filename).fs(fs).fsConf(fsConf);
+    public ReaderTableConfiguration forFile(String filename, FileSystem fs, Configuration fsConf,
+        CryptoService cs) {
+      filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
       return this;
     }
 
@@ -433,11 +430,6 @@ public ReaderBuilder withFileLenCache(Cache<String,Long> fileLenCache) {
       return this;
     }
 
-    public ReaderBuilder withCryptoService(CryptoService cs) {
-      cryptoService(cs);
-      return this;
-    }
-
     public ReaderBuilder withRateLimiter(RateLimiter rateLimiter) {
       rateLimiter(rateLimiter);
       return this;
@@ -464,10 +456,10 @@ public FileSKVIterator build() throws IOException {
        * If the table configuration disallows caching, rewrite the options object to not pass the
        * caches.
        */
-      if (!tableConfiguration.getBoolean(Property.TABLE_INDEXCACHE_ENABLED)) {
+      if (!getTableConfiguration().getBoolean(Property.TABLE_INDEXCACHE_ENABLED)) {
         withIndexCache(null);
       }
-      if (!tableConfiguration.getBoolean(Property.TABLE_BLOCKCACHE_ENABLED)) {
+      if (!getTableConfiguration().getBoolean(Property.TABLE_BLOCKCACHE_ENABLED)) {
         withDataCache(null);
       }
       return openReader(
@@ -487,8 +479,8 @@ public FileSKVIterator build() throws IOException {
     private Cache<String,Long> fileLenCache = null;
 
     public IndexReaderTableConfiguration forFile(String filename, FileSystem fs,
-        Configuration fsConf) {
-      filename(filename).fs(fs).fsConf(fsConf);
+        Configuration fsConf, CryptoService cs) {
+      filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
       return this;
     }
 
@@ -519,8 +511,8 @@ public FileSKVIterator build() throws IOException {
     private boolean inclusive;
 
     public ScanReaderTableConfiguration forFile(String filename, FileSystem fs,
-        Configuration fsConf) {
-      filename(filename).fs(fs).fsConf(fsConf);
+        Configuration fsConf, CryptoService cs) {
+      filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
       return this;
     }
 
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 b13ccbec0f..f1243782b3 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
@@ -23,6 +23,7 @@
 import java.io.UncheckedIOException;
 import java.util.Collections;
 import java.util.Map;
+import java.util.Objects;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Supplier;
@@ -316,7 +317,7 @@ private Reader(String cacheId, IoeSupplier<InputStream> inputSupplier,
       this.readLimiter = readLimiter;
       this.conf = conf;
       this.accumuloConfiguration = accumuloConfiguration;
-      this.cryptoService = cryptoService;
+      this.cryptoService = Objects.requireNonNull(cryptoService);
     }
 
     public Reader(FileSystem fs, Path dataFile, Configuration conf, BlockCache data,
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/CreateEmpty.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/CreateEmpty.java
index 763e6bd03c..2652fb718b 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/CreateEmpty.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/CreateEmpty.java
@@ -22,6 +22,7 @@
 
 import org.apache.accumulo.core.cli.Help;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.file.rfile.bcfile.Compression;
 import org.apache.accumulo.core.util.CachedConfiguration;
@@ -83,7 +84,7 @@ public static void main(String[] args) throws Exception {
       Path path = new Path(arg);
       log.info("Writing to file '{}'", path);
       FileSKVWriter writer = (new RFileOperations()).newWriterBuilder()
-          .forFile(arg, path.getFileSystem(conf), conf)
+          .forFile(arg, path.getFileSystem(conf), conf, CryptoServiceFactory.newDefaultInstance())
           .withTableConfiguration(DefaultConfiguration.getInstance()).withCompression(opts.codec)
           .build();
       writer.close();
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 7a68203946..36c5f6b938 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
@@ -25,6 +25,7 @@
 import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.crypto.CryptoUtils;
 import org.apache.accumulo.core.cryptoImpl.NoFileEncrypter;
 import org.apache.accumulo.core.data.ByteSequence;
@@ -178,7 +179,7 @@ public void execute(final String[] args) throws Exception {
       printCryptoParams(path, fs);
 
       CachableBlockFile.Reader _rdr = new CachableBlockFile.Reader(fs, path, conf, null, null,
-          siteConfig, CryptoServiceFactory.newInstance(siteConfig));
+          siteConfig, CryptoServiceFactory.newInstance(siteConfig, ClassloaderType.ACCUMULO));
       Reader iter = new RFile.Reader(_rdr);
       MetricsGatherer<Map<String,ArrayList<VisibilityMetric>>> vmg = new VisMetricsGatherer();
 
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 74fcd912d8..ee35fd5538 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
@@ -23,7 +23,6 @@
 import org.apache.accumulo.core.client.sample.Sampler;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
@@ -34,7 +33,6 @@
 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;
-import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -131,15 +129,8 @@ protected FileSKVWriter openWriter(FileOptions options) throws IOException {
       outputStream = fs.create(new Path(file), false, bufferSize, (short) rep, block);
     }
 
-    // calls to openWriter from the tserver will already have a crypto service initialized
-    // calls from clients will require a new crypto service
-    CryptoService cryptoService = options.cryptoService;
-    if (cryptoService == null) {
-      cryptoService = CryptoServiceFactory.newInstance(acuconf);
-    }
-
     BCFile.Writer _cbw = new BCFile.Writer(outputStream, options.getRateLimiter(), compression,
-        conf, acuconf, cryptoService);
+        conf, acuconf, options.cryptoService);
 
     return new RFile.Writer(_cbw, (int) blockSize, (int) indexBlockSize, samplerConfig, sampler);
   }
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 d53190a2c1..3c566742af 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
@@ -24,7 +24,7 @@
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.cryptoImpl.NoCryptoService;
+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;
@@ -64,7 +64,7 @@ public static void main(String[] args) throws Exception {
     for (String file : opts.files) {
       AccumuloConfiguration aconf = DefaultConfiguration.getInstance();
       CryptoService cryptoService = ConfigurationTypeHelper.getClassInstance(null, opts.cryptoClass,
-          CryptoService.class, new NoCryptoService());
+          CryptoService.class, CryptoServiceFactory.newDefaultInstance());
       Path path = new Path(file);
       CachableBlockFile.Reader rdr = new CachableBlockFile.Reader(fs, path, conf, null, null, aconf,
           cryptoService);
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java
index a691a9cf5c..2747afe56b 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java
@@ -34,7 +34,6 @@
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.crypto.CryptoUtils;
 import org.apache.accumulo.core.cryptoImpl.CryptoEnvironmentImpl;
 import org.apache.accumulo.core.cryptoImpl.NoFileDecrypter;
@@ -642,9 +641,6 @@ public long getRawSize() {
       metaIndex = new MetaIndex(this.in);
 
       CryptoEnvironment cryptoEnvironment = null;
-      if (cryptoService == null) {
-        cryptoService = CryptoServiceFactory.newInstance(aconf);
-      }
 
       // backwards compatibility
       if (version.equals(API_VERSION_1)) {
@@ -680,9 +676,6 @@ public long getRawSize() {
 
       decryptionParams = CryptoUtils.readParams(dis);
       CryptoEnvironmentImpl env = new CryptoEnvironmentImpl(Scope.RFILE, decryptionParams);
-      if (cryptoService == null) {
-        cryptoService = CryptoServiceFactory.newInstance(aconf);
-      }
       this.decrypter = cryptoService.getFileDecrypter(env);
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java
index c224121533..cc0f26e6ad 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java
@@ -24,6 +24,7 @@
 import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.file.rfile.bcfile.BCFile.MetaIndexEntry;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -39,7 +40,7 @@ public static void printMetaBlockInfo(SiteConfiguration siteConfig, Configuratio
     BCFile.Reader bcfr = null;
     try {
       bcfr = new BCFile.Reader(fsin, fs.getFileStatus(path).getLen(), conf, siteConfig,
-          CryptoServiceFactory.newInstance(siteConfig));
+          CryptoServiceFactory.newInstance(siteConfig, ClassloaderType.ACCUMULO));
 
       Set<Entry<String,MetaIndexEntry>> es = bcfr.metaIndex.index.entrySet();
 
diff --git a/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java
index 745d4b4e41..70ee484b0c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java
@@ -51,7 +51,7 @@
 import org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.cryptoImpl.NoCryptoService;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -816,9 +816,8 @@ public void testWrongGroup() throws Exception {
 
   private Reader getReader(LocalFileSystem localFs, String testFile) throws IOException {
     return (Reader) FileOperations.getInstance().newReaderBuilder()
-        .forFile(testFile, localFs, localFs.getConf())
-        .withTableConfiguration(DefaultConfiguration.getInstance())
-        .withCryptoService(new NoCryptoService()).build();
+        .forFile(testFile, localFs, localFs.getConf(), CryptoServiceFactory.newDefaultInstance())
+        .withTableConfiguration(DefaultConfiguration.getInstance()).build();
   }
 
   @Test
diff --git a/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java b/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java
index 8e67cbb0a6..740f5acce7 100644
--- a/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java
@@ -53,11 +53,11 @@
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.crypto.streams.NoFlushOutputStream;
 import org.apache.accumulo.core.cryptoImpl.AESCryptoService;
 import org.apache.accumulo.core.cryptoImpl.AESKeyUtils;
 import org.apache.accumulo.core.cryptoImpl.CryptoEnvironmentImpl;
-import org.apache.accumulo.core.cryptoImpl.NoCryptoService;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
@@ -177,7 +177,7 @@ public void testAESCryptoServiceRFILE() throws Exception {
 
   @Test
   public void testNoEncryptionWAL() throws Exception {
-    NoCryptoService cs = new NoCryptoService();
+    CryptoService cs = CryptoServiceFactory.newDefaultInstance();
     byte[] encryptedBytes = encrypt(cs, Scope.WAL, CRYPTO_OFF_CONF);
 
     String stringifiedBytes = Arrays.toString(encryptedBytes);
@@ -191,7 +191,7 @@ public void testNoEncryptionWAL() throws Exception {
 
   @Test
   public void testNoEncryptionRFILE() throws Exception {
-    NoCryptoService cs = new NoCryptoService();
+    CryptoService cs = CryptoServiceFactory.newDefaultInstance();
     byte[] encryptedBytes = encrypt(cs, Scope.RFILE, CRYPTO_OFF_CONF);
 
     String stringifiedBytes = Arrays.toString(encryptedBytes);
@@ -393,7 +393,7 @@ private void decrypt(byte[] resultingBytes, Scope scope, String configFile) thro
       byte[] params = CryptoUtils.readParams(dataIn);
 
       AccumuloConfiguration conf = getAccumuloConfig(configFile);
-      CryptoService cryptoService = CryptoServiceFactory.newInstance(conf);
+      CryptoService cryptoService = CryptoServiceFactory.newInstance(conf, ClassloaderType.JAVA);
       CryptoEnvironment env = new CryptoEnvironmentImpl(scope, params);
 
       FileDecrypter decrypter = cryptoService.getFileDecrypter(env);
diff --git a/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java b/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java
index fe3a83e8c1..6afa07ad8e 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java
@@ -29,6 +29,7 @@
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
@@ -85,7 +86,8 @@ public void test() throws IOException {
     // get output file name
     String suffix = FileOperations.getNewFileExtension(acuconf);
     String fname = new File(tempDir.getRoot(), testName + "." + suffix).getAbsolutePath();
-    FileSKVWriter bmfw = FileOperations.getInstance().newWriterBuilder().forFile(fname, fs, conf)
+    FileSKVWriter bmfw = FileOperations.getInstance().newWriterBuilder()
+        .forFile(fname, fs, conf, CryptoServiceFactory.newDefaultInstance())
         .withTableConfiguration(acuconf).build();
 
     // write data to file
@@ -102,7 +104,8 @@ public void test() throws IOException {
     bmfw.close();
 
     t1 = System.currentTimeMillis();
-    FileSKVIterator bmfr = FileOperations.getInstance().newReaderBuilder().forFile(fname, fs, conf)
+    FileSKVIterator bmfr = FileOperations.getInstance().newReaderBuilder()
+        .forFile(fname, fs, conf, CryptoServiceFactory.newDefaultInstance())
         .withTableConfiguration(acuconf).build();
     t2 = System.currentTimeMillis();
     log.debug("Opened {} in {}", fname, (t2 - t1));
diff --git a/core/src/test/java/org/apache/accumulo/core/file/FileOperationsTest.java b/core/src/test/java/org/apache/accumulo/core/file/FileOperationsTest.java
index a29a41316c..97bc9a982f 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/FileOperationsTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/FileOperationsTest.java
@@ -24,6 +24,7 @@
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -50,7 +51,8 @@ public void handlesFilenamesWithMoreThanOneDot() throws IOException {
       Configuration conf = new Configuration();
       FileSystem fs = FileSystem.getLocal(conf);
       AccumuloConfiguration acuconf = DefaultConfiguration.getInstance();
-      writer = fileOperations.newWriterBuilder().forFile(filename, fs, conf)
+      writer = fileOperations.newWriterBuilder()
+          .forFile(filename, fs, conf, CryptoServiceFactory.newDefaultInstance())
           .withTableConfiguration(acuconf).build();
       writer.close();
     } catch (Exception ex) {
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/CreateCompatTestFile.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/CreateCompatTestFile.java
index edb0955fa4..a42260684c 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/CreateCompatTestFile.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/CreateCompatTestFile.java
@@ -22,6 +22,7 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -60,7 +61,7 @@ public static void main(String[] args) throws Exception {
     FileSystem fs = FileSystem.get(conf);
     AccumuloConfiguration aconf = DefaultConfiguration.getInstance();
     BCFile.Writer _cbw = new BCFile.Writer(fs.create(new Path(args[0])), null, "gz", conf, aconf,
-        CryptoServiceFactory.newInstance(aconf));
+        CryptoServiceFactory.newInstance(aconf, ClassloaderType.JAVA));
     RFile.Writer writer = new RFile.Writer(_cbw, 1000);
 
     writer.startNewLocalityGroup("lg1",
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 1daff4f3a0..21436b0b9a 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
@@ -27,6 +27,7 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 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.rfile.MultiLevelIndex.BufferedWriter;
@@ -61,7 +62,7 @@ private void runTest(int maxBlockSize, int num) throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     FSDataOutputStream dos = new FSDataOutputStream(baos, new FileSystem.Statistics("a"));
     BCFile.Writer _cbw = new BCFile.Writer(dos, null, "gz", CachedConfiguration.getInstance(),
-        aconf, CryptoServiceFactory.newInstance(aconf));
+        aconf, CryptoServiceFactory.newInstance(aconf, ClassloaderType.JAVA));
 
     BufferedWriter mliw = new BufferedWriter(new Writer(_cbw, maxBlockSize));
 
@@ -82,7 +83,8 @@ private void runTest(int maxBlockSize, int num) throws IOException {
     SeekableByteArrayInputStream bais = new SeekableByteArrayInputStream(data);
     FSDataInputStream in = new FSDataInputStream(bais);
     CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in, data.length,
-        CachedConfiguration.getInstance(), aconf, CryptoServiceFactory.newInstance(aconf));
+        CachedConfiguration.getInstance(), aconf,
+        CryptoServiceFactory.newInstance(aconf, ClassloaderType.JAVA));
 
     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 ed1fa8c9d0..0b750f057b 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
@@ -40,6 +40,7 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -152,7 +153,7 @@ public void openWriter(boolean startDLG) throws IOException {
       Path path = new Path("file://" + rfile);
       dos = fs.create(path, true);
       BCFile.Writer _cbw = new BCFile.Writer(dos, null, "gz", conf, accumuloConfiguration,
-          CryptoServiceFactory.newInstance(accumuloConfiguration));
+          CryptoServiceFactory.newInstance(accumuloConfiguration, ClassloaderType.JAVA));
       SamplerConfigurationImpl samplerConfig = SamplerConfigurationImpl
           .newSamplerConfig(accumuloConfiguration);
       Sampler sampler = null;
@@ -186,7 +187,7 @@ public void openReader() throws IOException {
 
       // the caches used to obfuscate the multithreaded issues
       CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(fs, path, conf, null, null,
-          defaultConf, CryptoServiceFactory.newInstance(defaultConf));
+          defaultConf, CryptoServiceFactory.newInstance(defaultConf, ClassloaderType.JAVA));
       reader = new RFile.Reader(_cbr);
       iter = new ColumnFamilySkippingIterator(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 ece5d6aacb..40f728afab 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
@@ -56,6 +56,7 @@
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.crypto.CryptoTest;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
@@ -242,7 +243,7 @@ public void openWriter(boolean startDLG, int blockSize) throws IOException {
       baos = new ByteArrayOutputStream();
       dos = new FSDataOutputStream(baos, new FileSystem.Statistics("a"));
       BCFile.Writer _cbw = new BCFile.Writer(dos, null, "gz", conf, accumuloConfiguration,
-          CryptoServiceFactory.newInstance(accumuloConfiguration));
+          CryptoServiceFactory.newInstance(accumuloConfiguration, ClassloaderType.JAVA));
 
       SamplerConfigurationImpl samplerConfig = SamplerConfigurationImpl
           .newSamplerConfig(accumuloConfiguration);
@@ -305,7 +306,7 @@ public void openReader(boolean cfsi) throws IOException {
 
       CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader("source-1", in, fileLength, conf,
           dataCache, indexCache, accumuloConfiguration,
-          CryptoServiceFactory.newInstance(accumuloConfiguration));
+          CryptoServiceFactory.newInstance(accumuloConfiguration, ClassloaderType.JAVA));
       reader = new RFile.Reader(_cbr);
       if (cfsi)
         iter = new ColumnFamilySkippingIterator(reader);
@@ -1735,7 +1736,8 @@ private void runVersionTest(int version, AccumuloConfiguration aconf) throws IOE
     SeekableByteArrayInputStream bais = new SeekableByteArrayInputStream(data);
     FSDataInputStream in2 = new FSDataInputStream(bais);
     CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in2, data.length,
-        CachedConfiguration.getInstance(), aconf, CryptoServiceFactory.newInstance(aconf));
+        CachedConfiguration.getInstance(), aconf,
+        CryptoServiceFactory.newInstance(aconf, ClassloaderType.JAVA));
     Reader reader = new RFile.Reader(_cbr);
     checkIndex(reader);
 
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
index e7bd8f7263..41ae3683dd 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
@@ -33,6 +33,7 @@
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -195,7 +196,8 @@ private void handleWriteTests(boolean content) throws Exception {
       Configuration conf = CachedConfiguration.getInstance();
       DefaultConfiguration acuconf = DefaultConfiguration.getInstance();
       FileSKVIterator sample = RFileOperations.getInstance().newReaderBuilder()
-          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf)
+          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf,
+              CryptoServiceFactory.newDefaultInstance())
           .withTableConfiguration(acuconf).build()
           .getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
       assertNotNull(sample);
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
index 7c482a46ee..2c8c8ff550 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
@@ -30,6 +30,7 @@
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -208,7 +209,8 @@ private void handleWriteTests(boolean content) throws Exception {
       Configuration conf = CachedConfiguration.getInstance();
       DefaultConfiguration acuconf = DefaultConfiguration.getInstance();
       FileSKVIterator sample = RFileOperations.getInstance().newReaderBuilder()
-          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf)
+          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf,
+              CryptoServiceFactory.newDefaultInstance())
           .withTableConfiguration(acuconf).build()
           .getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
       assertNotNull(sample);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
index 80d6d904f5..f18a68ad6b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
@@ -34,6 +34,7 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.rpc.SslConnectionParams;
 import org.apache.accumulo.core.singletons.SingletonReservation;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
@@ -117,7 +118,7 @@ public synchronized void setupCrypto() throws CryptoService.CryptoException {
           + " already exists and cannot be setup again");
 
     AccumuloConfiguration acuConf = getConfiguration();
-    cryptoService = CryptoServiceFactory.newInstance(acuConf);
+    cryptoService = CryptoServiceFactory.newInstance(acuConf, ClassloaderType.ACCUMULO);
   }
 
   public void teardownServer() {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
index 9144f6698b..1b9323a56c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
@@ -63,6 +63,7 @@
 import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.accumulo.core.util.StopWatch;
 import org.apache.accumulo.fate.util.LoggingRunnable;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.util.FileUtil;
@@ -79,7 +80,7 @@
 
   private static final Logger log = LoggerFactory.getLogger(BulkImporter.class);
 
-  public static List<String> bulkLoad(ClientContext context, long tid, String tableId,
+  public static List<String> bulkLoad(ServerContext context, long tid, String tableId,
       List<String> files, String errorDir, boolean setTime) throws IOException, AccumuloException,
       AccumuloSecurityException, ThriftTableOperationException {
     AssignmentStats stats = new BulkImporter(context, tid, tableId, setTime).importFiles(files,
@@ -97,12 +98,12 @@
     EXAMINE_MAP_FILES, QUERY_METADATA, IMPORT_MAP_FILES, SLEEP, TOTAL
   }
 
-  private final ClientContext context;
+  private final ServerContext context;
   private String tableId;
   private long tid;
   private boolean setTime;
 
-  public BulkImporter(ClientContext context, long tid, String tableId, boolean setTime) {
+  public BulkImporter(ServerContext context, long tid, String tableId, boolean setTime) {
     this.context = context;
     this.tid = tid;
     this.tableId = tableId;
@@ -388,8 +389,8 @@ public void run() {
           Map<KeyExtent,Long> estimatedSizes = null;
 
           try {
-            estimatedSizes = FileUtil.estimateSizes(acuConf, entry.getKey(),
-                mapFileSizes.get(entry.getKey()), extentsOf(entry.getValue()), conf, vm);
+            estimatedSizes = FileUtil.estimateSizes(context, entry.getKey(),
+                mapFileSizes.get(entry.getKey()), extentsOf(entry.getValue()), conf);
           } catch (IOException e) {
             log.warn("Failed to estimate map file sizes {}", e.getMessage());
           }
@@ -650,12 +651,12 @@ public String toString() {
     }
   }
 
-  public static List<TabletLocation> findOverlappingTablets(ClientContext context, VolumeManager fs,
+  public static List<TabletLocation> findOverlappingTablets(ServerContext context, VolumeManager fs,
       TabletLocator locator, Path file) throws Exception {
     return findOverlappingTablets(context, fs, locator, file, null, null);
   }
 
-  public static List<TabletLocation> findOverlappingTablets(ClientContext context, VolumeManager fs,
+  public static List<TabletLocation> findOverlappingTablets(ServerContext context, VolumeManager fs,
       TabletLocator locator, Path file, KeyExtent failed) throws Exception {
     locator.invalidateCache(failed);
     Text start = getStartRowForExtent(failed);
@@ -675,7 +676,7 @@ protected static Text getStartRowForExtent(KeyExtent extent) {
 
   static final byte[] byte0 = {0};
 
-  public static List<TabletLocation> findOverlappingTablets(ClientContext context, VolumeManager vm,
+  public static List<TabletLocation> findOverlappingTablets(ServerContext context, VolumeManager vm,
       TabletLocator locator, Path file, Text startRow, Text endRow) throws Exception {
     List<TabletLocation> result = new ArrayList<>();
     Collection<ByteSequence> columnFamilies = Collections.emptyList();
@@ -683,8 +684,8 @@ protected static Text getStartRowForExtent(KeyExtent extent) {
     // log.debug(filename + " finding overlapping tablets " + startRow + " -> " + endRow);
     FileSystem fs = vm.getVolumeByPath(file).getFileSystem();
     try (FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-        .forFile(filename, fs, fs.getConf()).withTableConfiguration(context.getConfiguration())
-        .seekToBeginning().build()) {
+        .forFile(filename, fs, fs.getConf(), context.getCryptoService())
+        .withTableConfiguration(context.getConfiguration()).seekToBeginning().build()) {
       Text row = startRow;
       if (row == null)
         row = new Text();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index a785e6df66..805efa14b3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -47,6 +47,8 @@
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
+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.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
@@ -72,6 +74,7 @@
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
@@ -541,8 +544,11 @@ private static void createMetadataFile(VolumeManager volmanager, String fileName
       createEntriesForTablet(sorted, tablet);
     }
     FileSystem fs = volmanager.getVolumeByPath(new Path(fileName)).getFileSystem();
+
+    CryptoService cs = CryptoServiceFactory.newInstance(conf, ClassloaderType.ACCUMULO);
+
     FileSKVWriter tabletWriter = FileOperations.getInstance().newWriterBuilder()
-        .forFile(fileName, fs, fs.getConf()).withTableConfiguration(conf).build();
+        .forFile(fileName, fs, fs.getConf(), cs).withTableConfiguration(conf).build();
     tabletWriter.startDefaultLocalityGroup();
 
     for (Entry<Key,Value> entry : sorted.entrySet()) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
index 4250a3e149..74f408be5a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
@@ -46,6 +46,7 @@
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.volume.Volume;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.hadoop.conf.Configuration;
@@ -78,8 +79,11 @@ public Text getLastRow() {
 
   private static final Logger log = LoggerFactory.getLogger(FileUtil.class);
 
-  private static Path createTmpDir(AccumuloConfiguration acuConf, VolumeManager fs,
-      String tabletDirectory) throws IOException {
+  private static Path createTmpDir(ServerContext context, String tabletDirectory)
+      throws IOException {
+
+    VolumeManager fs = context.getVolumeManager();
+
     Path result = null;
     while (result == null) {
       result = new Path(tabletDirectory + Path.SEPARATOR + "tmp/idxReduce_"
@@ -105,9 +109,12 @@ private static Path createTmpDir(AccumuloConfiguration acuConf, VolumeManager fs
     return result;
   }
 
-  public static Collection<String> reduceFiles(AccumuloConfiguration acuConf, Configuration conf,
-      VolumeManager fs, Text prevEndRow, Text endRow, Collection<String> mapFiles, int maxFiles,
-      Path tmpDir, int pass) throws IOException {
+  public static Collection<String> reduceFiles(ServerContext context, Configuration conf,
+      Text prevEndRow, Text endRow, Collection<String> mapFiles, int maxFiles, Path tmpDir,
+      int pass) throws IOException {
+
+    AccumuloConfiguration acuConf = context.getConfiguration();
+
     ArrayList<String> paths = new ArrayList<>(mapFiles);
 
     if (paths.size() <= maxFiles)
@@ -130,17 +137,20 @@ private static Path createTmpDir(AccumuloConfiguration acuConf, VolumeManager fs
       String newMapFile = String.format("%s/%04d.%s", newDir, count++, RFile.EXTENSION);
 
       outFiles.add(newMapFile);
-      FileSystem ns = fs.getVolumeByPath(new Path(newMapFile)).getFileSystem();
+      FileSystem ns = context.getVolumeManager().getVolumeByPath(new Path(newMapFile))
+          .getFileSystem();
       FileSKVWriter writer = new RFileOperations().newWriterBuilder()
-          .forFile(newMapFile.toString(), ns, ns.getConf()).withTableConfiguration(acuConf).build();
+          .forFile(newMapFile.toString(), ns, ns.getConf(), context.getCryptoService())
+          .withTableConfiguration(acuConf).build();
       writer.startDefaultLocalityGroup();
       List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<>(inFiles.size());
 
       FileSKVIterator reader = null;
       try {
         for (String s : inFiles) {
-          ns = fs.getVolumeByPath(new Path(s)).getFileSystem();
-          reader = FileOperations.getInstance().newIndexReaderBuilder().forFile(s, ns, ns.getConf())
+          ns = context.getVolumeManager().getVolumeByPath(new Path(s)).getFileSystem();
+          reader = FileOperations.getInstance().newIndexReaderBuilder()
+              .forFile(s, ns, ns.getConf(), context.getCryptoService())
               .withTableConfiguration(acuConf).build();
           iters.add(reader);
         }
@@ -187,36 +197,35 @@ private static Path createTmpDir(AccumuloConfiguration acuConf, VolumeManager fs
       }
     }
 
-    return reduceFiles(acuConf, conf, fs, prevEndRow, endRow, outFiles, maxFiles, tmpDir, pass + 1);
+    return reduceFiles(context, conf, prevEndRow, endRow, outFiles, maxFiles, tmpDir, pass + 1);
   }
 
-  public static SortedMap<Double,Key> findMidPoint(VolumeManager fs, String tabletDir,
-      AccumuloConfiguration acuConf, Text prevEndRow, Text endRow, Collection<String> mapFiles,
-      double minSplit) throws IOException {
-    return findMidPoint(fs, tabletDir, acuConf, prevEndRow, endRow, mapFiles, minSplit, true);
+  public static SortedMap<Double,Key> findMidPoint(ServerContext context, String tabletDir,
+      Text prevEndRow, Text endRow, Collection<String> mapFiles, double minSplit)
+      throws IOException {
+    return findMidPoint(context, tabletDir, prevEndRow, endRow, mapFiles, minSplit, true);
   }
 
-  public static double estimatePercentageLTE(VolumeManager fs, String tabletDir,
-      AccumuloConfiguration acuconf, Text prevEndRow, Text endRow, Collection<String> mapFiles,
-      Text splitRow) throws IOException {
+  public static double estimatePercentageLTE(ServerContext context, String tabletDir,
+      Text prevEndRow, Text endRow, Collection<String> mapFiles, Text splitRow) throws IOException {
 
     Configuration conf = CachedConfiguration.getInstance();
 
     Path tmpDir = null;
 
-    int maxToOpen = acuconf.getCount(Property.TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN);
+    int maxToOpen = context.getConfiguration()
+        .getCount(Property.TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN);
     ArrayList<FileSKVIterator> readers = new ArrayList<>(mapFiles.size());
 
     try {
       if (mapFiles.size() > maxToOpen) {
-        tmpDir = createTmpDir(acuconf, fs, tabletDir);
+        tmpDir = createTmpDir(context, tabletDir);
 
         log.debug("Too many indexes ({}) to open at once for {} {}, reducing in tmpDir = {}",
             mapFiles.size(), endRow, prevEndRow, tmpDir);
 
         long t1 = System.currentTimeMillis();
-        mapFiles = reduceFiles(acuconf, conf, fs, prevEndRow, endRow, mapFiles, maxToOpen, tmpDir,
-            0);
+        mapFiles = reduceFiles(context, conf, prevEndRow, endRow, mapFiles, maxToOpen, tmpDir, 0);
         long t2 = System.currentTimeMillis();
 
         log.debug("Finished reducing indexes for {} {} in {}", endRow, prevEndRow,
@@ -228,7 +237,7 @@ public static double estimatePercentageLTE(VolumeManager fs, String tabletDir,
 
       long numKeys = 0;
 
-      numKeys = countIndexEntries(acuconf, prevEndRow, endRow, mapFiles, true, conf, fs, readers);
+      numKeys = countIndexEntries(context, prevEndRow, endRow, mapFiles, true, conf, readers);
 
       if (numKeys == 0) {
         // not enough info in the index to answer the question, so instead of going to
@@ -261,7 +270,7 @@ public static double estimatePercentageLTE(VolumeManager fs, String tabletDir,
       return (numLte + 1) / (double) (numKeys + 2);
 
     } finally {
-      cleanupIndexOp(tmpDir, fs, readers);
+      cleanupIndexOp(tmpDir, context.getVolumeManager(), readers);
     }
   }
 
@@ -275,16 +284,17 @@ public static double estimatePercentageLTE(VolumeManager fs, String tabletDir,
    *          would be tricky to use this method in conjunction with an in memory map because the
    *          indexing interval is unknown.
    */
-  public static SortedMap<Double,Key> findMidPoint(VolumeManager fs, String tabletDirectory,
-      AccumuloConfiguration acuConf, Text prevEndRow, Text endRow, Collection<String> mapFiles,
-      double minSplit, boolean useIndex) throws IOException {
+  public static SortedMap<Double,Key> findMidPoint(ServerContext context, String tabletDirectory,
+      Text prevEndRow, Text endRow, Collection<String> mapFiles, double minSplit, boolean useIndex)
+      throws IOException {
     Configuration conf = CachedConfiguration.getInstance();
 
     Collection<String> origMapFiles = mapFiles;
 
     Path tmpDir = null;
 
-    int maxToOpen = acuConf.getCount(Property.TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN);
+    int maxToOpen = context.getConfiguration()
+        .getCount(Property.TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN);
     ArrayList<FileSKVIterator> readers = new ArrayList<>(mapFiles.size());
 
     try {
@@ -292,14 +302,13 @@ public static double estimatePercentageLTE(VolumeManager fs, String tabletDir,
         if (!useIndex)
           throw new IOException(
               "Cannot find mid point using data files, too many " + mapFiles.size());
-        tmpDir = createTmpDir(acuConf, fs, tabletDirectory);
+        tmpDir = createTmpDir(context, tabletDirectory);
 
         log.debug("Too many indexes ({}) to open at once for {} {}, reducing in tmpDir = {}",
             mapFiles.size(), endRow, prevEndRow, tmpDir);
 
         long t1 = System.currentTimeMillis();
-        mapFiles = reduceFiles(acuConf, conf, fs, prevEndRow, endRow, mapFiles, maxToOpen, tmpDir,
-            0);
+        mapFiles = reduceFiles(context, conf, prevEndRow, endRow, mapFiles, maxToOpen, tmpDir, 0);
         long t2 = System.currentTimeMillis();
 
         log.debug("Finished reducing indexes for {} {} in {}", endRow, prevEndRow,
@@ -313,8 +322,8 @@ public static double estimatePercentageLTE(VolumeManager fs, String tabletDir,
 
       long numKeys = 0;
 
-      numKeys = countIndexEntries(acuConf, prevEndRow, endRow, mapFiles,
-          tmpDir == null ? useIndex : false, conf, fs, readers);
+      numKeys = countIndexEntries(context, prevEndRow, endRow, mapFiles,
+          tmpDir == null ? useIndex : false, conf, readers);
 
       if (numKeys == 0) {
         if (useIndex) {
@@ -323,8 +332,8 @@ public static double estimatePercentageLTE(VolumeManager fs, String tabletDir,
                   + " data files which is slower. No entries between {} and {} for {}",
               prevEndRow, endRow, mapFiles);
           // need to pass original map files, not possibly reduced indexes
-          return findMidPoint(fs, tabletDirectory, acuConf, prevEndRow, endRow, origMapFiles,
-              minSplit, false);
+          return findMidPoint(context, tabletDirectory, prevEndRow, endRow, origMapFiles, minSplit,
+              false);
         }
         throw new IOException("Failed to find mid point, no entries between " + prevEndRow + " and "
             + endRow + " for " + mapFiles);
@@ -385,7 +394,7 @@ public static double estimatePercentageLTE(VolumeManager fs, String tabletDir,
 
       return ret;
     } finally {
-      cleanupIndexOp(tmpDir, fs, readers);
+      cleanupIndexOp(tmpDir, context.getVolumeManager(), readers);
     }
   }
 
@@ -413,26 +422,29 @@ protected static void cleanupIndexOp(Path tmpDir, VolumeManager fs,
     }
   }
 
-  private static long countIndexEntries(AccumuloConfiguration acuConf, Text prevEndRow, Text endRow,
-      Collection<String> mapFiles, boolean useIndex, Configuration conf, VolumeManager fs,
+  private static long countIndexEntries(ServerContext context, Text prevEndRow, Text endRow,
+      Collection<String> mapFiles, boolean useIndex, Configuration conf,
       ArrayList<FileSKVIterator> readers) throws IOException {
 
+    AccumuloConfiguration acuConf = context.getConfiguration();
+
     long numKeys = 0;
 
     // count the total number of index entries
     for (String ref : mapFiles) {
       FileSKVIterator reader = null;
       Path path = new Path(ref);
-      FileSystem ns = fs.getVolumeByPath(path).getFileSystem();
+      FileSystem ns = context.getVolumeManager().getVolumeByPath(path).getFileSystem();
       try {
         if (useIndex)
           reader = FileOperations.getInstance().newIndexReaderBuilder()
-              .forFile(path.toString(), ns, ns.getConf()).withTableConfiguration(acuConf).build();
+              .forFile(path.toString(), ns, ns.getConf(), context.getCryptoService())
+              .withTableConfiguration(acuConf).build();
         else
           reader = FileOperations.getInstance().newScanReaderBuilder()
-              .forFile(path.toString(), ns, ns.getConf()).withTableConfiguration(acuConf)
-              .overRange(new Range(prevEndRow, false, null, true), LocalityGroupUtil.EMPTY_CF_SET,
-                  false)
+              .forFile(path.toString(), ns, ns.getConf(), context.getCryptoService())
+              .withTableConfiguration(acuConf).overRange(new Range(prevEndRow, false, null, true),
+                  LocalityGroupUtil.EMPTY_CF_SET, false)
               .build();
 
         while (reader.hasTop()) {
@@ -455,20 +467,21 @@ else if (prevEndRow == null || key.compareRow(prevEndRow) > 0)
 
       if (useIndex)
         readers.add(FileOperations.getInstance().newIndexReaderBuilder()
-            .forFile(path.toString(), ns, ns.getConf()).withTableConfiguration(acuConf).build());
+            .forFile(path.toString(), ns, ns.getConf(), context.getCryptoService())
+            .withTableConfiguration(acuConf).build());
       else
         readers.add(FileOperations.getInstance().newScanReaderBuilder()
-            .forFile(path.toString(), ns, ns.getConf()).withTableConfiguration(acuConf)
-            .overRange(new Range(prevEndRow, false, null, true), LocalityGroupUtil.EMPTY_CF_SET,
-                false)
+            .forFile(path.toString(), ns, ns.getConf(), context.getCryptoService())
+            .withTableConfiguration(acuConf).overRange(new Range(prevEndRow, false, null, true),
+                LocalityGroupUtil.EMPTY_CF_SET, false)
             .build());
 
     }
     return numKeys;
   }
 
-  public static Map<FileRef,FileInfo> tryToGetFirstAndLastRows(VolumeManager fs,
-      AccumuloConfiguration acuConf, Set<FileRef> mapfiles) {
+  public static Map<FileRef,FileInfo> tryToGetFirstAndLastRows(ServerContext context,
+      Set<FileRef> mapfiles) {
 
     HashMap<FileRef,FileInfo> mapFilesInfo = new HashMap<>();
 
@@ -477,10 +490,11 @@ else if (prevEndRow == null || key.compareRow(prevEndRow) > 0)
     for (FileRef mapfile : mapfiles) {
 
       FileSKVIterator reader = null;
-      FileSystem ns = fs.getVolumeByPath(mapfile.path()).getFileSystem();
+      FileSystem ns = context.getVolumeManager().getVolumeByPath(mapfile.path()).getFileSystem();
       try {
         reader = FileOperations.getInstance().newReaderBuilder()
-            .forFile(mapfile.toString(), ns, ns.getConf()).withTableConfiguration(acuConf).build();
+            .forFile(mapfile.toString(), ns, ns.getConf(), context.getCryptoService())
+            .withTableConfiguration(context.getConfiguration()).build();
 
         Key firstKey = reader.getFirstKey();
         if (firstKey != null) {
@@ -509,16 +523,17 @@ else if (prevEndRow == null || key.compareRow(prevEndRow) > 0)
     return mapFilesInfo;
   }
 
-  public static WritableComparable<Key> findLastKey(VolumeManager fs, AccumuloConfiguration acuConf,
+  public static WritableComparable<Key> findLastKey(ServerContext context,
       Collection<FileRef> mapFiles) throws IOException {
+
     Key lastKey = null;
 
     for (FileRef ref : mapFiles) {
       Path path = ref.path();
-      FileSystem ns = fs.getVolumeByPath(path).getFileSystem();
+      FileSystem ns = context.getVolumeManager().getVolumeByPath(path).getFileSystem();
       FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-          .forFile(path.toString(), ns, ns.getConf()).withTableConfiguration(acuConf)
-          .seekToBeginning().build();
+          .forFile(path.toString(), ns, ns.getConf(), context.getCryptoService())
+          .withTableConfiguration(context.getConfiguration()).seekToBeginning().build();
 
       try {
         if (!reader.hasTop())
@@ -543,12 +558,12 @@ else if (prevEndRow == null || key.compareRow(prevEndRow) > 0)
 
   }
 
-  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path mapFile,
-      long fileSize, List<KeyExtent> extents, Configuration conf, VolumeManager fs)
-      throws IOException {
+  public static Map<KeyExtent,Long> estimateSizes(ServerContext context, Path mapFile,
+      long fileSize, List<KeyExtent> extents, Configuration conf) throws IOException {
 
-    FileSystem ns = fs.getVolumeByPath(mapFile).getFileSystem();
-    return BulkImport.estimateSizes(acuConf, mapFile, fileSize, extents, ns, null);
+    FileSystem ns = context.getVolumeManager().getVolumeByPath(mapFile).getFileSystem();
+    return BulkImport.estimateSizes(context.getConfiguration(), mapFile, fileSize, extents, ns,
+        null, context.getCryptoService());
   }
 
   public static Collection<String> toPathStrings(Collection<FileRef> refs) {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java b/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
index 9ebcabdee4..75eb277078 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
@@ -34,6 +34,7 @@
 import org.apache.accumulo.core.clientImpl.TabletLocator;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
@@ -42,6 +43,7 @@
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.commons.lang.NotImplementedException;
@@ -114,14 +116,17 @@ public void invalidateCache(ClientContext context, String server) {
   public void testFindOverlappingTablets() throws Exception {
     MockTabletLocator locator = new MockTabletLocator();
     FileSystem fs = FileSystem.getLocal(CachedConfiguration.getInstance());
-    ClientContext context = EasyMock.createMock(ClientContext.class);
+    ServerContext context = EasyMock.createMock(ServerContext.class);
     EasyMock.expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance())
         .anyTimes();
+    EasyMock.expect(context.getCryptoService()).andReturn(CryptoServiceFactory.newDefaultInstance())
+        .anyTimes();
     EasyMock.replay(context);
     String file = "target/testFile.rf";
     fs.delete(new Path(file), true);
     FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder()
-        .forFile(file, fs, fs.getConf()).withTableConfiguration(context.getConfiguration()).build();
+        .forFile(file, fs, fs.getConf(), CryptoServiceFactory.newDefaultInstance())
+        .withTableConfiguration(context.getConfiguration()).build();
     writer.startDefaultLocalityGroup();
     Value empty = new Value(new byte[] {});
     writer.append(new Key("a", "cf", "cq"), empty);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
index 10659209a9..ba87d68847 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
@@ -323,11 +323,10 @@ private void closeReaders(Collection<FileSKVIterator> filesToClose) {
         FileSystem ns = fs.getVolumeByPath(path).getFileSystem();
         // log.debug("Opening "+file + " path " + path);
         FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-            .forFile(path.toString(), ns, ns.getConf())
+            .forFile(path.toString(), ns, ns.getConf(), context.getCryptoService())
             .withTableConfiguration(
                 context.getServerConfFactory().getTableConfiguration(tablet.getTableId()))
-            .withBlockCache(dataCache, indexCache).withFileLenCache(fileLenCache)
-            .withCryptoService(context.getCryptoService()).build();
+            .withBlockCache(dataCache, indexCache).withFileLenCache(fileLenCache).build();
         readersReserved.put(reader, file);
       } catch (Exception e) {
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
index 23bd6c91dc..e93e7f60b8 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
@@ -641,9 +641,9 @@ private synchronized FileSKVIterator getReader() throws IOException {
         Configuration conf = CachedConfiguration.getInstance();
         FileSystem fs = FileSystem.getLocal(conf);
 
-        reader = new RFileOperations().newReaderBuilder().forFile(memDumpFile, fs, conf)
-            .withTableConfiguration(context.getConfiguration())
-            .withCryptoService(context.getCryptoService()).seekToBeginning().build();
+        reader = new RFileOperations().newReaderBuilder()
+            .forFile(memDumpFile, fs, conf, context.getCryptoService())
+            .withTableConfiguration(context.getConfiguration()).seekToBeginning().build();
         if (iflag != null)
           reader.setInterruptFlag(iflag);
 
@@ -813,8 +813,9 @@ public void delete(long waitTime) {
           aconf = createSampleConfig(aconf);
         }
 
-        FileSKVWriter out = new RFileOperations().newWriterBuilder().forFile(tmpFile, fs, newConf)
-            .withTableConfiguration(aconf).withCryptoService(context.getCryptoService()).build();
+        FileSKVWriter out = new RFileOperations().newWriterBuilder()
+            .forFile(tmpFile, fs, newConf, context.getCryptoService()).withTableConfiguration(aconf)
+            .build();
 
         InterruptibleIterator iter = map.skvIterator(null);
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
index 91890231de..44c3b7e61f 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
@@ -178,7 +178,8 @@ public FileSKVIterator openReader(FileRef ref) throws IOException {
     // @TODO ensure these files are always closed?
     FileOperations fileFactory = FileOperations.getInstance();
     FileSystem ns = volumeManager.getVolumeByPath(ref.path()).getFileSystem();
-    return fileFactory.newReaderBuilder().forFile(ref.path().toString(), ns, ns.getConf())
+    return fileFactory.newReaderBuilder()
+        .forFile(ref.path().toString(), ns, ns.getConf(), context.getCryptoService())
         .withTableConfiguration(tableConfig).seekToBeginning().build();
   }
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index dd5824d940..cdb787feb6 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -17,7 +17,6 @@
 package org.apache.accumulo.tserver.log;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.cryptoImpl.CryptoEnvironmentImpl.Scope;
 import static org.apache.accumulo.tserver.logger.LogEvents.COMPACTION_FINISH;
 import static org.apache.accumulo.tserver.logger.LogEvents.COMPACTION_START;
 import static org.apache.accumulo.tserver.logger.LogEvents.DEFINE_TABLET;
@@ -43,6 +42,7 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.crypto.CryptoUtils;
 import org.apache.accumulo.core.crypto.streams.NoFlushOutputStream;
 import org.apache.accumulo.core.cryptoImpl.CryptoEnvironmentImpl;
@@ -50,6 +50,7 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
+import org.apache.accumulo.core.spi.crypto.CryptoEnvironment.Scope;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.spi.crypto.FileDecrypter;
 import org.apache.accumulo.core.spi.crypto.FileEncrypter;
@@ -370,7 +371,8 @@ public static DFSLoggerInputStreams readHeaderAndReturnStream(FSDataInputStream
       input.readFully(magicBuffer);
       if (Arrays.equals(magicBuffer, magic)) {
         byte[] params = CryptoUtils.readParams(input);
-        CryptoService cryptoService = CryptoServiceFactory.newInstance(conf);
+        CryptoService cryptoService = CryptoServiceFactory.newInstance(conf,
+            ClassloaderType.ACCUMULO);
         CryptoEnvironment env = new CryptoEnvironmentImpl(Scope.WAL, params);
 
         FileDecrypter decrypter = cryptoService.getFileDecrypter(env);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
index b7b6eda9db..fbc988da07 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
@@ -200,9 +200,9 @@ public CompactionStats call() throws IOException, CompactionCanceledException {
     try {
       FileOperations fileFactory = FileOperations.getInstance();
       FileSystem ns = this.fs.getVolumeByPath(outputFilePath).getFileSystem();
-      mfw = fileFactory.newWriterBuilder().forFile(outputFilePathName, ns, ns.getConf())
-          .withTableConfiguration(acuTableConf).withRateLimiter(env.getWriteLimiter())
-          .withCryptoService(context.getCryptoService()).build();
+      mfw = fileFactory.newWriterBuilder()
+          .forFile(outputFilePathName, ns, ns.getConf(), context.getCryptoService())
+          .withTableConfiguration(acuTableConf).withRateLimiter(env.getWriteLimiter()).build();
 
       Map<String,Set<ByteSequence>> lGroups;
       try {
@@ -290,9 +290,9 @@ public CompactionStats call() throws IOException, CompactionCanceledException {
         FileSystem fs = this.fs.getVolumeByPath(mapFile.path()).getFileSystem();
         FileSKVIterator reader;
 
-        reader = fileFactory.newReaderBuilder().forFile(mapFile.path().toString(), fs, fs.getConf())
-            .withTableConfiguration(acuTableConf).withRateLimiter(env.getReadLimiter())
-            .withCryptoService(context.getCryptoService()).build();
+        reader = fileFactory.newReaderBuilder()
+            .forFile(mapFile.path().toString(), fs, fs.getConf(), context.getCryptoService())
+            .withTableConfiguration(acuTableConf).withRateLimiter(env.getReadLimiter()).build();
 
         readers.add(reader);
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 9cde1b163f..19d9bb2a9a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -1628,9 +1628,8 @@ private SplitRowSpec findSplitRow(Collection<FileRef> files) {
 
     try {
       // we should make .25 below configurable
-      keys = FileUtil.findMidPoint(getTabletServer().getFileSystem(), tabletDirectory,
-          getTabletServer().getConfiguration(), extent.getPrevEndRow(), extent.getEndRow(),
-          FileUtil.toPathStrings(files), .25);
+      keys = FileUtil.findMidPoint(context, tabletDirectory, extent.getPrevEndRow(),
+          extent.getEndRow(), FileUtil.toPathStrings(files), .25);
     } catch (IOException e) {
       log.error("Failed to find midpoint {}", e.getMessage());
       return null;
@@ -1641,8 +1640,7 @@ private SplitRowSpec findSplitRow(Collection<FileRef> files) {
 
       Text lastRow;
       if (extent.getEndRow() == null) {
-        Key lastKey = (Key) FileUtil.findLastKey(getTabletServer().getFileSystem(),
-            getTabletServer().getConfiguration(), files);
+        Key lastKey = (Key) FileUtil.findLastKey(context, files);
         lastRow = lastKey.getRow();
       } else {
         lastRow = extent.getEndRow();
@@ -1736,7 +1734,7 @@ private static int longestCommonLength(Text text, Text beforeMid) {
       FileRef file = entry.getKey();
       FileSystem ns = fs.getVolumeByPath(file.path()).getFileSystem();
       try (FileSKVIterator openReader = fileFactory.newReaderBuilder()
-          .forFile(file.path().toString(), ns, ns.getConf())
+          .forFile(file.path().toString(), ns, ns.getConf(), context.getCryptoService())
           .withTableConfiguration(this.getTableConfiguration()).seekToBeginning().build()) {
         Key first = openReader.getFirstKey();
         Key last = openReader.getLastKey();
@@ -2265,8 +2263,7 @@ public boolean isMajorCompactionQueued() {
     // this info is used for optimization... it is ok if map files are missing
     // from the set... can still query and insert into the tablet while this
     // map file operation is happening
-    Map<FileRef,FileUtil.FileInfo> firstAndLastRows = FileUtil.tryToGetFirstAndLastRows(
-        getTabletServer().getFileSystem(), getTabletServer().getConfiguration(),
+    Map<FileRef,FileUtil.FileInfo> firstAndLastRows = FileUtil.tryToGetFirstAndLastRows(context,
         getDatafileManager().getFiles());
 
     synchronized (this) {
@@ -2281,9 +2278,8 @@ public boolean isMajorCompactionQueued() {
       else {
         Text tsp = new Text(sp);
         splitPoint = new SplitRowSpec(
-            FileUtil.estimatePercentageLTE(getTabletServer().getFileSystem(), tabletDirectory,
-                getTabletServer().getConfiguration(), extent.getPrevEndRow(), extent.getEndRow(),
-                FileUtil.toPathStrings(getDatafileManager().getFiles()), tsp),
+            FileUtil.estimatePercentageLTE(context, tabletDirectory, extent.getPrevEndRow(),
+                extent.getEndRow(), FileUtil.toPathStrings(getDatafileManager().getFiles()), tsp),
             tsp);
       }
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
index 557a6d0c1f..5734747f8c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
@@ -155,8 +155,8 @@ public TabletData(ServerContext context, VolumeManager fs, ZooReader rdr,
       FileSystem ns = fs.getVolumeByPath(path).getFileSystem();
       long maxTime = -1;
       try (FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-          .forFile(path.toString(), ns, ns.getConf()).withTableConfiguration(conf).seekToBeginning()
-          .withCryptoService(context.getCryptoService()).build()) {
+          .forFile(path.toString(), ns, ns.getConf(), context.getCryptoService())
+          .withTableConfiguration(conf).seekToBeginning().build()) {
         while (reader.hasTop()) {
           maxTime = Math.max(maxTime, reader.getTopKey().getTimestamp());
           reader.next();
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java
index f449e46e26..1b828876a9 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java
@@ -44,7 +44,7 @@
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.cryptoImpl.NoCryptoService;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -110,7 +110,8 @@ public static void setUp() throws Exception {
 
   public static ServerContext getServerContext() {
     ServerContext context = EasyMock.createMock(ServerContext.class);
-    EasyMock.expect(context.getCryptoService()).andReturn(new NoCryptoService()).anyTimes();
+    EasyMock.expect(context.getCryptoService()).andReturn(CryptoServiceFactory.newDefaultInstance())
+        .anyTimes();
     EasyMock.expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance())
         .anyTimes();
     EasyMock.replay(context);
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java
index e4d95a2da8..a4efc22d9b 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java
@@ -31,7 +31,7 @@
 
 import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.cryptoImpl.NoCryptoService;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
@@ -64,7 +64,8 @@
 
   public static ServerContext getServerContext() {
     ServerContext context = EasyMock.createMock(ServerContext.class);
-    EasyMock.expect(context.getCryptoService()).andReturn(new NoCryptoService()).anyTimes();
+    EasyMock.expect(context.getCryptoService()).andReturn(CryptoServiceFactory.newDefaultInstance())
+        .anyTimes();
     EasyMock.replay(context);
     return context;
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/BulkImportMonitoringIT.java b/test/src/main/java/org/apache/accumulo/test/BulkImportMonitoringIT.java
index 96ba085145..84ed7f6d1f 100644
--- a/test/src/main/java/org/apache/accumulo/test/BulkImportMonitoringIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BulkImportMonitoringIT.java
@@ -31,6 +31,7 @@
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
@@ -97,7 +98,8 @@ public void test() throws Exception {
           fs.mkdirs(files);
           for (int i1 = 0; i1 < 10; i1++) {
             FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder()
-                .forFile(files + "/bulk_" + i1 + "." + RFile.EXTENSION, fs, fs.getConf())
+                .forFile(files + "/bulk_" + i1 + "." + RFile.EXTENSION, fs, fs.getConf(),
+                    CryptoServiceFactory.newDefaultInstance())
                 .withTableConfiguration(DefaultConfiguration.getInstance()).build();
             writer.startDefaultLocalityGroup();
             for (int j = 0x100; j < 0xfff; j += 3) {
diff --git a/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java b/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
index 477aba4131..576a2aaa19 100644
--- a/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
@@ -34,6 +34,7 @@
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
@@ -127,7 +128,8 @@ public void compareOldNewBulkImportTest() throws Exception {
           fs.mkdirs(files);
           for (int i1 = 0; i1 < 100; i1++) {
             FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder()
-                .forFile(files + "/bulk_" + i1 + "." + RFile.EXTENSION, fs, fs.getConf())
+                .forFile(files + "/bulk_" + i1 + "." + RFile.EXTENSION, fs, fs.getConf(),
+                    CryptoServiceFactory.newDefaultInstance())
                 .withTableConfiguration(DefaultConfiguration.getInstance()).build();
             writer.startDefaultLocalityGroup();
             for (int j = 0x100; j < 0xfff; j += 3) {
diff --git a/test/src/main/java/org/apache/accumulo/test/CreateRandomRFile.java b/test/src/main/java/org/apache/accumulo/test/CreateRandomRFile.java
index 7431eb8c9c..0f10bab18e 100644
--- a/test/src/main/java/org/apache/accumulo/test/CreateRandomRFile.java
+++ b/test/src/main/java/org/apache/accumulo/test/CreateRandomRFile.java
@@ -22,6 +22,7 @@
 import java.util.Random;
 
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileSKVWriter;
@@ -71,7 +72,8 @@ public static void main(String[] args) {
     FileSKVWriter mfw;
     try {
       FileSystem fs = FileSystem.get(conf);
-      mfw = new RFileOperations().newWriterBuilder().forFile(file, fs, conf)
+      mfw = new RFileOperations().newWriterBuilder()
+          .forFile(file, fs, conf, CryptoServiceFactory.newDefaultInstance())
           .withTableConfiguration(DefaultConfiguration.getInstance()).build();
     } catch (IOException e) {
       throw new RuntimeException(e);
diff --git a/test/src/main/java/org/apache/accumulo/test/GenerateSequentialRFile.java b/test/src/main/java/org/apache/accumulo/test/GenerateSequentialRFile.java
index 8c90ae3fc2..a1f55687e4 100644
--- a/test/src/main/java/org/apache/accumulo/test/GenerateSequentialRFile.java
+++ b/test/src/main/java/org/apache/accumulo/test/GenerateSequentialRFile.java
@@ -20,6 +20,7 @@
 
 import org.apache.accumulo.core.cli.Help;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
@@ -60,7 +61,7 @@ public void run() {
       Path p = new Path(opts.filePath);
       final FileSystem fs = p.getFileSystem(conf);
       FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder()
-          .forFile(opts.filePath, fs, conf)
+          .forFile(opts.filePath, fs, conf, CryptoServiceFactory.newDefaultInstance())
           .withTableConfiguration(DefaultConfiguration.getInstance()).build();
 
       writer.startDefaultLocalityGroup();
diff --git a/test/src/main/java/org/apache/accumulo/test/InMemoryMapIT.java b/test/src/main/java/org/apache/accumulo/test/InMemoryMapIT.java
index d51ed74901..99aa185fc8 100644
--- a/test/src/main/java/org/apache/accumulo/test/InMemoryMapIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/InMemoryMapIT.java
@@ -34,7 +34,7 @@
 
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.cryptoImpl.NoCryptoService;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -104,7 +104,8 @@ public static void ensureNativeLibrary() throws FileNotFoundException {
 
   public static ServerContext getServerContext() {
     ServerContext context = EasyMock.createMock(ServerContext.class);
-    EasyMock.expect(context.getCryptoService()).andReturn(new NoCryptoService()).anyTimes();
+    EasyMock.expect(context.getCryptoService()).andReturn(CryptoServiceFactory.newDefaultInstance())
+        .anyTimes();
     EasyMock.replay(context);
     return context;
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index 893857c371..a09a32d29b 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@ -73,6 +73,7 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
@@ -1398,9 +1399,11 @@ public void importDirectory() throws Exception {
     fs.mkdirs(new Path(errorsDir.toString()));
     AccumuloConfiguration aconf = DefaultConfiguration.getInstance();
     FileSKVWriter evenWriter = FileOperations.getInstance().newWriterBuilder()
-        .forFile(even, fs, conf).withTableConfiguration(aconf).build();
+        .forFile(even, fs, conf, CryptoServiceFactory.newDefaultInstance())
+        .withTableConfiguration(aconf).build();
     evenWriter.startDefaultLocalityGroup();
-    FileSKVWriter oddWriter = FileOperations.getInstance().newWriterBuilder().forFile(odd, fs, conf)
+    FileSKVWriter oddWriter = FileOperations.getInstance().newWriterBuilder()
+        .forFile(odd, fs, conf, CryptoServiceFactory.newDefaultInstance())
         .withTableConfiguration(aconf).build();
     oddWriter.startDefaultLocalityGroup();
     long timestamp = System.currentTimeMillis();
diff --git a/test/src/main/java/org/apache/accumulo/test/TestIngest.java b/test/src/main/java/org/apache/accumulo/test/TestIngest.java
index c0790a452b..6fc7b6d05f 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestIngest.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestIngest.java
@@ -36,6 +36,7 @@
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.clientImpl.TabletServerBatchWriter;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.ConstraintViolationSummary;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -230,7 +231,8 @@ public static void ingest(AccumuloClient accumuloClient, FileSystem fs, Opts opt
     if (opts.outputFile != null) {
       Configuration conf = CachedConfiguration.getInstance();
       writer = FileOperations.getInstance().newWriterBuilder()
-          .forFile(opts.outputFile + "." + RFile.EXTENSION, fs, conf)
+          .forFile(opts.outputFile + "." + RFile.EXTENSION, fs, conf,
+              CryptoServiceFactory.newDefaultInstance())
           .withTableConfiguration(DefaultConfiguration.getInstance()).build();
       writer.startDefaultLocalityGroup();
     } else {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkFileIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkFileIT.java
index d814825000..26af281c1e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkFileIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkFileIT.java
@@ -27,6 +27,7 @@
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
@@ -105,8 +106,9 @@ public void testBulkFile() throws Exception {
   private void writeData(Configuration conf, AccumuloConfiguration aconf, FileSystem fs, String dir,
       String file, int start, int end) throws IOException, Exception {
     FileSKVWriter writer1 = FileOperations.getInstance().newWriterBuilder()
-        .forFile(dir + "/" + file + "." + RFile.EXTENSION, fs, conf).withTableConfiguration(aconf)
-        .build();
+        .forFile(dir + "/" + file + "." + RFile.EXTENSION, fs, conf,
+            CryptoServiceFactory.newDefaultInstance())
+        .withTableConfiguration(aconf).build();
     writer1.startDefaultLocalityGroup();
     for (int i = start; i <= end; i++) {
       writer1.append(new Key(new Text(String.format("%04d", i))),
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkLoadIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkLoadIT.java
index 24fd475522..3fe673ef51 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkLoadIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkLoadIT.java
@@ -44,6 +44,7 @@
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.LoadPlan;
 import org.apache.accumulo.core.data.LoadPlan.RangeType;
@@ -415,7 +416,8 @@ private String writeData(String file, AccumuloConfiguration aconf, int s, int e)
     FileSystem fs = getCluster().getFileSystem();
     String filename = file + RFile.EXTENSION;
     try (FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder()
-        .forFile(filename, fs, fs.getConf()).withTableConfiguration(aconf).build()) {
+        .forFile(filename, fs, fs.getConf(), CryptoServiceFactory.newDefaultInstance())
+        .withTableConfiguration(aconf).build()) {
       writer.startDefaultLocalityGroup();
       for (int i = s; i <= e; i++) {
         writer.append(new Key(new Text(row(i))), new Value(Integer.toString(i).getBytes(UTF_8)));
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
index a7a3815ee2..7e1ccc2fa1 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
@@ -35,6 +35,7 @@
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -194,7 +195,8 @@ private void handleWriteTests(boolean content) throws Exception {
       Configuration conf = CachedConfiguration.getInstance();
       DefaultConfiguration acuconf = DefaultConfiguration.getInstance();
       FileSKVIterator sample = RFileOperations.getInstance().newReaderBuilder()
-          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf)
+          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf,
+              CryptoServiceFactory.newDefaultInstance())
           .withTableConfiguration(acuconf).build()
           .getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
       assertNotNull(sample);
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
index 49095a6edc..92118e6ae2 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
@@ -32,6 +32,7 @@
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -207,7 +208,8 @@ private void handleWriteTests(boolean content) throws Exception {
       Configuration conf = CachedConfiguration.getInstance();
       DefaultConfiguration acuconf = DefaultConfiguration.getInstance();
       FileSKVIterator sample = RFileOperations.getInstance().newReaderBuilder()
-          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf)
+          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf,
+              CryptoServiceFactory.newDefaultInstance())
           .withTableConfiguration(acuconf).build()
           .getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
       assertNotNull(sample);
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
index a0e3f1a341..cb3906dc2b 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
@@ -41,6 +41,7 @@
 import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Column;
@@ -467,7 +468,9 @@ private static int readFiles(VolumeManager fs, AccumuloConfiguration aconf, List
     for (FileRef file : files) {
       FileSystem ns = fs.getVolumeByPath(file.path()).getFileSystem();
       FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-          .forFile(file.path().toString(), ns, ns.getConf()).withTableConfiguration(aconf).build();
+          .forFile(file.path().toString(), ns, ns.getConf(),
+              CryptoServiceFactory.newDefaultInstance())
+          .withTableConfiguration(aconf).build();
       Range range = new Range(ke.getPrevEndRow(), false, ke.getEndRow(), true);
       reader.seek(range, columnSet, columnSet.size() != 0);
       while (reader.hasTop() && !range.afterEndKey(reader.getTopKey())) {
@@ -499,7 +502,8 @@ private static int readFilesUsingIterStack(VolumeManager fs, ServerConfiguration
     for (FileRef file : files) {
       FileSystem ns = fs.getVolumeByPath(file.path()).getFileSystem();
       readers.add(FileOperations.getInstance().newReaderBuilder()
-          .forFile(file.path().toString(), ns, ns.getConf())
+          .forFile(file.path().toString(), ns, ns.getConf(),
+              CryptoServiceFactory.newDefaultInstance())
           .withTableConfiguration(aconf.getSystemConfiguration()).build());
     }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
index 11b3db3274..eee6f58a6d 100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
@@ -52,6 +52,7 @@
 import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
@@ -2165,7 +2166,7 @@ public void bulkImport() throws Exception {
     // Write an RFile
     String filename = dir + "/bulk/import/rfile.rf";
     FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder()
-        .forFile(filename, fs, fs.getConf())
+        .forFile(filename, fs, fs.getConf(), CryptoServiceFactory.newDefaultInstance())
         .withTableConfiguration(DefaultConfiguration.getInstance()).build();
     writer.startDefaultLocalityGroup();
     writer.append(


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services