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 2013/06/13 19:53:28 UTC

svn commit: r1492786 - in /accumulo/trunk: core/src/main/java/org/apache/accumulo/core/conf/ core/src/test/java/org/apache/accumulo/core/conf/ server/src/main/java/org/apache/accumulo/server/ server/src/main/java/org/apache/accumulo/server/client/ serv...

Author: ctubbsii
Date: Thu Jun 13 17:53:28 2013
New Revision: 1492786

URL: http://svn.apache.org/r1492786
Log:
ACCUMULO-1313 Annotate configuration properties and behave accordingly as Interpolated(interpolate), Deprecated(strikethrough in docs), Sensitive(mask value or hide), and Experimental(hide from docs)

Added:
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Experimental.java   (with props)
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Interpolated.java   (with props)
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Sensitive.java   (with props)
Modified:
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Property.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/Accumulo.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java

Added: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Experimental.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Experimental.java?rev=1492786&view=auto
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Experimental.java (added)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Experimental.java Thu Jun 13 17:53:28 2013
@@ -0,0 +1,30 @@
+/*
+ * 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 experimental {@link AccumuloConfiguration} {@link Property} keys.
+ */
+@Inherited
+@Retention(RetentionPolicy.RUNTIME)
+@interface Experimental {
+  
+}

Propchange: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Experimental.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Interpolated.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Interpolated.java?rev=1492786&view=auto
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Interpolated.java (added)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Interpolated.java Thu Jun 13 17:53:28 2013
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+@Inherited
+@Retention(RetentionPolicy.RUNTIME)
+@interface Interpolated {
+  
+}

Propchange: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Interpolated.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Property.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Property.java?rev=1492786&r1=1492785&r2=1492786&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Property.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Property.java Thu Jun 13 17:53:28 2013
@@ -34,27 +34,35 @@ import org.apache.log4j.Logger;
 
 public enum Property {
   // Crypto-related properties
-  CRYPTO_PREFIX("crypto.", null, PropertyType.PREFIX, "Properties in this category related to the configuration of both default and custom crypto modules.",
-      true, false),
+  @Experimental
+  CRYPTO_PREFIX("crypto.", null, PropertyType.PREFIX, "Properties in this category related to the configuration of both default and custom crypto modules."),
+  @Experimental
   CRYPTO_MODULE_CLASS("crypto.module.class", "NullCryptoModule", PropertyType.STRING,
       "Fully qualified class name of the class that implements the CryptoModule interface, to be used in setting up encryption at rest for the WAL and "
-          + "(future) other parts of the code.", true, false),
-  CRYPTO_CIPHER_SUITE("crypto.cipher.suite", "NullCipher", PropertyType.STRING, "Describes the cipher suite to use for the write-ahead log", true, false),
+          + "(future) other parts of the code."),
+  @Experimental
+  CRYPTO_CIPHER_SUITE("crypto.cipher.suite", "NullCipher", PropertyType.STRING, "Describes the cipher suite to use for the write-ahead log"),
+  @Experimental
   CRYPTO_CIPHER_ALGORITHM_NAME("crypto.cipher.algorithm.name", "NullCipher", PropertyType.STRING,
-      "States the name of the algorithm used in the corresponding cipher suite. Do not make these different, unless you enjoy mysterious exceptions and bugs.",
-      true, false),
+      "States the name of the algorithm used in the corresponding cipher suite. Do not make these different, unless you enjoy mysterious exceptions and bugs."),
+  @Experimental
   CRYPTO_CIPHER_KEY_LENGTH("crypto.cipher.key.length", "128", PropertyType.STRING,
-      "Specifies the key length *in bits* to use for the symmetric key, should probably be 128 or 256 unless you really know what you're doing", true, false),
+      "Specifies the key length *in bits* to use for the symmetric key, should probably be 128 or 256 unless you really know what you're doing"),
+  @Experimental
   CRYPTO_SECURE_RNG("crypto.secure.rng", "SHA1PRNG", PropertyType.STRING,
-      "States the secure random number generator to use, and defaults to the built-in Sun SHA1PRNG", true, false),
+      "States the secure random number generator to use, and defaults to the built-in Sun SHA1PRNG"),
+  @Experimental
   CRYPTO_SECURE_RNG_PROVIDER("crypto.secure.rng.provider", "SUN", PropertyType.STRING,
-      "States the secure random number generator provider to use, and defaults to the built-in SUN provider", true, false),
+      "States the secure random number generator provider to use, and defaults to the built-in SUN provider"),
+  @Experimental
   CRYPTO_SECRET_KEY_ENCRYPTION_STRATEGY_CLASS("crypto.secret.key.encryption.strategy.class", "NullSecretKeyEncryptionStrategy", PropertyType.STRING,
-      "The class Accumulo should use for its key encryption strategy.", true, false),
+      "The class Accumulo should use for its key encryption strategy."),
+  @Experimental
   CRYPTO_DEFAULT_KEY_STRATEGY_HDFS_URI("crypto.default.key.strategy.hdfs.uri", "", PropertyType.STRING,
-      "The URL Accumulo should use to connect to DFS. If this is blank, Accumulo will obtain this information from the Hadoop configuration", true, false),
+      "The URL Accumulo should use to connect to DFS. If this is blank, Accumulo will obtain this information from the Hadoop configuration"),
+  @Experimental
   CRYPTO_DEFAULT_KEY_STRATEGY_KEY_LOCATION("crypto.default.key.strategy.key.location", "/accumulo/crypto/secret/keyEncryptionKey", PropertyType.ABSOLUTEPATH,
-      "The absolute path of where to store the key encryption key within HDFS.", true, false),
+      "The absolute path of where to store the key encryption key within HDFS."),
   
   // instance properties (must be the same for every node in an instance)
   INSTANCE_PREFIX("instance.", null, PropertyType.PREFIX,
@@ -66,6 +74,7 @@ public enum Property {
       "The url accumulo should use to connect to DFS.  If this is empty, accumulo will obtain this information from the hadoop configuration."),
   INSTANCE_DFS_DIR("instance.dfs.dir", "/accumulo", PropertyType.ABSOLUTEPATH,
       "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 to communicate with one another."
           + " Change it before initialization. To change it later use ./bin/accumulo accumulo.server.util.ChangeSecret [oldpasswd] [newpasswd], "
@@ -207,7 +216,7 @@ public enum Property {
       "The property only needs to be set if upgrading from 1.4 which used to store write-ahead logs on the local filesystem. In 1.5 write-ahead logs are "
           + "stored in DFS.  When 1.5 is started for the first time it will copy any 1.4 write ahead logs into DFS.  It is possible to specify a "
           + "comma-separated list of directories."),
-
+  
   // accumulo garbage collector properties
   GC_PREFIX("gc.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of the accumulo garbage collector."),
   GC_CYCLE_START("gc.cycle.start", "30s", PropertyType.TIMEDURATION, "Time to wait before attempting to garbage collect any old files."),
@@ -225,16 +234,24 @@ public enum Property {
   MONITOR_BANNER_COLOR("monitor.banner.color", "#c4c4c4", PropertyType.STRING, "The color of the banner text displayed on the monitor page."),
   MONITOR_BANNER_BACKGROUND("monitor.banner.background", "#304065", PropertyType.STRING,
       "The background color of the banner text displayed on the monitor page."),
-  MONITOR_SSL_KEYSTORE("monitor.ssl.keyStore", "", PropertyType.PATH, "The keystore for enabling monitor SSL.", true, false),
-  MONITOR_SSL_KEYSTOREPASS("monitor.ssl.keyStorePassword", "", PropertyType.STRING, "The keystore password for enabling monitor SSL.", true, false),
-  MONITOR_SSL_TRUSTSTORE("monitor.ssl.trustStore", "", PropertyType.PATH, "The truststore for enabling monitor SSL.", true, false),
-  MONITOR_SSL_TRUSTSTOREPASS("monitor.ssl.trustStorePassword", "", PropertyType.STRING, "The truststore password for enabling monitor SSL.", true, false),
+  @Experimental
+  MONITOR_SSL_KEYSTORE("monitor.ssl.keyStore", "", PropertyType.PATH, "The keystore for enabling monitor SSL."),
+  @Experimental
+  @Sensitive
+  MONITOR_SSL_KEYSTOREPASS("monitor.ssl.keyStorePassword", "", PropertyType.STRING, "The keystore password for enabling monitor SSL."),
+  @Experimental
+  MONITOR_SSL_TRUSTSTORE("monitor.ssl.trustStore", "", PropertyType.PATH, "The truststore for enabling monitor SSL."),
+  @Experimental
+  @Sensitive
+  MONITOR_SSL_TRUSTSTOREPASS("monitor.ssl.trustStorePassword", "", PropertyType.STRING, "The truststore password for enabling monitor SSL."),
   
   TRACE_PREFIX("trace.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of distributed tracing."),
   TRACE_PORT("trace.port.client", "12234", PropertyType.PORT, "The listening port for the trace server"),
   TRACE_TABLE("trace.table", "trace", PropertyType.STRING, "The name of the table to store distributed traces"),
   TRACE_USER("trace.user", "root", PropertyType.STRING, "The name of the user to store distributed traces"),
+  @Sensitive
   TRACE_PASSWORD("trace.password", "secret", PropertyType.STRING, "The password for the user used to store distributed traces"),
+  @Sensitive
   TRACE_TOKEN_PROPERTY_PREFIX("trace.token.property", null, PropertyType.PREFIX,
       "The prefix used to create a token for storing distributed traces.  For each propetry required by trace.token.type, place this prefix in front of it."),
   TRACE_TOKEN_TYPE("trace.token.type", PasswordToken.class.getName(), PropertyType.CLASSNAME, "An AuthenticationToken type supported by the authorizer"),
@@ -344,27 +361,20 @@ public enum Property {
           + "You can enable post delegation for a context, which will load classes from the context first instead of the parent first.  "
           + "Do this by setting 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, "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}", false, true);
+          + " This should be on local disk on each node with sufficient space. It defaults to ${java.io.tmpdir}/accumulo-vfs-cache-${user.name}");
   
   private String key, defaultValue, description;
   private PropertyType type;
-  private boolean experimental;
-  private boolean interpolated;
   static Logger log = Logger.getLogger(Property.class);
   
-  private Property(String name, String defaultValue, PropertyType type, String description, boolean experimental, boolean interpolated) {
+  private Property(String name, String defaultValue, PropertyType type, String description) {
     this.key = name;
     this.defaultValue = defaultValue;
     this.description = description;
     this.type = type;
-    this.experimental = experimental;
-    this.interpolated = interpolated;
-  }
-  
-  private Property(String name, String defaultValue, PropertyType type, String description) {
-    this(name, defaultValue, type, description, false, false);
   }
   
   @Override
@@ -381,7 +391,7 @@ public enum Property {
   }
   
   public String getDefaultValue() {
-    if (this.interpolated) {
+    if (isInterpolated()) {
       PropertiesConfiguration pconf = new PropertiesConfiguration();
       pconf.append(new SystemConfiguration());
       pconf.addProperty("hack_default_value", this.defaultValue);
@@ -403,8 +413,54 @@ public enum Property {
     return this.description;
   }
   
+  private boolean isInterpolated() {
+    return hasAnnotation(Interpolated.class) || hasPrefixWithAnnotation(getKey(), Interpolated.class);
+  }
+  
   public boolean isExperimental() {
-    return experimental;
+    return hasAnnotation(Experimental.class) || hasPrefixWithAnnotation(getKey(), Experimental.class);
+  }
+  
+  public boolean isDeprecated() {
+    return hasAnnotation(Deprecated.class) || hasPrefixWithAnnotation(getKey(), Deprecated.class);
+  }
+  
+  public boolean isSensitive() {
+    return hasAnnotation(Sensitive.class) || hasPrefixWithAnnotation(getKey(), Sensitive.class);
+  }
+  
+  public static boolean isSensitive(String key) {
+    return hasPrefixWithAnnotation(key, Sensitive.class);
+  }
+  
+  private <T extends Annotation> boolean hasAnnotation(Class<T> annotationType) {
+    Logger log = Logger.getLogger(getClass());
+    try {
+      for (Annotation a : getClass().getField(name()).getAnnotations())
+        if (annotationType.isInstance(a))
+          return true;
+    } catch (SecurityException e) {
+      log.error(e, e);
+    } catch (NoSuchFieldException e) {
+      log.error(e, e);
+    }
+    return false;
+  }
+  
+  private static <T extends Annotation> boolean hasPrefixWithAnnotation(String key, Class<T> annotationType) {
+    // relies on side-effects of isValidPropertyKey to populate validPrefixes
+    if (isValidPropertyKey(key)) {
+      // check if property exists on its own and has the annotation
+      if (Property.getPropertyByKey(key) != null)
+        return getPropertyByKey(key).hasAnnotation(annotationType);
+      // can't find the property, so check the prefixes
+      boolean prefixHasAnnotation = false;
+      for (String prefix : validPrefixes)
+        if (key.startsWith(prefix))
+          prefixHasAnnotation = prefixHasAnnotation || getPropertyByKey(prefix).hasAnnotation(annotationType);
+      return prefixHasAnnotation;
+    }
+    return false;
   }
   
   private static HashSet<String> validTableProperties = null;
@@ -485,20 +541,6 @@ public enum Property {
         || key.equals(Property.TABLE_LOAD_BALANCER.getKey());
   }
   
-  public boolean isDeprecated() {
-    Logger log = Logger.getLogger(getClass());
-    try {
-      for (Annotation a : getClass().getField(name()).getAnnotations())
-        if (a instanceof Deprecated)
-          return true;
-    } catch (SecurityException e) {
-      log.error(e, e);
-    } catch (NoSuchFieldException e) {
-      log.error(e, e);
-    }
-    return false;
-  }
-  
   public static <T> T createInstanceFromPropertyName(AccumuloConfiguration conf, Property property, Class<T> base, T defaultInstance) {
     String clazzName = conf.get(property);
     T instance = null;

Added: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Sensitive.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Sensitive.java?rev=1492786&view=auto
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Sensitive.java (added)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Sensitive.java Thu Jun 13 17:53:28 2013
@@ -0,0 +1,30 @@
+/*
+ * 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 which are sensitive, and should be masked or hidden when printed.
+ */
+@Inherited
+@Retention(RetentionPolicy.RUNTIME)
+@interface Sensitive {
+  
+}

Propchange: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/conf/Sensitive.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java?rev=1492786&r1=1492785&r2=1492786&view=diff
==============================================================================
--- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java (original)
+++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java Thu Jun 13 17:53:28 2013
@@ -22,6 +22,10 @@ import static org.junit.Assert.assertTru
 
 import java.io.File;
 import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+import java.util.TreeSet;
 
 import org.junit.Test;
 
@@ -98,7 +102,8 @@ public class PropertyTest {
     typeCheckValidFormat(PropertyType.ABSOLUTEPATH, "/foo", "/foo/c", "/");
     // in hadoop 2.0 Path only normalizes Windows paths properly when run on a Windows system
     // this makes the following checks fail
-    // typeCheckValidFormat(PropertyType.ABSOLUTEPATH, "d:\\foo12", "c:\\foo\\g", "c:\\foo\\c", "c:\\");
+    if (System.getProperty("os.name").toLowerCase().contains("windows"))
+      typeCheckValidFormat(PropertyType.ABSOLUTEPATH, "d:\\foo12", "c:\\foo\\g", "c:\\foo\\c", "c:\\");
     typeCheckInvalidFormat(PropertyType.ABSOLUTEPATH, "foo12", "foo/g", "foo\\c");
   }
   
@@ -109,4 +114,50 @@ public class PropertyTest {
     assertEquals(new File(System.getProperty("java.io.tmpdir"), "accumulo-vfs-cache-" + System.getProperty("user.name")).getAbsolutePath(),
         conf.get(Property.VFS_CLASSLOADER_CACHE_DIR));
   }
+  
+  @Test
+  public void testSensitiveKeys() {
+    final TreeMap<String,String> extras = new TreeMap<String,String>();
+    extras.put("trace.token.property.blah", "something");
+    
+    AccumuloConfiguration conf = new DefaultConfiguration() {
+      @Override
+      public Iterator<Entry<String,String>> iterator() {
+        final Iterator<Entry<String,String>> parent = super.iterator();
+        final Iterator<Entry<String,String>> mine = extras.entrySet().iterator();
+        
+        return new Iterator<Entry<String,String>>() {
+          
+          @Override
+          public boolean hasNext() {
+            return parent.hasNext() || mine.hasNext();
+          }
+          
+          @Override
+          public Entry<String,String> next() {
+            return parent.hasNext() ? parent.next() : mine.next();
+          }
+          
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    TreeSet<String> expected = new TreeSet<String>();
+    for (Entry<String,String> entry : conf) {
+      String key = entry.getKey();
+      if (key.equals(Property.INSTANCE_SECRET.getKey()) || key.toLowerCase().contains("password") || key.toLowerCase().contains("secret")
+          || key.startsWith(Property.TRACE_TOKEN_PROPERTY_PREFIX.getKey()))
+        expected.add(key);
+    }
+    TreeSet<String> actual = new TreeSet<String>();
+    for (Entry<String,String> entry : conf) {
+      String key = entry.getKey();
+      if (Property.isSensitive(key))
+        actual.add(key);
+    }
+    assertEquals(expected, actual);
+  }
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/Accumulo.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/Accumulo.java?rev=1492786&r1=1492785&r2=1492786&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/Accumulo.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/Accumulo.java Thu Jun 13 17:53:28 2013
@@ -107,18 +107,16 @@ public class Accumulo {
     }
     // Turn off messages about not being able to reach the remote logger... we protect against that.
     LogLog.setQuietMode(true);
-
-      // Configure logging
+    
+    // Configure logging
     DOMConfigurator.configureAndWatch(logConfig, 5000);
-
+    
     // Read the auditing config
     String auditConfig = String.format("%s/conf/auditLog.xml", System.getenv("ACCUMULO_HOME"), application);
-
-     DOMConfigurator.configureAndWatch(auditConfig, 5000);
-
-
-
-      log.info(application + " starting");
+    
+    DOMConfigurator.configureAndWatch(auditConfig, 5000);
+    
+    log.info(application + " starting");
     log.info("Instance " + config.getInstance().getInstanceID());
     int dataVersion = Accumulo.getAccumuloPersistentVersion(fs);
     log.info("Data Version " + dataVersion);
@@ -134,11 +132,8 @@ public class Accumulo {
       sortedProps.put(entry.getKey(), entry.getValue());
     
     for (Entry<String,String> entry : sortedProps.entrySet()) {
-      if (entry.getKey().toLowerCase().contains("password") || entry.getKey().toLowerCase().contains("secret")
-          || entry.getKey().startsWith(Property.TRACE_TOKEN_PROPERTY_PREFIX.getKey()))
-        log.info(entry.getKey() + " = <hidden>");
-      else
-        log.info(entry.getKey() + " = " + entry.getValue());
+      String key = entry.getKey();
+      log.info(key + " = " + (Property.isSensitive(key) ? "<hidden>" : entry.getValue()));
     }
     
     monitorSwappiness();
@@ -221,7 +216,7 @@ public class Accumulo {
   private static boolean isInSafeMode(FileSystem fs) throws IOException {
     if (!(fs instanceof DistributedFileSystem))
       return false;
-    DistributedFileSystem dfs = (DistributedFileSystem)fs;
+    DistributedFileSystem dfs = (DistributedFileSystem) fs;
     // So this: if (!dfs.setSafeMode(SafeModeAction.SAFEMODE_GET))
     // Becomes this:
     Class<?> safeModeAction;

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java?rev=1492786&r1=1492785&r2=1492786&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java Thu Jun 13 17:53:28 2013
@@ -197,19 +197,15 @@ public class ClientServiceHandler implem
     return security.listUsers(credentials);
   }
   
-  static private Map<String,String> conf(TCredentials credentials, AccumuloConfiguration conf) throws TException {
+  private static Map<String,String> conf(TCredentials credentials, AccumuloConfiguration conf) throws TException {
     security.authenticateUser(credentials, credentials);
     conf.invalidateCache();
     
     Map<String,String> result = new HashMap<String,String>();
     for (Entry<String,String> entry : conf) {
-      if (entry.getKey().equals(Property.INSTANCE_SECRET.getKey()))
-        continue;
-      if (entry.getKey().toLowerCase().contains("password"))
-        continue;
-      if (entry.getKey().startsWith(Property.TRACE_TOKEN_PROPERTY_PREFIX.getKey()))
-        continue;
-      result.put(entry.getKey(), entry.getValue());
+      String key = entry.getKey();
+      if (!Property.isSensitive(key))
+        result.put(key, entry.getValue());
     }
     return result;
   }
@@ -293,7 +289,7 @@ public class ClientServiceHandler implem
     security.authenticateUser(credentials, credentials);
     
     String tableId = checkTableId(tableName, null);
-
+    
     ClassLoader loader = getClass().getClassLoader();
     Class shouldMatch;
     try {

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java?rev=1492786&r1=1492785&r2=1492786&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java Thu Jun 13 17:53:28 2013
@@ -53,10 +53,6 @@ public class TableConfiguration extends 
     this.observers = Collections.synchronizedSet(new HashSet<ConfigurationObserver>());
   }
   
-  /**
-   * @deprecated not for client use
-   */
-  @Deprecated
   private static ZooCache getTablePropCache() {
     Instance inst = HdfsZooInstance.getInstance();
     if (tablePropCache == null)
@@ -104,6 +100,7 @@ public class TableConfiguration extends 
       co.propertiesChanged();
   }
   
+  @Override
   public String get(Property property) {
     String key = property.getKey();
     String value = get(key);