You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2019/06/04 22:56:43 UTC

[accumulo] branch 2.0 updated: Remove custom property interpolations (#1179)

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

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


The following commit(s) were added to refs/heads/2.0 by this push:
     new 4b3887a  Remove custom property interpolations (#1179)
4b3887a is described below

commit 4b3887a5f5fc9909f6a56a3fd5afce69b69b0e9e
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Tue Jun 4 18:56:38 2019 -0400

    Remove custom property interpolations (#1179)
    
    * Remove custom property interpolations
    
    Remove unnecessary custom property interpolations in code. Instead, rely
    on property interpolation features of commons-configuration2, as needed,
    to set properties based on environment.
    
    A side effect of this change, as well as the fact that
    GENERAL_DYNAMIC_CLASSPATHS is deprecated, is that the default value for
    that property is now blank (and the empty lib/ext directory is no longer
    created in the assembly tarball).
    
    In future, commons-configuration2 could also be used to interpolate
    values of configuration properties stored in ZooKeeper, but that is out
    of scope of this issue, and can be added later, if there is interest.
    
    This change further decouples the reliance on specialized environment,
    and reduces assumptions made in our code, making it more
    environment-agnostic.
    
    * Fix unit tests and remove long-time ignored tests
---
 assemble/src/main/assemblies/component.xml         |   8 --
 .../java/org/apache/accumulo/core/Constants.java   |   8 --
 .../accumulo/core/conf/AccumuloConfiguration.java  |  15 ++--
 .../accumulo/core/conf/ConfigurationDocGen.java    |  11 ++-
 .../apache/accumulo/core/conf/Interpolated.java    |  36 --------
 .../org/apache/accumulo/core/conf/Property.java    |  74 +++------------
 .../apache/accumulo/core/conf/PropertyType.java    |   7 +-
 .../apache/accumulo/core/conf/PropertyTest.java    |  24 ++---
 .../core/security/VisibilityConstraintTest.java    |   7 --
 .../miniclusterImpl/MiniAccumuloConfigImpl.java    |  29 +++---
 .../classloader/vfs/AccumuloVFSClassLoader.java    |  29 +++---
 .../vfs/AccumuloReloadingVFSClassLoaderTest.java   | 100 ---------------------
 .../vfs/AccumuloVFSClassLoaderTest.java            |  28 +++++-
 13 files changed, 102 insertions(+), 274 deletions(-)

diff --git a/assemble/src/main/assemblies/component.xml b/assemble/src/main/assemblies/component.xml
index ba01def..f23a713 100644
--- a/assemble/src/main/assemblies/component.xml
+++ b/assemble/src/main/assemblies/component.xml
@@ -112,14 +112,6 @@
       <fileMode>0755</fileMode>
     </fileSet>
     <fileSet>
-      <directory>./</directory>
-      <outputDirectory>lib/ext</outputDirectory>
-      <directoryMode>0755</directoryMode>
-      <excludes>
-        <exclude>*/**</exclude>
-      </excludes>
-    </fileSet>
-    <fileSet>
       <directory>conf</directory>
       <directoryMode>0755</directoryMode>
       <fileMode>0644</fileMode>
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index 1d76fe5..7d232e1 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -18,10 +18,6 @@ package org.apache.accumulo.core;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
 public class Constants {
 
   public static final String VERSION = FilteredConstants.VERSION;
@@ -119,10 +115,6 @@ public class Constants {
   public static final String EXPORT_FILE = "exportMetadata.zip";
   public static final String EXPORT_INFO_FILE = "accumulo_export_info.txt";
 
-  // Variables that will be substituted with environment vars in PropertyType.PATH values
-  public static final Collection<String> PATH_PROPERTY_ENV_VARS =
-      Collections.unmodifiableCollection(Arrays.asList("ACCUMULO_HOME", "ACCUMULO_CONF_DIR"));
-
   public static final String HDFS_TABLES_DIR = "/tables";
 
   public static final int DEFAULT_VISIBILITY_CACHE_SIZE = 1000;
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
index 69ebfec..450b2f2 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
@@ -32,7 +32,6 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Predicate;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.PropertyType.PortRange;
 import org.apache.accumulo.core.spi.scan.SimpleScanDispatcher;
 import org.apache.accumulo.core.util.Pair;
@@ -325,27 +324,25 @@ public abstract class AccumuloConfiguration implements Iterable<Entry<String,Str
   }
 
   /**
-   * Gets a property of type {@link PropertyType#PATH}, interpreting the value properly, replacing
-   * supported environment variables.
+   * Gets a property of type {@link PropertyType#PATH}.
    *
    * @param property
    *          property to get
    * @return property value
    * @throws IllegalArgumentException
    *           if the property is of the wrong type
-   * @see Constants#PATH_PROPERTY_ENV_VARS
    */
   public String getPath(Property property) {
     checkType(property, PropertyType.PATH);
 
     String pathString = get(property);
-    if (pathString == null)
+    if (pathString == null) {
       return null;
+    }
 
-    for (String replaceableEnvVar : Constants.PATH_PROPERTY_ENV_VARS) {
-      String envValue = System.getenv(replaceableEnvVar);
-      if (envValue != null)
-        pathString = pathString.replace("$" + replaceableEnvVar, envValue);
+    if (pathString.contains("$ACCUMULO_")) {
+      throw new IllegalArgumentException("Environment variable interpolation not supported here. "
+          + "Consider using '${env:ACCUMULO_HOME}' or similar in your configuration file.");
     }
 
     return pathString;
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
index 296406f..12420df 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
@@ -91,7 +91,7 @@ class ConfigurationDocGen {
         (depr ? "**Deprecated.** " : "") + strike(sanitize(prop.getDescription()), depr) + "<br>");
     doc.print(strike("**type:** " + prop.getType().name(), depr) + ", ");
     doc.print(strike("**zk mutable:** " + isZooKeeperMutable(prop), depr) + ", ");
-    String defaultValue = sanitize(prop.getRawDefaultValue()).trim();
+    String defaultValue = sanitize(prop.getDefaultValue()).trim();
     if (defaultValue.length() == 0) {
       defaultValue = strike("**default value:** empty", depr);
     } else if (defaultValue.contains("\n")) {
@@ -114,8 +114,9 @@ class ConfigurationDocGen {
 
   void propertyTypeDescriptions() {
     for (PropertyType type : PropertyType.values()) {
-      if (type == PropertyType.PREFIX)
+      if (type == PropertyType.PREFIX) {
         continue;
+      }
       doc.println(
           "| " + sanitize(type.toString()) + " | " + sanitize(type.getFormatDescription()) + " |");
     }
@@ -133,10 +134,12 @@ class ConfigurationDocGen {
   }
 
   private String isZooKeeperMutable(Property prop) {
-    if (!Property.isValidZooPropertyKey(prop.getKey()))
+    if (!Property.isValidZooPropertyKey(prop.getKey())) {
       return "no";
-    if (Property.isFixedZooPropertyKey(prop))
+    }
+    if (Property.isFixedZooPropertyKey(prop)) {
       return "yes but requires restart of the " + prop.getKey().split("[.]")[0];
+    }
     return "yes";
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Interpolated.java b/core/src/main/java/org/apache/accumulo/core/conf/Interpolated.java
deleted file mode 100644
index b61de40..0000000
--- a/core/src/main/java/org/apache/accumulo/core/conf/Interpolated.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.conf;
-
-import java.lang.annotation.Inherited;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-
-/**
- * An annotation to denote {@link AccumuloConfiguration} {@link Property} keys, whose values should
- * be interpolated with system properties.
- *
- * Interpolated items need to be careful, as JVM properties could be updates and we may want that
- * propagated when those changes occur. Currently only VFS_CLASSLOADER_CACHE_DIR, which isn't ZK
- * mutable, is interpolated, so this shouldn't be an issue as java.io.tmpdir also shouldn't be
- * changing.
- */
-@Inherited
-@Retention(RetentionPolicy.RUNTIME)
-@interface Interpolated {
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 97d9dfd..4c88785 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -23,7 +23,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Properties;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -39,15 +38,11 @@ import org.apache.accumulo.core.util.format.DefaultFormatter;
 import org.apache.accumulo.core.util.interpret.DefaultScanInterpreter;
 import org.apache.accumulo.start.classloader.AccumuloClassLoader;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
-import org.apache.commons.configuration2.MapConfiguration;
-import org.apache.commons.configuration2.PropertiesConfiguration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
 public enum Property {
   // SSL properties local to each node (see also instance.ssl.enabled which must be consistent
   // across all nodes in an instance)
@@ -185,7 +180,10 @@ public enum Property {
           + "in jars or classes will force a reload of the classloader. Built-in dynamic class "
           + "loading will be removed in a future version. If this is needed, consider overriding "
           + "the Java system class loader with one that has this feature "
-          + "(https://docs.oracle.com/javase/8/docs/api/java/lang/ClassLoader.html#getSystemClassLoader--)."),
+          + "(https://docs.oracle.com/javase/8/docs/api/java/lang/ClassLoader.html#getSystemClassLoader--). "
+          + "Additionally, this property no longer does property interpolation of environment "
+          + "variables, such as '$ACCUMULO_HOME'. Use commons-configuration syntax,"
+          + "'${env:ACCUMULO_HOME}' instead."),
   GENERAL_RPC_TIMEOUT("general.rpc.timeout", "120s", PropertyType.TIMEDURATION,
       "Time to wait on I/O for simple, short RPC calls"),
   @Experimental
@@ -820,14 +818,11 @@ public enum Property {
           + " `general.vfs.context.classpath.<name>.delegation=post`, where `<name>` is"
           + " your context name. If delegation is not specified, it defaults to loading"
           + " from parent classloader first."),
-  @Interpolated
   VFS_CLASSLOADER_CACHE_DIR(AccumuloVFSClassLoader.VFS_CACHE_DIR,
-      "${java.io.tmpdir}" + File.separator + "accumulo-vfs-cache-${user.name}",
-      PropertyType.ABSOLUTEPATH,
+      File.separator + "tmp" + File.separator + "accumulo-vfs-cache", PropertyType.ABSOLUTEPATH,
       "Directory to use for the vfs cache. The cache will keep a soft reference"
           + " to all of the classes loaded in the VM. This should be on local disk on"
-          + " each node with sufficient space. It defaults to"
-          + " ${java.io.tmpdir}/accumulo-vfs-cache-${user.name}"),
+          + " each node with sufficient space."),
 
   // General properties for configuring replication
   REPLICATION_PREFIX("replication.", null, PropertyType.PREFIX,
@@ -928,14 +923,11 @@ public enum Property {
 
   private String key;
   private String defaultValue;
-  private String computedDefaultValue;
   private String description;
   private boolean annotationsComputed = false;
-  private boolean defaultValueComputed = false;
   private boolean isSensitive;
   private boolean isDeprecated;
   private boolean isExperimental;
-  private boolean isInterpolated;
   private Property replacedBy = null;
   private PropertyType type;
 
@@ -961,47 +953,13 @@ public enum Property {
   }
 
   /**
-   * Gets the default value for this property exactly as provided in its definition (i.e., without
-   * interpolation or conversion to absolute paths).
-   *
-   * @return raw default value
-   */
-  public String getRawDefaultValue() {
-    return this.defaultValue;
-  }
-
-  /**
    * Gets the default value for this property. System properties are interpolated into the value if
    * necessary.
    *
    * @return default value
    */
   public String getDefaultValue() {
-    Preconditions.checkState(defaultValueComputed,
-        "precomputeDefaultValue() must be called before calling this method");
-    return computedDefaultValue;
-  }
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "code runs in same security context as user who providing the file")
-  private void precomputeDefaultValue() {
-    String v;
-    if (isInterpolated()) {
-      PropertiesConfiguration pconf = new PropertiesConfiguration();
-      Properties systemProperties = System.getProperties();
-      synchronized (systemProperties) {
-        pconf.append(new MapConfiguration(systemProperties));
-      }
-      pconf.addProperty("hack_default_value", this.defaultValue);
-      v = pconf.getString("hack_default_value");
-    } else {
-      v = getRawDefaultValue();
-    }
-    if (this.type == PropertyType.ABSOLUTEPATH && !(v.trim().equals("")))
-      v = new File(v).getAbsolutePath();
-
-    computedDefaultValue = v;
-    defaultValueComputed = true;
+    return this.defaultValue;
   }
 
   /**
@@ -1022,12 +980,6 @@ public enum Property {
     return this.description;
   }
 
-  private boolean isInterpolated() {
-    Preconditions.checkState(annotationsComputed,
-        "precomputeAnnotations() must be called before calling this method");
-    return isInterpolated;
-  }
-
   /**
    * Checks if this property is experimental.
    *
@@ -1074,8 +1026,6 @@ public enum Property {
         hasAnnotation(Deprecated.class) || hasPrefixWithAnnotation(getKey(), Deprecated.class);
     isExperimental =
         hasAnnotation(Experimental.class) || hasPrefixWithAnnotation(getKey(), Experimental.class);
-    isInterpolated =
-        hasAnnotation(Interpolated.class) || hasPrefixWithAnnotation(getKey(), Interpolated.class);
     if (hasAnnotation(ReplacedBy.class)) {
       ReplacedBy rb = getAnnotation(ReplacedBy.class);
       if (rb != null) {
@@ -1112,9 +1062,11 @@ public enum Property {
   private <T extends Annotation> boolean hasAnnotation(Class<T> annotationType) {
     Logger log = LoggerFactory.getLogger(getClass());
     try {
-      for (Annotation a : getClass().getField(name()).getAnnotations())
-        if (annotationType.isInstance(a))
+      for (Annotation a : getClass().getField(name()).getAnnotations()) {
+        if (annotationType.isInstance(a)) {
           return true;
+        }
+      }
     } catch (SecurityException | NoSuchFieldException e) {
       log.error("{}", e.getMessage(), e);
     }
@@ -1157,8 +1109,9 @@ public enum Property {
 
   private static boolean isKeyValidlyPrefixed(String key) {
     for (String prefix : validPrefixes) {
-      if (key.startsWith(prefix))
+      if (key.startsWith(prefix)) {
         return true;
+      }
     }
 
     return false;
@@ -1351,7 +1304,6 @@ public enum Property {
     // order is very important here the following code relies on the maps and sets populated above
     for (Property p : Property.values()) {
       p.precomputeAnnotations();
-      p.precomputeDefaultValue();
     }
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
index ebca9f4..6ce1120 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
@@ -25,7 +25,6 @@ import java.util.function.Predicate;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.commons.lang.math.IntRange;
 import org.apache.hadoop.fs.Path;
@@ -99,9 +98,9 @@ public enum PropertyType {
 
   PATH("path", x -> true,
       "A string that represents a filesystem path, which can be either relative"
-          + " or absolute to some directory. The filesystem depends on the property."
-          + " The following environment variables will be substituted: "
-          + Constants.PATH_PROPERTY_ENV_VARS),
+          + " or absolute to some directory. The filesystem depends on the property. "
+          + "Substitutions of the ACCUMULO_HOME environment variable can be done in the system "
+          + "config file using '${env:ACCUMULO_HOME}' or similar."),
 
   ABSOLUTEPATH("absolute path",
       x -> x == null || x.trim().isEmpty() || new Path(x.trim()).isAbsolute(),
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java b/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java
index 42717c9..1d0b962 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 
-import java.io.File;
 import java.util.HashSet;
 import java.util.Map.Entry;
 import java.util.TreeMap;
@@ -40,9 +39,11 @@ public class PropertyTest {
   @Test
   public void testProperties() {
     HashSet<String> validPrefixes = new HashSet<>();
-    for (Property prop : Property.values())
-      if (prop.getType().equals(PropertyType.PREFIX))
+    for (Property prop : Property.values()) {
+      if (prop.getType().equals(PropertyType.PREFIX)) {
         validPrefixes.add(prop.getKey());
+      }
+    }
 
     HashSet<String> propertyNames = new HashSet<>();
     for (Property prop : Property.values()) {
@@ -63,11 +64,12 @@ public class PropertyTest {
 
       // make sure property starts with valid prefix
       boolean containsValidPrefix = false;
-      for (String pre : validPrefixes)
+      for (String pre : validPrefixes) {
         if (prop.getKey().startsWith(pre)) {
           containsValidPrefix = true;
           break;
         }
+      }
       assertTrue("Invalid prefix on prop " + prop, containsValidPrefix);
 
       // make sure properties aren't duplicate
@@ -81,24 +83,14 @@ public class PropertyTest {
   @Test
   public void testPorts() {
     HashSet<Integer> usedPorts = new HashSet<>();
-    for (Property prop : Property.values())
+    for (Property prop : Property.values()) {
       if (prop.getType().equals(PropertyType.PORT)) {
         int port = Integer.parseInt(prop.getDefaultValue());
         assertFalse("Port already in use: " + port, usedPorts.contains(port));
         usedPorts.add(port);
         assertTrue("Port out of range of valid ports: " + port, port > 1023 && port < 65536);
       }
-  }
-
-  @Test
-  public void testRawDefaultValues() {
-    AccumuloConfiguration conf = DefaultConfiguration.getInstance();
-    assertEquals("${java.io.tmpdir}" + File.separator + "accumulo-vfs-cache-${user.name}",
-        Property.VFS_CLASSLOADER_CACHE_DIR.getRawDefaultValue());
-    assertEquals(
-        new File(System.getProperty("java.io.tmpdir"),
-            "accumulo-vfs-cache-" + System.getProperty("user.name")).getAbsolutePath(),
-        conf.get(Property.VFS_CLASSLOADER_CACHE_DIR));
+    }
   }
 
   // This test verifies all "sensitive" properties are properly marked as sensitive
diff --git a/core/src/test/java/org/apache/accumulo/core/security/VisibilityConstraintTest.java b/core/src/test/java/org/apache/accumulo/core/security/VisibilityConstraintTest.java
index 7ed639e..e080664 100644
--- a/core/src/test/java/org/apache/accumulo/core/security/VisibilityConstraintTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/security/VisibilityConstraintTest.java
@@ -32,7 +32,6 @@ import org.apache.accumulo.core.constraints.VisibilityConstraint;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.Mutation;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 
 public class VisibilityConstraintTest {
@@ -96,10 +95,4 @@ public class VisibilityConstraintTest {
     assertEquals("unauthorized", ENOAUTH, vc.check(env, mutation));
   }
 
-  @Test
-  @Ignore
-  public void testMalformedVisibility() {
-    // TODO: ACCUMULO-1006 Should test for returning error code 1, but not sure how since
-    // ColumnVisibility won't let us construct a bad one in the first place
-  }
 }
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
index 6d935ff..aa2872e 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
@@ -112,8 +112,9 @@ public class MiniAccumuloConfigImpl {
   MiniAccumuloConfigImpl initialize() {
 
     // Sanity checks
-    if (this.getDir().exists() && !this.getDir().isDirectory())
+    if (this.getDir().exists() && !this.getDir().isDirectory()) {
       throw new IllegalArgumentException("Must pass in directory, " + this.getDir() + " is a file");
+    }
 
     if (this.getDir().exists()) {
       String[] children = this.getDir().list();
@@ -152,8 +153,9 @@ public class MiniAccumuloConfigImpl {
       @SuppressWarnings("deprecation")
       Property generalClasspaths = Property.GENERAL_CLASSPATHS;
       mergeProp(generalClasspaths.getKey(), libDir.getAbsolutePath() + "/[^.].*[.]jar");
-      mergeProp(Property.GENERAL_DYNAMIC_CLASSPATHS.getKey(),
-          libExtDir.getAbsolutePath() + "/[^.].*[.]jar");
+      @SuppressWarnings("deprecation")
+      Property generalDynamicClasspaths = Property.GENERAL_DYNAMIC_CLASSPATHS;
+      mergeProp(generalDynamicClasspaths.getKey(), libExtDir.getAbsolutePath() + "/[^.].*[.]jar");
       mergeProp(Property.GC_CYCLE_DELAY.getKey(), "4s");
       mergeProp(Property.GC_CYCLE_START.getKey(), "0s");
       mergePropWithRandomPort(Property.MASTER_CLIENTPORT.getKey());
@@ -176,8 +178,9 @@ public class MiniAccumuloConfigImpl {
           zkHost = existingZooKeepers;
         } else {
           // zookeeper port should be set explicitly in this class, not just on the site config
-          if (zooKeeperPort == 0)
+          if (zooKeeperPort == 0) {
             zooKeeperPort = PortUtils.getRandomFreePort();
+          }
 
           zkHost = "localhost:" + zooKeeperPort;
         }
@@ -264,8 +267,9 @@ public class MiniAccumuloConfigImpl {
    *          the number of tablet servers that mini accumulo cluster should start
    */
   public MiniAccumuloConfigImpl setNumTservers(int numTservers) {
-    if (numTservers < 1)
+    if (numTservers < 1) {
       throw new IllegalArgumentException("Must have at least one tablet server");
+    }
     this.numTservers = numTservers;
     return this;
   }
@@ -287,9 +291,10 @@ public class MiniAccumuloConfigImpl {
    *          key/values that you normally put in accumulo.properties can be put here.
    */
   public MiniAccumuloConfigImpl setSiteConfig(Map<String,String> siteConfig) {
-    if (existingInstance != null && existingInstance)
+    if (existingInstance != null && existingInstance) {
       throw new UnsupportedOperationException(
           "Cannot set set config info when using an existing instance.");
+    }
 
     this.existingInstance = Boolean.FALSE;
 
@@ -297,9 +302,10 @@ public class MiniAccumuloConfigImpl {
   }
 
   public MiniAccumuloConfigImpl setClientProps(Map<String,String> clientProps) {
-    if (existingInstance != null && existingInstance)
+    if (existingInstance != null && existingInstance) {
       throw new UnsupportedOperationException(
           "Cannot set zookeeper info when using an existing instance.");
+    }
     this.existingInstance = Boolean.FALSE;
     this.clientProps = clientProps;
     return this;
@@ -320,9 +326,10 @@ public class MiniAccumuloConfigImpl {
    * @since 1.6.0
    */
   public MiniAccumuloConfigImpl setZooKeeperPort(int zooKeeperPort) {
-    if (existingInstance != null && existingInstance)
+    if (existingInstance != null && existingInstance) {
       throw new UnsupportedOperationException(
           "Cannot set zookeeper info when using an existing instance.");
+    }
 
     this.existingInstance = Boolean.FALSE;
 
@@ -341,9 +348,10 @@ public class MiniAccumuloConfigImpl {
    * @since 1.6.1
    */
   public MiniAccumuloConfigImpl setZooKeeperStartupTime(long zooKeeperStartupTime) {
-    if (existingInstance != null && existingInstance)
+    if (existingInstance != null && existingInstance) {
       throw new UnsupportedOperationException(
           "Cannot set zookeeper info when using an existing instance.");
+    }
 
     this.existingInstance = Boolean.FALSE;
 
@@ -707,9 +715,10 @@ public class MiniAccumuloConfigImpl {
    */
   public MiniAccumuloConfigImpl useExistingInstance(File accumuloProps, File hadoopConfDir)
       throws IOException {
-    if (existingInstance != null && !existingInstance)
+    if (existingInstance != null && !existingInstance) {
       throw new UnsupportedOperationException(
           "Cannot set to useExistingInstance after specifying config/zookeeper");
+    }
 
     this.existingInstance = Boolean.TRUE;
 
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
index f3a2150..9390475 100644
--- a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
+++ b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
@@ -86,7 +86,7 @@ public class AccumuloVFSClassLoader {
 
   public static final String DYNAMIC_CLASSPATH_PROPERTY_NAME = "general.dynamic.classpaths";
 
-  public static final String DEFAULT_DYNAMIC_CLASSPATH_VALUE = "$ACCUMULO_HOME/lib/ext/[^.].*.jar";
+  public static final String DEFAULT_DYNAMIC_CLASSPATH_VALUE = "";
 
   public static final String VFS_CLASSLOADER_SYSTEM_CLASSPATH_PROPERTY = "general.vfs.classpaths";
 
@@ -126,8 +126,9 @@ public class AccumuloVFSClassLoader {
 
   static FileObject[] resolve(FileSystemManager vfs, String uris,
       ArrayList<FileObject> pathsToMonitor) throws FileSystemException {
-    if (uris == null)
+    if (uris == null) {
       return new FileObject[0];
+    }
 
     ArrayList<FileObject> classpath = new ArrayList<>();
 
@@ -137,8 +138,9 @@ public class AccumuloVFSClassLoader {
 
       path = path.trim();
 
-      if (path.equals(""))
+      if (path.equals("")) {
         continue;
+      }
 
       path = AccumuloClassLoader.replaceEnvVars(path, System.getenv());
 
@@ -188,8 +190,9 @@ public class AccumuloVFSClassLoader {
 
     ReloadingClassLoader wrapper = () -> parent;
 
-    if (dynamicCPath == null || dynamicCPath.equals(""))
+    if (dynamicCPath == null || dynamicCPath.equals("")) {
       return wrapper;
+    }
 
     // TODO monitor time for lib/ext was 1 sec... should this be configurable? - ACCUMULO-1301
     return new AccumuloReloadingVFSClassLoader(dynamicCPath, generateVfs(), wrapper, 1000, true);
@@ -316,13 +319,16 @@ public class AccumuloVFSClassLoader {
   }
 
   private static void printJar(Printer out, String jarPath, boolean debug, boolean sawFirst) {
-    if (debug)
+    if (debug) {
       out.print("\t");
-    if (!debug && sawFirst)
+    }
+    if (!debug && sawFirst) {
       out.print(":");
+    }
     out.print(jarPath);
-    if (debug)
+    if (debug) {
       out.print("\n");
+    }
   }
 
   public static void printClassPath(Printer out, boolean debug) {
@@ -377,23 +383,26 @@ public class AccumuloVFSClassLoader {
 
         boolean sawFirst = false;
         if (classLoader instanceof URLClassLoader) {
-          if (debug)
+          if (debug) {
             out.print("Level " + classLoaderDescription + " URL classpath items are:\n");
+          }
           for (URL u : ((URLClassLoader) classLoader).getURLs()) {
             printJar(out, u.getFile(), debug, sawFirst);
             sawFirst = true;
           }
         } else if (classLoader instanceof VFSClassLoader) {
-          if (debug)
+          if (debug) {
             out.print("Level " + classLoaderDescription + " VFS classpaths items are:\n");
+          }
           VFSClassLoader vcl = (VFSClassLoader) classLoader;
           for (FileObject f : vcl.getFileObjects()) {
             printJar(out, f.getURL().getFile(), debug, sawFirst);
             sawFirst = true;
           }
         } else {
-          if (debug)
+          if (debug) {
             out.print("Unknown classloader configuration " + classLoader.getClass() + "\n");
+          }
         }
       }
       out.print("\n");
diff --git a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoaderTest.java b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoaderTest.java
index e81dc14..c3806b5 100644
--- a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoaderTest.java
+++ b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoaderTest.java
@@ -31,7 +31,6 @@ import org.apache.commons.vfs2.FileSystemManager;
 import org.apache.commons.vfs2.impl.VFSClassLoader;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
@@ -174,105 +173,6 @@ public class AccumuloReloadingVFSClassLoaderTest {
     arvcl.close();
   }
 
-  // This test fails because of an error with the underlying monitor (ACCUMULO-1507/VFS-487).
-  // Uncomment when this has been addressed.
-  //
-  // This is caused by the filed being deleted and then readded in the same monitor tick. This
-  // causes the file to ultimately register the deletion over any
-  // other events.
-  @Test
-  @Ignore
-  public void testFastDeleteAndReAdd() throws Exception {
-    FileObject testDir = vfs.resolveFile(folder1.getRoot().toURI().toString());
-    FileObject[] dirContents = testDir.getChildren();
-
-    AccumuloReloadingVFSClassLoader arvcl = new AccumuloReloadingVFSClassLoader(folderPath, vfs,
-        ClassLoader::getSystemClassLoader, 1000, true);
-
-    FileObject[] files = ((VFSClassLoader) arvcl.getClassLoader()).getFileObjects();
-    assertArrayEquals(createFileSystems(dirContents), files);
-
-    Class<?> clazz1 = arvcl.getClassLoader().loadClass("test.HelloWorld");
-    Object o1 = clazz1.newInstance();
-    assertEquals("Hello World!", o1.toString());
-
-    // Check that the class is the same before the update
-    Class<?> clazz1_5 = arvcl.getClassLoader().loadClass("test.HelloWorld");
-    assertEquals(clazz1, clazz1_5);
-
-    assertTrue(new File(folder1.getRoot(), "HelloWorld.jar").delete());
-
-    // Update the class
-    FileUtils.copyURLToFile(this.getClass().getResource("/HelloWorld.jar"),
-        folder1.newFile("HelloWorld.jar"));
-
-    // Wait for the monitor to notice
-    // VFS-487 significantly wait to avoid failure
-    Thread.sleep(7000);
-
-    Class<?> clazz2 = arvcl.getClassLoader().loadClass("test.HelloWorld");
-    Object o2 = clazz2.newInstance();
-    assertEquals("Hello World!", o2.toString());
-
-    // This is false because they are loaded by a different classloader
-    assertNotEquals(clazz1, clazz2);
-    assertNotEquals(o1, o2);
-
-    arvcl.close();
-  }
-
-  @Test
-  @Ignore
-  public void testModifiedClass() throws Exception {
-
-    FileObject testDir = vfs.resolveFile(folder1.getRoot().toURI().toString());
-    FileObject[] dirContents = testDir.getChildren();
-
-    AccumuloReloadingVFSClassLoader arvcl = new AccumuloReloadingVFSClassLoader(folderPath, vfs,
-        ClassLoader::getSystemClassLoader, 1000, true);
-
-    FileObject[] files = ((VFSClassLoader) arvcl.getClassLoader()).getFileObjects();
-    assertArrayEquals(createFileSystems(dirContents), files);
-
-    ClassLoader loader1 = arvcl.getClassLoader();
-    Class<?> clazz1 = loader1.loadClass("test.HelloWorld");
-    Object o1 = clazz1.newInstance();
-    assertEquals("Hello World!", o1.toString());
-
-    // Check that the class is the same before the update
-    Class<?> clazz1_5 = arvcl.getClassLoader().loadClass("test.HelloWorld");
-    assertEquals(clazz1, clazz1_5);
-
-    // java does aggressive caching of jar files. When using java code to read jar files that are
-    // created in the same second, it will only see the first jar
-    // file
-    Thread.sleep(1000);
-
-    assertTrue(new File(folder1.getRoot(), "HelloWorld.jar").delete());
-
-    // Update the class
-    FileUtils.copyURLToFile(this.getClass().getResource("/HelloWorld2.jar"),
-        folder1.newFile("HelloWorld.jar"));
-
-    // Wait for the monitor to notice
-    // VFS-487 significantly wait to avoid failure
-    Thread.sleep(7000);
-
-    Class<?> clazz2 = arvcl.getClassLoader().loadClass("test.HelloWorld");
-    Object o2 = clazz2.newInstance();
-    assertEquals("Hallo Welt", o2.toString());
-
-    // This is false because they are loaded by a different classloader
-    assertNotEquals(clazz1, clazz2);
-    assertNotEquals(o1, o2);
-
-    Class<?> clazz3 = loader1.loadClass("test.HelloWorld");
-    Object o3 = clazz3.newInstance();
-    assertEquals("Hello World!", o3.toString());
-
-    arvcl.close();
-  }
-
   @After
   public void tearDown() throws Exception {
     folder1.delete();
diff --git a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java
index 3057b87..2600948 100644
--- a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java
+++ b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java
@@ -33,6 +33,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
+import org.powermock.core.classloader.MockClassLoader;
 import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.core.classloader.annotations.SuppressStaticInitializationFor;
@@ -62,7 +63,7 @@ public class AccumuloVFSClassLoaderTest {
   }
 
   /*
-   * Test that if enabled, but not configured, that the code creates the 2nd level classloader
+   * Test that the default (empty dynamic class paths) does not create the 2nd level loader
    */
   @Test
   public void testDefaultConfig() throws Exception {
@@ -79,6 +80,30 @@ public class AccumuloVFSClassLoaderTest {
     Whitebox.setInternalState(AccumuloClassLoader.class, "accumuloConfigUrl", conf.toURI().toURL());
     Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object());
     ClassLoader acl = AccumuloVFSClassLoader.getClassLoader();
+    assertTrue((acl instanceof URLClassLoader));
+    // no second level means the parent is the system loader (in this case, PowerMock's loader)
+    assertTrue((acl.getParent() instanceof MockClassLoader));
+  }
+
+  /*
+   * Test that if configured with dynamic class paths, that the code creates the 2nd level loader
+   */
+  @Test
+  public void testDynamicConfig() throws Exception {
+
+    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader",
+        (AccumuloReloadingVFSClassLoader) null);
+
+    File conf = folder1.newFile("accumulo.properties");
+    FileWriter out = new FileWriter(conf);
+    out.append("general.classpaths=\n");
+    out.append("general.vfs.classpaths=\n");
+    out.append("general.dynamic.classpaths=" + System.getProperty("user.dir") + "\n");
+    out.close();
+
+    Whitebox.setInternalState(AccumuloClassLoader.class, "accumuloConfigUrl", conf.toURI().toURL());
+    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object());
+    ClassLoader acl = AccumuloVFSClassLoader.getClassLoader();
     assertTrue((acl instanceof VFSClassLoader));
     assertTrue((acl.getParent() instanceof URLClassLoader));
   }
@@ -101,6 +126,7 @@ public class AccumuloVFSClassLoaderTest {
     out.append("general.classpaths=\n");
     out.append(
         "general.vfs.classpaths=" + new File(folder1.getRoot(), "HelloWorld.jar").toURI() + "\n");
+    out.append("general.dynamic.classpaths=" + System.getProperty("user.dir") + "\n");
     out.close();
 
     Whitebox.setInternalState(AccumuloClassLoader.class, "accumuloConfigUrl", conf.toURI().toURL());