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/11/14 02:19:50 UTC

[2/4] git commit: ACCUMULO-1859 dramatically sped up iterating over Accumulo configs

ACCUMULO-1859 dramatically sped up iterating over Accumulo configs


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/c32fb190
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/c32fb190
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/c32fb190

Branch: refs/heads/master
Commit: c32fb19084688ff6c4e2c246c72b9e9783d84915
Parents: 9c16ef0
Author: Keith Turner <kt...@apache.org>
Authored: Wed Nov 13 17:50:48 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Wed Nov 13 18:48:08 2013 -0500

----------------------------------------------------------------------
 .../apache/accumulo/core/cli/ClientOpts.java    | 16 ++--
 .../core/client/mock/MockConfiguration.java     | 11 ++-
 .../core/conf/AccumuloConfiguration.java        | 43 +++++++++--
 .../accumulo/core/conf/ConfigurationCopy.java   |  9 ++-
 .../core/conf/DefaultConfiguration.java         | 26 +++++--
 .../accumulo/core/conf/SiteConfiguration.java   | 21 ++---
 .../accumulo/core/iterators/IteratorUtil.java   | 80 ++++++++++----------
 .../server/conf/TableConfiguration.java         | 35 ++++-----
 .../accumulo/server/conf/ZooConfiguration.java  | 30 ++------
 9 files changed, 144 insertions(+), 127 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c32fb190/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
index 9247d56..3013cec 100644
--- a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
+++ b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
@@ -16,11 +16,9 @@
  */
 package org.apache.accumulo.core.cli;
 
-import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.TreeMap;
 import java.util.UUID;
 
 import org.apache.accumulo.core.Constants;
@@ -202,13 +200,13 @@ public class ClientOpts extends Help {
         }
         
         @Override
-        public Iterator<Entry<String,String>> iterator() {
-          TreeMap<String,String> map = new TreeMap<String,String>();
-          for (Entry<String,String> props : DefaultConfiguration.getInstance())
-            map.put(props.getKey(), props.getValue());
-          for (Entry<String,String> props : xml)
-            map.put(props.getKey(), props.getValue());
-          return map.entrySet().iterator();
+        public void getProperties(Map<String,String> props, PropertyFilter filter) {
+          for (Entry<String,String> prop : DefaultConfiguration.getInstance())
+            if (filter.accept(prop.getKey()))
+              props.put(prop.getKey(), prop.getValue());
+          for (Entry<String,String> prop : xml)
+            if (filter.accept(prop.getKey()))
+              props.put(prop.getKey(), prop.getValue());
         }
         
         @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c32fb190/core/src/main/java/org/apache/accumulo/core/client/mock/MockConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConfiguration.java
index b3a0007..ce262a2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConfiguration.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 
@@ -38,9 +37,13 @@ class MockConfiguration extends AccumuloConfiguration {
   public String get(Property property) {
     return map.get(property.getKey());
   }
-  
+
   @Override
-  public Iterator<Entry<String,String>> iterator() {
-    return map.entrySet().iterator();
+  public void getProperties(Map<String,String> props, PropertyFilter filter) {
+    for (Entry<String,String> entry : map.entrySet()) {
+      if (filter.accept(entry.getKey())) {
+        props.put(entry.getKey(), entry.getValue());
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c32fb190/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
----------------------------------------------------------------------
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 0a456ba..da170e9 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
@@ -20,6 +20,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.TreeMap;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.Connector;
@@ -29,12 +30,44 @@ import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 import org.apache.log4j.Logger;
 
 public abstract class AccumuloConfiguration implements Iterable<Entry<String,String>> {
+
+  public static interface PropertyFilter {
+    boolean accept(String key);
+  }
+
+  public static class AllFilter implements PropertyFilter {
+    @Override
+    public boolean accept(String key) {
+      return true;
+    }
+  }
+
+  public static class PrefixFilter implements PropertyFilter {
+
+    private String prefix;
+
+    public PrefixFilter(String prefix) {
+      this.prefix = prefix;
+    }
+
+    @Override
+    public boolean accept(String key) {
+      return key.startsWith(prefix);
+    }
+  }
+
   private static final Logger log = Logger.getLogger(AccumuloConfiguration.class);
   
   public abstract String get(Property property);
   
+  public abstract void getProperties(Map<String,String> props, PropertyFilter filter);
+
   @Override
-  public abstract Iterator<Entry<String,String>> iterator();
+  public Iterator<Entry<String,String>> iterator() {
+    TreeMap<String,String> entries = new TreeMap<String,String>();
+    getProperties(entries, new AllFilter());
+    return entries.entrySet().iterator();
+  }
   
   private void checkType(Property property, PropertyType type) {
     if (!property.getType().equals(type)) {
@@ -56,13 +89,7 @@ public abstract class AccumuloConfiguration implements Iterable<Entry<String,Str
     checkType(property, PropertyType.PREFIX);
     
     Map<String,String> propMap = new HashMap<String,String>();
-    
-    for (Entry<String,String> entry : this) {
-      if (entry.getKey().startsWith(property.getKey())) {
-        propMap.put(entry.getKey(), entry.getValue());
-      }
-    }
-    
+    getProperties(propMap, new PrefixFilter(property.getKey()));
     return propMap;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c32fb190/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationCopy.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationCopy.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationCopy.java
index 4755ce1..756a97b 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationCopy.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationCopy.java
@@ -18,7 +18,6 @@ package org.apache.accumulo.core.conf;
 
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 
@@ -45,8 +44,12 @@ public class ConfigurationCopy extends AccumuloConfiguration {
   }
   
   @Override
-  public Iterator<Entry<String,String>> iterator() {
-    return copy.entrySet().iterator();
+  public void getProperties(Map<String,String> props, PropertyFilter filter) {
+    for (Entry<String,String> entry : copy.entrySet()) {
+      if (filter.accept(entry.getKey())) {
+        props.put(entry.getKey(), entry.getValue());
+      }
+    }
   }
   
   public void set(Property prop, String value) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c32fb190/core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java
index 29cc69e..cfc660e 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java
@@ -21,7 +21,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
 import java.util.ArrayList;
-import java.util.Iterator;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.TreeMap;
 
@@ -30,6 +31,7 @@ import org.apache.log4j.Logger;
 public class DefaultConfiguration extends AccumuloConfiguration {
   private static DefaultConfiguration instance = null;
   private static Logger log = Logger.getLogger(DefaultConfiguration.class);
+  private Map<String,String> resolvedProps = null;
 
   synchronized public static DefaultConfiguration getInstance() {
     if (instance == null) {
@@ -44,14 +46,22 @@ public class DefaultConfiguration extends AccumuloConfiguration {
     return property.getDefaultValue();
   }
 
-  @Override
-  public Iterator<Entry<String,String>> iterator() {
-    TreeMap<String,String> entries = new TreeMap<String,String>();
-    for (Property prop : Property.values())
-      if (!prop.isExperimental() && !prop.getType().equals(PropertyType.PREFIX))
-        entries.put(prop.getKey(), prop.getDefaultValue());
+  private synchronized Map<String,String> getResolvedProps() {
+    if (resolvedProps == null) {
+      // the following loop is super slow, it takes a few milliseconds, so cache it
+      resolvedProps = new HashMap<String,String>();
+      for (Property prop : Property.values())
+        if (!prop.isExperimental() && !prop.getType().equals(PropertyType.PREFIX))
+          resolvedProps.put(prop.getKey(), prop.getDefaultValue());
+    }
+    return resolvedProps;
+  }
 
-    return entries.entrySet().iterator();
+  @Override
+  public void getProperties(Map<String,String> props, PropertyFilter filter) {
+    for (Entry<String,String> entry : getResolvedProps().entrySet())
+      if (filter.accept(entry.getKey()))
+        props.put(entry.getKey(), entry.getValue());
   }
 
   protected static void generateDocumentation(PrintStream doc) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c32fb190/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
----------------------------------------------------------------------
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 a1e2572..8f1f72a 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
@@ -16,9 +16,8 @@
  */
 package org.apache.accumulo.core.conf;
 
-import java.util.Iterator;
+import java.util.Map;
 import java.util.Map.Entry;
-import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
@@ -71,18 +70,14 @@ public class SiteConfiguration extends AccumuloConfiguration {
   }
   
   @Override
-  public Iterator<Entry<String,String>> iterator() {
-    TreeMap<String,String> entries = new TreeMap<String,String>();
-    
-    for (Entry<String,String> parentEntry : parent)
-      entries.put(parentEntry.getKey(), parentEntry.getValue());
-    
-    for (Entry<String,String> siteEntry : getXmlConfig())
-      entries.put(siteEntry.getKey(), siteEntry.getValue());
-    
-    return entries.entrySet().iterator();
+  public void getProperties(Map<String,String> props, PropertyFilter filter) {
+    parent.getProperties(props, filter);
+
+    for (Entry<String,String> entry : getXmlConfig())
+      if (filter.accept(entry.getKey()))
+        props.put(entry.getKey(), entry.getValue());
   }
-  
+
   /**
    * method here to support testing, do not call
    */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c32fb190/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
index 55c3c27..977e304 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
@@ -100,51 +100,49 @@ public class IteratorUtil {
   }
   
   private static void parseIterConf(IteratorScope scope, List<IterInfo> iters, Map<String,Map<String,String>> allOptions, AccumuloConfiguration conf) {
-    for (Entry<String,String> entry : conf) {
-      if (entry.getKey().startsWith(Property.TABLE_ITERATOR_PREFIX.getKey())) {
-        
-        String suffix = entry.getKey().substring(Property.TABLE_ITERATOR_PREFIX.getKey().length());
-        String suffixSplit[] = suffix.split("\\.", 4);
-        
-        if (!suffixSplit[0].equals(scope.name())) {
-          
-          // do a sanity check to see if this is a valid scope
-          boolean found = false;
-          IteratorScope[] scopes = IteratorScope.values();
-          for (IteratorScope s : scopes) {
-            found = found || suffixSplit[0].equals(s.name());
-          }
-          
-          if (!found) {
-            log.warn("Option contains unknown scope: " + entry.getKey());
-          }
-          
-          continue;
+    for (Entry<String,String> entry : conf.getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_PREFIX).entrySet()) {
+
+      String suffix = entry.getKey().substring(Property.TABLE_ITERATOR_PREFIX.getKey().length());
+      String suffixSplit[] = suffix.split("\\.", 4);
+
+      if (!suffixSplit[0].equals(scope.name())) {
+
+        // do a sanity check to see if this is a valid scope
+        boolean found = false;
+        IteratorScope[] scopes = IteratorScope.values();
+        for (IteratorScope s : scopes) {
+          found = found || suffixSplit[0].equals(s.name());
         }
-        
-        if (suffixSplit.length == 2) {
-          String sa[] = entry.getValue().split(",");
-          int prio = Integer.parseInt(sa[0]);
-          String className = sa[1];
-          iters.add(new IterInfo(prio, className, suffixSplit[1]));
-        } else if (suffixSplit.length == 4 && suffixSplit[2].equals("opt")) {
-          String iterName = suffixSplit[1];
-          String optName = suffixSplit[3];
-          
-          Map<String,String> options = allOptions.get(iterName);
-          if (options == null) {
-            options = new HashMap<String,String>();
-            allOptions.put(iterName, options);
-          }
-          
-          options.put(optName, entry.getValue());
-          
-        } else {
-          log.warn("Unrecognizable option: " + entry.getKey());
+
+        if (!found) {
+          log.warn("Option contains unknown scope: " + entry.getKey());
+        }
+
+        continue;
+      }
+
+      if (suffixSplit.length == 2) {
+        String sa[] = entry.getValue().split(",");
+        int prio = Integer.parseInt(sa[0]);
+        String className = sa[1];
+        iters.add(new IterInfo(prio, className, suffixSplit[1]));
+      } else if (suffixSplit.length == 4 && suffixSplit[2].equals("opt")) {
+        String iterName = suffixSplit[1];
+        String optName = suffixSplit[3];
+
+        Map<String,String> options = allOptions.get(iterName);
+        if (options == null) {
+          options = new HashMap<String,String>();
+          allOptions.put(iterName, options);
         }
+
+        options.put(optName, entry.getValue());
+
+      } else {
+        log.warn("Unrecognizable option: " + entry.getKey());
       }
     }
-    
+
     Collections.sort(iters, new IterInfoComparator());
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c32fb190/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
index 4c58153..cb13817 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
@@ -19,11 +19,9 @@ package org.apache.accumulo.server.conf;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
-import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
@@ -103,7 +101,7 @@ public class TableConfiguration extends AccumuloConfiguration {
   @Override
   public String get(Property property) {
     String key = property.getKey();
-    String value = get(key);
+    String value = get(getTablePropCache(), key);
     
     if (value == null || !property.getType().isValidFormat(value)) {
       if (value != null)
@@ -113,32 +111,31 @@ public class TableConfiguration extends AccumuloConfiguration {
     return value;
   }
   
-  private String get(String key) {
+  private String get(ZooCache zc, String key) {
     String zPath = ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + table + Constants.ZTABLE_CONF + "/" + key;
-    byte[] v = getTablePropCache().get(zPath);
+    byte[] v = zc.get(zPath);
     String value = null;
     if (v != null)
       value = new String(v);
     return value;
   }
-  
+
   @Override
-  public Iterator<Entry<String,String>> iterator() {
-    TreeMap<String,String> entries = new TreeMap<String,String>();
-    
-    for (Entry<String,String> parentEntry : parent)
-      entries.put(parentEntry.getKey(), parentEntry.getValue());
-    
-    List<String> children = getTablePropCache().getChildren(ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + table + Constants.ZTABLE_CONF);
+  public void getProperties(Map<String,String> props, PropertyFilter filter) {
+    parent.getProperties(props, filter);
+
+    ZooCache zc = getTablePropCache();
+
+    List<String> children = zc.getChildren(ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + table + Constants.ZTABLE_CONF);
     if (children != null) {
       for (String child : children) {
-        String value = get(child);
-        if (child != null && value != null)
-          entries.put(child, value);
+        if (child != null && filter.accept(child)) {
+          String value = get(zc, child);
+          if (value != null)
+            props.put(child, value);
+        }
       }
     }
-    
-    return entries.entrySet().iterator();
   }
   
   public String getTableId() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c32fb190/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
index 18381c7..ee1a809 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
@@ -18,11 +18,8 @@ package org.apache.accumulo.server.conf;
 
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
-import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
@@ -120,29 +117,18 @@ public class ZooConfiguration extends AccumuloConfiguration {
   }
   
   @Override
-  public Iterator<Entry<String,String>> iterator() {
-    TreeMap<String,String> entries = new TreeMap<String,String>();
-    
-    for (Entry<String,String> parentEntry : parent)
-      entries.put(parentEntry.getKey(), parentEntry.getValue());
-    
+  public void getProperties(Map<String,String> props, PropertyFilter filter) {
+    parent.getProperties(props, filter);
+
     List<String> children = propCache.getChildren(ZooUtil.getRoot(instanceId) + Constants.ZCONFIG);
     if (children != null) {
       for (String child : children) {
-        String value = get(child);
-        if (child != null && value != null)
-          entries.put(child, value);
+        if (child != null && filter.accept(child)) {
+          String value = get(child);
+          if (value != null)
+            props.put(child, value);
+        }
       }
     }
-    
-    /*
-     * //this code breaks the shells ability to show updates just made //the code is probably not needed as fixed props are only obtained through get
-     * 
-     * for(Property prop : Property.getFixedProperties()) get(prop);
-     * 
-     * for(Entry<String, String> fprop : fixedProps.entrySet()) entries.put(fprop.getKey(), fprop.getValue());
-     */
-    
-    return entries.entrySet().iterator();
   }
 }