You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mw...@apache.org on 2018/10/30 19:51:47 UTC

[accumulo] branch master updated (2468f0b -> d83837c)

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

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


    from 2468f0b  Made AccumuloClient Closeable (#718)
     new 16722b1  Fixes #725 - Log deprecated properties (#726)
     new 565be32  Code review updates (#726)
     new d83837c  Merge branch '726a'

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


Summary of changes:
 .../accumulo/core/conf/ConfigSanityCheck.java      |  14 ---
 .../org/apache/accumulo/core/conf/Property.java    | 115 +++++++++++++--------
 .../conf/{Experimental.java => ReplacedBy.java}    |   7 +-
 .../accumulo/core/conf/SiteConfiguration.java      |  44 +++++---
 .../org/apache/accumulo/server/ServerUtil.java     |  11 ++
 5 files changed, 116 insertions(+), 75 deletions(-)
 copy core/src/main/java/org/apache/accumulo/core/conf/{Experimental.java => ReplacedBy.java} (76%)


[accumulo] 02/03: Code review updates (#726)

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

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

commit 565be322a29297913eb5e10b85e25b8e93579d07
Author: Mike Walch <mw...@apache.org>
AuthorDate: Mon Oct 29 16:25:36 2018 -0400

    Code review updates (#726)
    
    * Only print warnings if config property is set
    * Only print warnings in ServerUtil when process starts
---
 .../accumulo/core/conf/ConfigSanityCheck.java      |  20 ----
 .../org/apache/accumulo/core/conf/Property.java    | 129 ++++++++++++---------
 .../org/apache/accumulo/core/conf/ReplacedBy.java  |   2 +-
 .../accumulo/core/conf/SiteConfiguration.java      |  44 ++++---
 .../org/apache/accumulo/server/ServerUtil.java     |  11 ++
 5 files changed, 112 insertions(+), 94 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java
index 8073d8c..e606057 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java
@@ -33,10 +33,6 @@ public class ConfigSanityCheck {
 
   private static final Logger log = LoggerFactory.getLogger(ConfigSanityCheck.class);
   private static final String PREFIX = "BAD CONFIG ";
-  @SuppressWarnings("deprecation")
-  private static final Property INSTANCE_DFS_URI = Property.INSTANCE_DFS_URI;
-  @SuppressWarnings("deprecation")
-  private static final Property INSTANCE_DFS_DIR = Property.INSTANCE_DFS_DIR;
 
   /**
    * Validates the given configuration entries. A valid configuration contains only valid properties
@@ -51,7 +47,6 @@ public class ConfigSanityCheck {
    */
   public static void validate(Iterable<Entry<String,String>> entries) {
     String instanceZkTimeoutValue = null;
-    boolean usingVolumes = false;
     for (Entry<String,String> entry : entries) {
       String key = entry.getKey();
       String value = entry.getValue();
@@ -66,24 +61,10 @@ public class ConfigSanityCheck {
         fatal(PREFIX + "improperly formatted value for key (" + key + ", type=" + prop.getType()
             + ") : " + value);
 
-      if (prop!=null) { 
-          if (prop.hasAnnotation(Deprecated.class)) {
-            log.warn("Use of {} is deprecated.", key);
-          }
-          ReplacedBy replacedBy = prop.getAnnotation(ReplacedBy.class);
-          if (replacedBy != null) {
-            log.warn("Consider using {} instead of {}.", replacedBy.replacedByProperty(), key);
-          }
-      }
-
       if (key.equals(Property.INSTANCE_ZK_TIMEOUT.getKey())) {
         instanceZkTimeoutValue = value;
       }
 
-      if (key.equals(Property.INSTANCE_VOLUMES.getKey())) {
-        usingVolumes = value != null && !value.isEmpty();
-      }
-
       // If the block size or block size index is configured to be too large, we throw an exception
       // to avoid potentially corrupting RFiles later
       if (key.equals(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX.getKey())
@@ -103,7 +84,6 @@ public class ConfigSanityCheck {
       checkTimeDuration(Property.INSTANCE_ZK_TIMEOUT, instanceZkTimeoutValue,
           new CheckTimeDurationBetween(1000, 300000));
     }
-
   }
 
   private interface CheckTimeDuration {
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 2e919be..65a3a96 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
@@ -116,23 +116,6 @@ public enum Property {
           + " reference files created at that location before the config change. To use"
           + " a comma or other reserved characters in a URI use standard URI hex"
           + " encoding. For example replace commas with %2C."),
-  @Deprecated
-  @ReplacedBy(replacedByProperty=INSTANCE_VOLUMES)
-  INSTANCE_DFS_URI("instance.dfs.uri", "", PropertyType.URI,
-		  "This property is deprecated since 1.6.0. "
-				  + "A url accumulo should use to connect to DFS. If this is empty, accumulo"
-				  + " will obtain this information from the hadoop configuration. This property"
-				  + " will only be used when creating new files if instance.volumes is empty."
-				  + " After an upgrade to 1.6.0 Accumulo will start using absolute paths to"
-				  + " reference files. Files created before a 1.6.0 upgrade are referenced via"
-				  + " relative paths. Relative paths will always be resolved using this config"
-				  + " (if empty using the hadoop config)."),
-  @Deprecated
-  @ReplacedBy(replacedByProperty=INSTANCE_VOLUMES)
-  INSTANCE_DFS_DIR("instance.dfs.dir", "/accumulo", PropertyType.ABSOLUTEPATH,
-		  "This property is deprecated since 1.6.0. "
-				  + "HDFS directory in which accumulo instance will run. "
-				  + "Do not change after accumulo is initialized."),
   INSTANCE_VOLUMES_REPLACEMENTS("instance.volumes.replacements", "", PropertyType.STRING,
       "Since accumulo stores absolute URIs changing the location of a namenode "
           + "could prevent Accumulo from starting. The property helps deal with "
@@ -195,13 +178,6 @@ public enum Property {
   GENERAL_PREFIX("general.", null, PropertyType.PREFIX,
       "Properties in this category affect the behavior of accumulo overall, but"
           + " do not have to be consistent throughout a cloud."),
-  @Deprecated
-  GENERAL_CLASSPATHS(AccumuloClassLoader.GENERAL_CLASSPATHS, "", PropertyType.STRING,
-      "This property is deprecated since 2.0.0. The class path should instead be configured"
-          + " by the launch environment (for example, accumulo-env.sh). A list of all"
-          + " of the places to look for a class. Order does matter, as it will look for"
-          + " the jar starting in the first location to the last. Supports full regex"
-          + " on filename alone."),
   GENERAL_DYNAMIC_CLASSPATHS(AccumuloVFSClassLoader.DYNAMIC_CLASSPATH_PROPERTY_NAME,
       AccumuloVFSClassLoader.DEFAULT_DYNAMIC_CLASSPATH_VALUE, PropertyType.STRING,
       "A list of all of the places where changes in jars or classes will force "
@@ -386,17 +362,6 @@ public enum Property {
       "When a tablet server's SimpleTimer thread triggers to check idle"
           + " sessions, this configurable option will be used to evaluate update"
           + " sessions to determine if they can be closed due to inactivity"),
-  @Deprecated
-  TSERV_READ_AHEAD_MAXCONCURRENT("tserver.readahead.concurrent.max", "16", PropertyType.COUNT,
-      "This property is deprecated since 2.0.0, use tserver.scan.executors.default.threads "
-          + "instead. The maximum number of concurrent read ahead that will execute. This "
-          + "effectively limits the number of long running scans that can run concurrently "
-          + "per tserver.\""),
-  @Deprecated
-  TSERV_METADATA_READ_AHEAD_MAXCONCURRENT("tserver.metadata.readahead.concurrent.max", "8",
-      PropertyType.COUNT,
-      "This property is deprecated since 2.0.0, use tserver.scan.executors.meta.threads instead. "
-          + "The maximum number of concurrent metadata read ahead that will execute."),
   TSERV_SCAN_EXECUTORS_PREFIX("tserver.scan.executors.", null, PropertyType.PREFIX,
       "Prefix for defining executors to service scans. See "
           + "[scan executors]({% durl administration/scan-executors %}) for an overview of why and"
@@ -486,9 +451,6 @@ public enum Property {
   TSERV_WAL_SYNC("tserver.wal.sync", "true", PropertyType.BOOLEAN,
       "Use the SYNC_BLOCK create flag to sync WAL writes to disk. Prevents"
           + " problems recovering from sudden system resets."),
-  @Deprecated
-  TSERV_WAL_SYNC_METHOD("tserver.wal.sync.method", "hsync", PropertyType.STRING,
-      "This property is deprecated since 1.7.0. Use table.durability instead."),
   TSERV_ASSIGNMENT_DURATION_WARNING("tserver.assignment.duration.warning", "10m",
       PropertyType.TIMEDURATION,
       "The amount of time an assignment can run before the server will print a"
@@ -570,7 +532,6 @@ public enum Property {
           + " monitor.ssl.include.ciphers to allow ciphers"),
   MONITOR_SSL_INCLUDE_PROTOCOLS("monitor.ssl.include.protocols", "TLSv1.2", PropertyType.STRING,
       "A comma-separate list of allowed SSL protocols"),
-
   MONITOR_LOCK_CHECK_INTERVAL("monitor.lock.check.interval", "5s", PropertyType.TIMEDURATION,
       "The amount of time to sleep between checking for the Montior ZooKeeper lock"),
   MONITOR_RESOURCES_EXTERNAL("monitor.resources.external", "", PropertyType.STRING,
@@ -691,9 +652,6 @@ public enum Property {
           + " Summary.getFileStatistics().getLarge(). When adjusting this consider the"
           + " expected number RFiles with summaries on each tablet server and the"
           + " summary cache size."),
-  @Deprecated
-  TABLE_WALOG_ENABLED("table.walog.enabled", "true", PropertyType.BOOLEAN,
-      "This setting is deprecated since 1.7.0. Use table.durability=none instead."),
   TABLE_BLOOM_ENABLED("table.bloom.enabled", "false", PropertyType.BOOLEAN,
       "Use bloom filters on this table."),
   TABLE_BLOOM_LOAD_THRESHOLD("table.bloom.load.threshold", "1", PropertyType.COUNT,
@@ -719,6 +677,7 @@ public enum Property {
           + " none, which skips the write-ahead log; log, which sends the data to the"
           + " write-ahead log, but does nothing to make it durable; flush, which pushes"
           + " data to the file system; and sync, which ensures the data is written to disk."),
+
   TABLE_FAILURES_IGNORE("table.failures.ignore", "false", PropertyType.BOOLEAN,
       "If you want queries for your table to hang or fail when data is missing"
           + " from the system, then set this to false. When this set to true missing"
@@ -906,7 +865,53 @@ public enum Property {
       "The sampling percentage to use for replication traces"),
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
-          + " the attempt. See replication.work.attempts.");
+          + " the attempt. See replication.work.attempts."),
+  // deprecated properties grouped at the end to reference property that replaces them
+  @Deprecated
+  @ReplacedBy(property = INSTANCE_VOLUMES)
+  INSTANCE_DFS_URI("instance.dfs.uri", "", PropertyType.URI,
+      "This property is deprecated since 1.6.0. "
+          + "A url accumulo should use to connect to DFS. If this is empty, accumulo"
+          + " will obtain this information from the hadoop configuration. This property"
+          + " will only be used when creating new files if instance.volumes is empty."
+          + " After an upgrade to 1.6.0 Accumulo will start using absolute paths to"
+          + " reference files. Files created before a 1.6.0 upgrade are referenced via"
+          + " relative paths. Relative paths will always be resolved using this config"
+          + " (if empty using the hadoop config)."),
+  @Deprecated
+  @ReplacedBy(property = INSTANCE_VOLUMES)
+  INSTANCE_DFS_DIR("instance.dfs.dir", "/accumulo", PropertyType.ABSOLUTEPATH,
+      "This property is deprecated since 1.6.0. "
+          + "HDFS directory in which accumulo instance will run. "
+          + "Do not change after accumulo is initialized."),
+  @Deprecated
+  GENERAL_CLASSPATHS(AccumuloClassLoader.GENERAL_CLASSPATHS, "", PropertyType.STRING,
+      "This property is deprecated since 2.0.0. The class path should instead be configured"
+          + " by the launch environment (for example, accumulo-env.sh). A list of all"
+          + " of the places to look for a class. Order does matter, as it will look for"
+          + " the jar starting in the first location to the last. Supports full regex"
+          + " on filename alone."),
+  @Deprecated
+  @ReplacedBy(property = TABLE_DURABILITY)
+  TSERV_WAL_SYNC_METHOD("tserver.wal.sync.method", "hsync", PropertyType.STRING,
+      "This property is deprecated since 1.7.0. Use table.durability instead."),
+  @Deprecated
+  @ReplacedBy(property = TABLE_DURABILITY)
+  TABLE_WALOG_ENABLED("table.walog.enabled", "true", PropertyType.BOOLEAN,
+      "This setting is deprecated since 1.7.0. Use table.durability=none instead."),
+  @Deprecated
+  @ReplacedBy(property = TSERV_SCAN_EXECUTORS_DEFAULT_THREADS)
+  TSERV_READ_AHEAD_MAXCONCURRENT("tserver.readahead.concurrent.max", "16", PropertyType.COUNT,
+      "This property is deprecated since 2.0.0, use tserver.scan.executors.default.threads "
+          + "instead. The maximum number of concurrent read ahead that will execute. This "
+          + "effectively limits the number of long running scans that can run concurrently "
+          + "per tserver.\""),
+  @Deprecated
+  @ReplacedBy(property = TSERV_SCAN_EXECUTORS_META_THREADS)
+  TSERV_METADATA_READ_AHEAD_MAXCONCURRENT("tserver.metadata.readahead.concurrent.max", "8",
+      PropertyType.COUNT,
+      "This property is deprecated since 2.0.0, use tserver.scan.executors.meta.threads instead. "
+          + "The maximum number of concurrent metadata read ahead that will execute.");
 
   private String key;
   private String defaultValue;
@@ -918,6 +923,7 @@ public enum Property {
   private boolean isDeprecated;
   private boolean isExperimental;
   private boolean isInterpolated;
+  private Property replacedBy = null;
   private PropertyType type;
 
   private Property(String name, String defaultValue, PropertyType type, String description) {
@@ -1042,6 +1048,12 @@ public enum Property {
     return isSensitive;
   }
 
+  public Property replacedBy() {
+    Preconditions.checkState(annotationsComputed,
+        "precomputeAnnotations() must be called before calling this method");
+    return replacedBy;
+  }
+
   private void precomputeAnnotations() {
     isSensitive = hasAnnotation(Sensitive.class)
         || hasPrefixWithAnnotation(getKey(), Sensitive.class);
@@ -1051,6 +1063,12 @@ public enum Property {
         || hasPrefixWithAnnotation(getKey(), Experimental.class);
     isInterpolated = hasAnnotation(Interpolated.class)
         || hasPrefixWithAnnotation(getKey(), Interpolated.class);
+    if (hasAnnotation(ReplacedBy.class)) {
+      ReplacedBy rb = getAnnotation(ReplacedBy.class);
+      if (rb != null) {
+        replacedBy = rb.property();
+      }
+    }
     annotationsComputed = true;
   }
 
@@ -1075,11 +1093,10 @@ public enum Property {
         }
       }
     }
-
     return false;
   }
 
-  public <T extends Annotation> boolean hasAnnotation(Class<T> annotationType) {
+  private <T extends Annotation> boolean hasAnnotation(Class<T> annotationType) {
     Logger log = LoggerFactory.getLogger(getClass());
     try {
       for (Annotation a : getClass().getField(name()).getAnnotations())
@@ -1091,17 +1108,17 @@ public enum Property {
     return false;
   }
 
-  public <T extends Annotation> T getAnnotation(Class<T> annotationType) {
-	    Logger log = LoggerFactory.getLogger(getClass());
-	    try {
-	      for (Annotation a : getClass().getField(name()).getAnnotations())
-	        if (annotationType.isInstance(a))
-	          return (T) a;
-	    } catch (SecurityException | NoSuchFieldException e) {
-	      log.error("{}", e.getMessage(), e);
-	    }
-	    return null;
-	  }
+  private <T extends Annotation> T getAnnotation(Class<T> annotationType) {
+    Logger log = LoggerFactory.getLogger(getClass());
+    try {
+      for (Annotation a : getClass().getField(name()).getAnnotations())
+        if (annotationType.isInstance(a))
+          return (T) a;
+    } catch (SecurityException | NoSuchFieldException e) {
+      log.error("{}", e.getMessage(), e);
+    }
+    return null;
+  }
 
   private static <T extends Annotation> boolean hasPrefixWithAnnotation(String key,
       Class<T> annotationType) {
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ReplacedBy.java b/core/src/main/java/org/apache/accumulo/core/conf/ReplacedBy.java
index 6d5e525..5d5b2f6 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ReplacedBy.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ReplacedBy.java
@@ -29,5 +29,5 @@ import java.lang.annotation.RetentionPolicy;
 @Retention(RetentionPolicy.RUNTIME)
 @interface ReplacedBy {
 
-  Property replacedByProperty();
+  Property property();
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
index ccadbee..7a7e095 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
@@ -37,8 +37,6 @@ import org.apache.commons.configuration.PropertiesConfiguration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
-
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 /**
@@ -181,19 +179,9 @@ public class SiteConfiguration extends AccumuloConfiguration {
     String key = property.getKey();
     // If the property is sensitive, see if CredentialProvider was configured.
     if (property.isSensitive()) {
-      org.apache.hadoop.conf.Configuration hadoopConf = getHadoopConfiguration();
-      if (null != hadoopConf) {
-        // Try to find the sensitive value from the CredentialProvider
-        try {
-          char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf,
-              key);
-          if (null != value) {
-            return new String(value);
-          }
-        } catch (IOException e) {
-          log.warn("Failed to extract sensitive property (" + key
-              + ") from Hadoop CredentialProvider, falling back to accumulo.properties", e);
-        }
+      String hadoopVal = getSensitiveFromHadoop(property);
+      if (hadoopVal != null) {
+        return hadoopVal;
       }
     }
 
@@ -214,10 +202,32 @@ public class SiteConfiguration extends AccumuloConfiguration {
     return value;
   }
 
+  private String getSensitiveFromHadoop(Property property) {
+    org.apache.hadoop.conf.Configuration hadoopConf = getHadoopConfiguration();
+    if (null != hadoopConf) {
+      // Try to find the sensitive value from the CredentialProvider
+      try {
+        char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf,
+            property.getKey());
+        if (null != value) {
+          return new String(value);
+        }
+      } catch (IOException e) {
+        log.warn("Failed to extract sensitive property (" + property.getKey()
+            + ") from Hadoop CredentialProvider, falling back to accumulo.properties", e);
+      }
+    }
+    return null;
+  }
+
   @Override
   public boolean isPropertySet(Property prop) {
-    Preconditions.checkArgument(!prop.isSensitive(),
-        "This method not implemented for sensitive props");
+    if (prop.isSensitive()) {
+      String hadoopVal = getSensitiveFromHadoop(prop);
+      if (hadoopVal != null) {
+        return true;
+      }
+    }
     return overrides.containsKey(prop.getKey()) || staticConfigs.containsKey(prop.getKey())
         || getConfiguration().containsKey(prop.getKey()) || parent.isPropertySet(prop);
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerUtil.java b/server/base/src/main/java/org/apache/accumulo/server/ServerUtil.java
index 1491d91..7466185 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerUtil.java
@@ -125,6 +125,17 @@ public class ServerUtil {
     for (Entry<String,String> entry : sortedProps.entrySet()) {
       String key = entry.getKey();
       log.info("{} = {}", key, (Property.isSensitive(key) ? "<hidden>" : entry.getValue()));
+      Property prop = Property.getPropertyByKey(key);
+      if (prop != null && conf.isPropertySet(prop)) {
+        if (prop.isDeprecated()) {
+          Property replacedBy = prop.replacedBy();
+          if (replacedBy != null) {
+            log.warn("{} is deprecated, use {} instead.", prop.getKey(), replacedBy.getKey());
+          } else {
+            log.warn("{} is deprecated", prop.getKey());
+          }
+        }
+      }
     }
 
     monitorSwappiness(conf);


[accumulo] 03/03: Merge branch '726a'

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

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

commit d83837c81a29f253e41bac313ae768963320e71a
Merge: 2468f0b 565be32
Author: Mike Walch <mw...@apache.org>
AuthorDate: Tue Oct 30 15:51:09 2018 -0400

    Merge branch '726a'

 .../accumulo/core/conf/ConfigSanityCheck.java      |  14 ---
 .../org/apache/accumulo/core/conf/Property.java    | 115 +++++++++++++--------
 .../org/apache/accumulo/core/conf/ReplacedBy.java  |  33 ++++++
 .../accumulo/core/conf/SiteConfiguration.java      |  44 +++++---
 .../org/apache/accumulo/server/ServerUtil.java     |  11 ++
 5 files changed, 144 insertions(+), 73 deletions(-)


[accumulo] 01/03: Fixes #725 - Log deprecated properties (#726)

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

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

commit 16722b17b0f04aff726be3bd222b4ed2f72a43b0
Author: Michael Schnitzler <sc...@gmail.com>
AuthorDate: Fri Oct 26 12:25:38 2018 +0200

    Fixes #725 - Log deprecated properties (#726)
---
 .../accumulo/core/conf/ConfigSanityCheck.java      | 14 +++++--
 .../org/apache/accumulo/core/conf/Property.java    | 46 ++++++++++++++--------
 .../org/apache/accumulo/core/conf/ReplacedBy.java  | 33 ++++++++++++++++
 3 files changed, 73 insertions(+), 20 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java
index 1e0d43b..8073d8c 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java
@@ -66,6 +66,16 @@ public class ConfigSanityCheck {
         fatal(PREFIX + "improperly formatted value for key (" + key + ", type=" + prop.getType()
             + ") : " + value);
 
+      if (prop!=null) { 
+          if (prop.hasAnnotation(Deprecated.class)) {
+            log.warn("Use of {} is deprecated.", key);
+          }
+          ReplacedBy replacedBy = prop.getAnnotation(ReplacedBy.class);
+          if (replacedBy != null) {
+            log.warn("Consider using {} instead of {}.", replacedBy.replacedByProperty(), key);
+          }
+      }
+
       if (key.equals(Property.INSTANCE_ZK_TIMEOUT.getKey())) {
         instanceZkTimeoutValue = value;
       }
@@ -94,10 +104,6 @@ public class ConfigSanityCheck {
           new CheckTimeDurationBetween(1000, 300000));
     }
 
-    if (!usingVolumes) {
-      log.warn("Use of {} and {} are deprecated. Consider using {} instead.", INSTANCE_DFS_URI,
-          INSTANCE_DFS_DIR, Property.INSTANCE_VOLUMES);
-    }
   }
 
   private interface CheckTimeDuration {
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 5272646..2e919be 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
@@ -96,21 +96,6 @@ public enum Property {
       "Zookeeper session timeout; "
           + "max value when represented as milliseconds should be no larger than "
           + Integer.MAX_VALUE),
-  @Deprecated
-  INSTANCE_DFS_URI("instance.dfs.uri", "", PropertyType.URI,
-      "This property is deprecated since 1.6.0. "
-          + "A url accumulo should use to connect to DFS. If this is empty, accumulo"
-          + " will obtain this information from the hadoop configuration. This property"
-          + " will only be used when creating new files if instance.volumes is empty."
-          + " After an upgrade to 1.6.0 Accumulo will start using absolute paths to"
-          + " reference files. Files created before a 1.6.0 upgrade are referenced via"
-          + " relative paths. Relative paths will always be resolved using this config"
-          + " (if empty using the hadoop config)."),
-  @Deprecated
-  INSTANCE_DFS_DIR("instance.dfs.dir", "/accumulo", PropertyType.ABSOLUTEPATH,
-      "This property is deprecated since 1.6.0. "
-          + "HDFS directory in which accumulo instance will run. "
-          + "Do not change after accumulo is initialized."),
   @Sensitive
   INSTANCE_SECRET("instance.secret", "DEFAULT", PropertyType.STRING,
       "A secret unique to a given instance that all servers must know in order"
@@ -131,6 +116,23 @@ public enum Property {
           + " reference files created at that location before the config change. To use"
           + " a comma or other reserved characters in a URI use standard URI hex"
           + " encoding. For example replace commas with %2C."),
+  @Deprecated
+  @ReplacedBy(replacedByProperty=INSTANCE_VOLUMES)
+  INSTANCE_DFS_URI("instance.dfs.uri", "", PropertyType.URI,
+		  "This property is deprecated since 1.6.0. "
+				  + "A url accumulo should use to connect to DFS. If this is empty, accumulo"
+				  + " will obtain this information from the hadoop configuration. This property"
+				  + " will only be used when creating new files if instance.volumes is empty."
+				  + " After an upgrade to 1.6.0 Accumulo will start using absolute paths to"
+				  + " reference files. Files created before a 1.6.0 upgrade are referenced via"
+				  + " relative paths. Relative paths will always be resolved using this config"
+				  + " (if empty using the hadoop config)."),
+  @Deprecated
+  @ReplacedBy(replacedByProperty=INSTANCE_VOLUMES)
+  INSTANCE_DFS_DIR("instance.dfs.dir", "/accumulo", PropertyType.ABSOLUTEPATH,
+		  "This property is deprecated since 1.6.0. "
+				  + "HDFS directory in which accumulo instance will run. "
+				  + "Do not change after accumulo is initialized."),
   INSTANCE_VOLUMES_REPLACEMENTS("instance.volumes.replacements", "", PropertyType.STRING,
       "Since accumulo stores absolute URIs changing the location of a namenode "
           + "could prevent Accumulo from starting. The property helps deal with "
@@ -1077,7 +1079,7 @@ public enum Property {
     return false;
   }
 
-  private <T extends Annotation> boolean hasAnnotation(Class<T> annotationType) {
+  public <T extends Annotation> boolean hasAnnotation(Class<T> annotationType) {
     Logger log = LoggerFactory.getLogger(getClass());
     try {
       for (Annotation a : getClass().getField(name()).getAnnotations())
@@ -1089,6 +1091,18 @@ public enum Property {
     return false;
   }
 
+  public <T extends Annotation> T getAnnotation(Class<T> annotationType) {
+	    Logger log = LoggerFactory.getLogger(getClass());
+	    try {
+	      for (Annotation a : getClass().getField(name()).getAnnotations())
+	        if (annotationType.isInstance(a))
+	          return (T) a;
+	    } catch (SecurityException | NoSuchFieldException e) {
+	      log.error("{}", e.getMessage(), e);
+	    }
+	    return null;
+	  }
+
   private static <T extends Annotation> boolean hasPrefixWithAnnotation(String key,
       Class<T> annotationType) {
     for (String prefix : validPrefixes) {
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ReplacedBy.java b/core/src/main/java/org/apache/accumulo/core/conf/ReplacedBy.java
new file mode 100644
index 0000000..6d5e525
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ReplacedBy.java
@@ -0,0 +1,33 @@
+/*
+ * 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 that an {@link AccumuloConfiguration} {@link Property} key is replaced by
+ * a different {@link AccumuloConfiguration} {@link Property} key. This usually means that the
+ * {@link AccumuloConfiguration} {@link Property} key is {@link Deprecated}.
+ */
+@Inherited
+@Retention(RetentionPolicy.RUNTIME)
+@interface ReplacedBy {
+
+  Property replacedByProperty();
+}