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/12/05 00:57:56 UTC

[04/50] [abbrv] git commit: ACCUMULO-802 added a few more namespace-specific randomwalk tests

ACCUMULO-802 added a few more namespace-specific randomwalk tests


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 431f4e2c72a45e86c5f4a59fa3ec9119349b4906
Parents: be1ff27
Author: Sean Hickey <ta...@gmail.com>
Authored: Wed Jul 31 12:50:06 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:09 2013 -0500

----------------------------------------------------------------------
 .../conf/TableNamespaceConfiguration.java       |  37 ++++---
 .../test/randomwalk/concurrent/Config.java      | 111 ++++++++++++++-----
 .../concurrent/OfflineTableNamespace.java       |  53 +++++++++
 .../randomwalk/conf/modules/Concurrent.xml      |   5 +
 4 files changed, 162 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/431f4e2c/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java
index e3c58f3..60da78a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java
@@ -16,6 +16,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;
@@ -25,6 +28,7 @@ import java.util.TreeMap;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
@@ -33,7 +37,7 @@ import org.apache.log4j.Logger;
 
 public class TableNamespaceConfiguration extends AccumuloConfiguration {
   private static final Logger log = Logger.getLogger(TableNamespaceConfiguration.class);
-  
+
   private final AccumuloConfiguration parent;
   private static ZooCache propCache = null;
   private String namespaceId = null;
@@ -46,12 +50,12 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     this.namespaceId = namespaceId;
     this.observers = Collections.synchronizedSet(new HashSet<ConfigurationObserver>());
   }
-  
+
   @Override
   public String get(Property property) {
     String key = property.getKey();
     String value = get(key);
-    
+
     if (value == null || !property.getType().isValidFormat(value)) {
       if (value != null)
         log.error("Using default value for " + key + " due to improperly formatted " + property.getType() + ": " + value);
@@ -59,17 +63,16 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     }
     return value;
   }
-  
+
   private String get(String key) {
-    String zPath = ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF + "/"
-        + key;
+    String zPath = ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF + "/" + key;
     byte[] v = getPropCache().get(zPath);
     String value = null;
     if (v != null)
       value = new String(v, Constants.UTF8);
     return value;
   }
-  
+
   private static ZooCache getPropCache() {
     Instance inst = HdfsZooInstance.getInstance();
     if (propCache == null)
@@ -79,14 +82,14 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
       }
     return propCache;
   }
-  
+
   @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 = getPropCache().getChildren(
         ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF);
     if (children != null) {
@@ -96,14 +99,14 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
           entries.put(child, value);
       }
     }
-    
+
     return entries.entrySet().iterator();
   }
-  
+
   private String getNamespaceId() {
     return namespaceId;
   }
-  
+
   public void addObserver(ConfigurationObserver co) {
     if (namespaceId == null) {
       String err = "Attempt to add observer for non-table-namespace configuration";
@@ -113,7 +116,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     iterator();
     observers.add(co);
   }
-  
+
   public void removeObserver(ConfigurationObserver configObserver) {
     if (namespaceId == null) {
       String err = "Attempt to remove observer for non-table-namespace configuration";
@@ -122,19 +125,19 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     }
     observers.remove(configObserver);
   }
-  
+
   public void expireAllObservers() {
     Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
     for (ConfigurationObserver co : copy)
       co.sessionExpired();
   }
-  
+
   public void propertyChanged(String key) {
     Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
     for (ConfigurationObserver co : copy)
       co.propertyChanged(key);
   }
-  
+
   public void propertiesChanged(String key) {
     Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
     for (ConfigurationObserver co : copy)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/431f4e2c/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
index b47f1a9..5a2008f 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
@@ -29,24 +29,30 @@ import org.apache.commons.math.random.RandomData;
 import org.apache.commons.math.random.RandomDataImpl;
 
 public class Config extends Test {
-  
+
   private static final String LAST_SETTING = "lastSetting";
 
   private static final String LAST_TABLE_SETTING = "lastTableSetting";
 
+  private static final String LAST_NAMESPACE_SETTING = "lastNamespaceSetting";
+
   static class Setting {
     public Property property;
     public long min;
     public long max;
+
     public Setting(Property property, long min, long max) {
       this.property = property;
       this.min = min;
       this.max = max;
     }
   }
+
   static Setting s(Property property, long min, long max) {
     return new Setting(property, min, max);
   }
+
+  /* @formatter:off */
   Setting[] settings = {
       s(Property.TSERV_BLOOM_LOAD_MAXCONCURRENT, 1, 10),
       s(Property.TSERV_BULK_PROCESS_THREADS, 1, 10),
@@ -61,19 +67,19 @@ public class Config extends Test {
       s(Property.TSERV_MAJC_THREAD_MAXOPEN, 3, 100),
       s(Property.TSERV_MINC_MAXCONCURRENT, 1, 10),
       s(Property.TSERV_DEFAULT_BLOCKSIZE, 100000, 10000000L),
-      s(Property.TSERV_MAX_IDLE, 10000, 500*1000),
-      s(Property.TSERV_MAXMEM, 1000000, 3*1024*1024*1024L),
+      s(Property.TSERV_MAX_IDLE, 10000, 500 * 1000),
+      s(Property.TSERV_MAXMEM, 1000000, 3 * 1024 * 1024 * 1024L),
       s(Property.TSERV_READ_AHEAD_MAXCONCURRENT, 1, 25),
       s(Property.TSERV_MIGRATE_MAXCONCURRENT, 1, 10),
-      s(Property.TSERV_MUTATION_QUEUE_MAX, 10000, 1024*1024),
+      s(Property.TSERV_MUTATION_QUEUE_MAX, 10000, 1024 * 1024),
       s(Property.TSERV_RECOVERY_MAX_CONCURRENT, 1, 100),
       s(Property.TSERV_SCAN_MAX_OPENFILES, 10, 1000),
       s(Property.TSERV_THREADCHECK, 100, 10000),
       s(Property.TSERV_MINTHREADS, 1, 100),
-      s(Property.TSERV_SESSION_MAXIDLE, 100, 5*60*1000),
-      s(Property.TSERV_SORT_BUFFER_SIZE, 1024*1024, 1024*1024*1024L),
+      s(Property.TSERV_SESSION_MAXIDLE, 100, 5 * 60 * 1000),
+      s(Property.TSERV_SORT_BUFFER_SIZE, 1024 * 1024, 1024 * 1024 * 1024L),
       s(Property.TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN, 5, 100),
-      s(Property.TSERV_WAL_BLOCKSIZE, 1024*1024, 1024*1024*1024*10L),
+      s(Property.TSERV_WAL_BLOCKSIZE, 1024 * 1024, 1024 * 1024 * 1024 * 10L),
       s(Property.TSERV_WORKQ_THREADS, 1, 10),
       s(Property.MASTER_BULK_THREADPOOL_SIZE, 1, 10),
       s(Property.MASTER_BULK_RETRIES, 1, 10),
@@ -83,21 +89,21 @@ public class Config extends Test {
       s(Property.MASTER_LEASE_RECOVERY_WAITING_PERIOD, 0, 10),
       s(Property.MASTER_RECOVERY_MAXTIME, 10, 1000),
       s(Property.MASTER_THREADCHECK, 100, 10000),
-      s(Property.MASTER_MINTHREADS, 1, 200),
-  };
-  
+      s(Property.MASTER_MINTHREADS, 1, 200),};
+
   Setting[] tableSettings = {
       s(Property.TABLE_MAJC_RATIO, 1, 10),
-      s(Property.TABLE_MAJC_COMPACTALL_IDLETIME, 100, 10*60*60*1000L),
-      s(Property.TABLE_SPLIT_THRESHOLD, 10*1024, 10L*1024*1024*1024),
-      s(Property.TABLE_MINC_COMPACT_IDLETIME, 100, 100*60*60*1000L),
-      s(Property.TABLE_SCAN_MAXMEM, 10*1024, 10*1024*1024),
-      s(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE, 10*1024, 10*1024*1024L), 
-      s(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX, 10*1024, 10*1024*1024L),
+      s(Property.TABLE_MAJC_COMPACTALL_IDLETIME, 100, 10 * 60 * 60 * 1000L),
+      s(Property.TABLE_SPLIT_THRESHOLD, 10 * 1024, 10L * 1024 * 1024 * 1024),
+      s(Property.TABLE_MINC_COMPACT_IDLETIME, 100, 100 * 60 * 60 * 1000L),
+      s(Property.TABLE_SCAN_MAXMEM, 10 * 1024, 10 * 1024 * 1024),
+      s(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE, 10 * 1024, 10 * 1024 * 1024L),
+      s(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX, 10 * 1024, 10 * 1024 * 1024L),
       s(Property.TABLE_FILE_REPLICATION, 0, 5),
-      s(Property.TABLE_FILE_MAX, 2, 50),
-  };
-  
+      s(Property.TABLE_FILE_MAX, 2, 50),};
+
+  /* @formatter:on */
+
   @Override
   public void visit(State state, Properties props) throws Exception {
     // reset any previous setting
@@ -120,7 +126,27 @@ public class Config extends Test {
           state.getConnector().tableOperations().setProperty(table, property.getKey(), property.getDefaultValue());
         } catch (AccumuloException ex) {
           if (ex.getCause() instanceof ThriftTableOperationException) {
-            ThriftTableOperationException ttoe = (ThriftTableOperationException)ex.getCause();
+            ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
+            if (ttoe.type == TableOperationExceptionType.NOTFOUND)
+              return;
+          }
+          throw ex;
+        }
+      }
+    }
+    lastSetting = state.getMap().get(LAST_NAMESPACE_SETTING);
+    if (lastSetting != null) {
+      String parts[] = lastSetting.toString().split(",");
+      String namespace = parts[0];
+      int choice = Integer.parseInt(parts[1]);
+      Property property = tableSettings[choice].property;
+      if (state.getConnector().tableNamespaceOperations().exists(namespace)) {
+        log.debug("Setting " + property.getKey() + " on " + namespace + " back to " + property.getDefaultValue());
+        try {
+          state.getConnector().tableNamespaceOperations().setProperty(namespace, property.getKey(), property.getDefaultValue());
+        } catch (AccumuloException ex) {
+          if (ex.getCause() instanceof ThriftTableOperationException) {
+            ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
             if (ttoe.type == TableOperationExceptionType.NOTFOUND)
               return;
           }
@@ -130,9 +156,13 @@ public class Config extends Test {
     }
     state.getMap().remove(LAST_SETTING);
     state.getMap().remove(LAST_TABLE_SETTING);
+    state.getMap().remove(LAST_NAMESPACE_SETTING);
     RandomData random = new RandomDataImpl();
-    if (random.nextInt(0, 1) == 0) {
+    int dice = random.nextInt(0, 2);
+    if (dice == 0) {
       changeTableSetting(random, state, props);
+    } else if (dice == 1) {
+      changeTableNamespaceSetting(random, state, props);
     } else {
       changeSetting(random, state, props);
     }
@@ -142,13 +172,13 @@ public class Config extends Test {
     // pick a random property
     int choice = random.nextInt(0, tableSettings.length - 1);
     Setting setting = tableSettings[choice];
-    
+
     // pick a random table
     SortedSet<String> tables = state.getConnector().tableOperations().list().tailSet("ctt").headSet("ctu");
     if (tables.isEmpty())
       return;
     String table = random.nextSample(tables, 1)[0].toString();
-    
+
     // generate a random value
     long newValue = random.nextLong(setting.min, setting.max);
     state.getMap().put(LAST_TABLE_SETTING, table + "," + choice);
@@ -157,14 +187,41 @@ public class Config extends Test {
       state.getConnector().tableOperations().setProperty(table, setting.property.getKey(), "" + newValue);
     } catch (AccumuloException ex) {
       if (ex.getCause() instanceof ThriftTableOperationException) {
-        ThriftTableOperationException ttoe = (ThriftTableOperationException)ex.getCause();
+        ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
+        if (ttoe.type == TableOperationExceptionType.NOTFOUND)
+          return;
+      }
+      throw ex;
+    }
+  }
+
+  private void changeTableNamespaceSetting(RandomData random, State state, Properties props) throws Exception {
+    // pick a random property
+    int choice = random.nextInt(0, tableSettings.length - 1);
+    Setting setting = tableSettings[choice];
+
+    // pick a random table
+    SortedSet<String> namespaces = state.getConnector().tableNamespaceOperations().list();
+    if (namespaces.isEmpty())
+      return;
+    String namespace = random.nextSample(namespaces, 1)[0].toString();
+
+    // generate a random value
+    long newValue = random.nextLong(setting.min, setting.max);
+    state.getMap().put(LAST_NAMESPACE_SETTING, namespace + "," + choice);
+    log.debug("Setting " + setting.property.getKey() + " on table namespace " + namespace + " to " + newValue);
+    try {
+      state.getConnector().tableNamespaceOperations().setProperty(namespace, setting.property.getKey(), "" + newValue);
+    } catch (AccumuloException ex) {
+      if (ex.getCause() instanceof ThriftTableOperationException) {
+        ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
         if (ttoe.type == TableOperationExceptionType.NOTFOUND)
           return;
       }
       throw ex;
     }
   }
-  
+
   private void changeSetting(RandomData random, State state, Properties props) throws Exception {
     // pick a random property
     int choice = random.nextInt(0, settings.length - 1);
@@ -173,7 +230,7 @@ public class Config extends Test {
     long newValue = random.nextLong(setting.min, setting.max);
     state.getMap().put(LAST_SETTING, "" + choice);
     log.debug("Setting " + setting.property.getKey() + " to " + newValue);
-    state.getConnector().instanceOperations().setProperty(setting.property.getKey(), ""+newValue);
+    state.getConnector().instanceOperations().setProperty(setting.property.getKey(), "" + newValue);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/431f4e2c/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
new file mode 100644
index 0000000..c895d04
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
@@ -0,0 +1,53 @@
+/*
+ * 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.test.randomwalk.concurrent;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.test.randomwalk.State;
+import org.apache.accumulo.test.randomwalk.Test;
+
+public class OfflineTableNamespace extends Test {
+  
+  @Override
+  public void visit(State state, Properties props) throws Exception {
+    Connector conn = state.getConnector();
+    
+    Random rand = (Random) state.get("rand");
+    
+    @SuppressWarnings("unchecked")
+    List<String> namespaces = (List<String>) state.get("namespaces");
+    
+    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
+    
+    try {
+      conn.tableNamespaceOperations().offline(namespace);
+      log.debug("Offlined namespace " + namespace);
+      UtilWaitThread.sleep(rand.nextInt(200));
+      conn.tableNamespaceOperations().online(namespace);
+      log.debug("Onlined namespace " + namespace);
+    } catch (TableNamespaceNotFoundException tne) {
+      log.debug("offline or online failed " + namespace + ", doesnt exist");
+    }
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/431f4e2c/test/system/randomwalk/conf/modules/Concurrent.xml
----------------------------------------------------------------------
diff --git a/test/system/randomwalk/conf/modules/Concurrent.xml b/test/system/randomwalk/conf/modules/Concurrent.xml
index 89b1a88..cce8675 100644
--- a/test/system/randomwalk/conf/modules/Concurrent.xml
+++ b/test/system/randomwalk/conf/modules/Concurrent.xml
@@ -53,6 +53,7 @@
   <edge id="ct.DeleteTableNamespace" weight="100"/>
   <edge id="ct.RenameTableNamespace" weight="100"/>
   <edge id="ct.CloneTableNamespace" weight="100"/>
+  <edge id="ct.OfflineTableNamespace" weight="100"/>
   <edge id="ct.Apocalypse" weight="10"/>
   <edge id="END" weight="1"/>
 </node>
@@ -183,4 +184,8 @@
   <edge id="ct.StartAll" weight="1"/>
 </node>
 
+<node id="ct.OfflineTableNamespace">
+  <edge id="ct.StartAll" weight="1"/>
+</node>
+
 </module>