You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2012/10/16 00:17:26 UTC

svn commit: r1398539 [4/4] - in /accumulo/branches/ACCUMULO-259: ./ assemble/ assemble/platform/ assemble/platform/debian/ assemble/platform/debian/init.d/ conf/examples/1GB/native-standalone/ conf/examples/1GB/standalone/ conf/examples/2GB/native-stan...

Modified: accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java?rev=1398539&r1=1398538&r2=1398539&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java Mon Oct 15 22:17:22 2012
@@ -138,9 +138,7 @@ public class DefaultLoadBalancerTest {
     
     // reassign offline extents
     assignTablets(remove.extents, servers, current, balancer);
-    
-    expectedCounts.put("t3", 3);
-    checkBalance(metadataTable, servers, expectedCounts);
+    checkBalance(metadataTable, servers, null);
   }
   
   private void assignTablets(List<KeyExtent> metadataTable, Map<TServerInstance,FakeTServer> servers, SortedMap<TServerInstance,TabletServerStatus> status,
@@ -259,16 +257,18 @@ public class DefaultLoadBalancerTest {
         fail("average number of tablets is " + average + " but a server has " + server.extents.size());
     }
     
-    for (FakeTServer server : servers.values()) {
-      Map<String,Integer> counts = new HashMap<String,Integer>();
-      for (KeyExtent extent : server.extents) {
-        String t = extent.getTableId().toString();
-        if (counts.get(t) == null)
-          counts.put(t, 0);
-        counts.put(t, counts.get(t) + 1);
-      }
-      for (Entry<String,Integer> entry : counts.entrySet()) {
-        assertEquals(expectedCounts.get(entry.getKey()), counts.get(entry.getKey()));
+    if (expectedCounts != null) {
+      for (FakeTServer server : servers.values()) {
+        Map<String,Integer> counts = new HashMap<String,Integer>();
+        for (KeyExtent extent : server.extents) {
+          String t = extent.getTableId().toString();
+          if (counts.get(t) == null)
+            counts.put(t, 0);
+          counts.put(t, counts.get(t) + 1);
+        }
+        for (Entry<String,Integer> entry : counts.entrySet()) {
+          assertEquals(expectedCounts.get(entry.getKey()), counts.get(entry.getKey()));
+        }
       }
     }
   }

Modified: accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/tabletserver/log/SortedLogRecoveryTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/tabletserver/log/SortedLogRecoveryTest.java?rev=1398539&r1=1398538&r2=1398539&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/tabletserver/log/SortedLogRecoveryTest.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/test/java/org/apache/accumulo/server/tabletserver/log/SortedLogRecoveryTest.java Mon Oct 15 22:17:22 2012
@@ -37,6 +37,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.server.data.ServerMutation;
 import org.apache.accumulo.server.logger.LogEvents;
 import org.apache.accumulo.server.logger.LogFileKey;
 import org.apache.accumulo.server.logger.LogFileValue;
@@ -91,10 +92,10 @@ public class SortedLogRecoveryTest {
         result.key.tablet = (KeyExtent) fileExtentMutation;
         break;
       case MUTATION:
-        result.value.mutations = new Mutation[] {(Mutation) fileExtentMutation};
+        result.value.mutations = Arrays.asList((Mutation) fileExtentMutation);
         break;
       case MANY_MUTATIONS:
-        result.value.mutations = (Mutation[]) fileExtentMutation;
+        result.value.mutations = Arrays.asList((Mutation[])fileExtentMutation);
     }
     return result;
   }
@@ -142,12 +143,11 @@ public class SortedLogRecoveryTest {
   
   @Test
   public void testCompactionCrossesLogs() throws IOException {
-    // Create a test log
-    Mutation ignored = new Mutation(new Text("ignored"));
+    Mutation ignored = new ServerMutation(new Text("ignored"));
     ignored.put(cf, cq, value);
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
-    Mutation m2 = new Mutation(new Text("row2"));
+    Mutation m2 = new ServerMutation(new Text("row2"));
     m2.put(cf, cq, value);
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, 1, "2"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
         createKeyValue(COMPACTION_START, 3, 1, "somefile"), createKeyValue(MUTATION, 2, 1, ignored), createKeyValue(MUTATION, 4, 1, ignored),};
@@ -180,15 +180,15 @@ public class SortedLogRecoveryTest {
   @Test
   public void testCompactionCrossesLogs5() throws IOException {
     // Create a test log
-    Mutation ignored = new Mutation(new Text("ignored"));
+    Mutation ignored = new ServerMutation(new Text("ignored"));
     ignored.put(cf, cq, value);
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
-    Mutation m2 = new Mutation(new Text("row2"));
+    Mutation m2 = new ServerMutation(new Text("row2"));
     m2.put(cf, cq, value);
-    Mutation m3 = new Mutation(new Text("row3"));
+    Mutation m3 = new ServerMutation(new Text("row3"));
     m3.put(cf, cq, value);
-    Mutation m4 = new Mutation(new Text("row4"));
+    Mutation m4 = new ServerMutation(new Text("row4"));
     m4.put(cf, cq, value);
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
         createKeyValue(COMPACTION_START, 3, 1, "somefile"), createKeyValue(MUTATION, 2, 1, ignored), createKeyValue(MUTATION, 4, 1, ignored),};
@@ -218,17 +218,17 @@ public class SortedLogRecoveryTest {
   @Test
   public void testCompactionCrossesLogs6() throws IOException {
     // Create a test log
-    Mutation ignored = new Mutation(new Text("ignored"));
+    Mutation ignored = new ServerMutation(new Text("ignored"));
     ignored.put(cf, cq, value);
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
-    Mutation m2 = new Mutation(new Text("row2"));
+    Mutation m2 = new ServerMutation(new Text("row2"));
     m2.put(cf, cq, value);
-    Mutation m3 = new Mutation(new Text("row3"));
+    Mutation m3 = new ServerMutation(new Text("row3"));
     m3.put(cf, cq, value);
-    Mutation m4 = new Mutation(new Text("row4"));
+    Mutation m4 = new ServerMutation(new Text("row4"));
     m4.put(cf, cq, value);
-    Mutation m5 = new Mutation(new Text("row5"));
+    Mutation m5 = new ServerMutation(new Text("row5"));
     m5.put(cf, cq, value);
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, 1, "2"), createKeyValue(DEFINE_TABLET, 1, 1, extent), createKeyValue(MUTATION, 1, 1, ignored),
         createKeyValue(MUTATION, 3, 1, m),};
@@ -277,7 +277,7 @@ public class SortedLogRecoveryTest {
   @Test
   public void testSimple() throws IOException {
     // Create a test log
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent), createKeyValue(MUTATION, 2, 1, m),};
     Map<String,KeyValue[]> logs = new TreeMap<String,KeyValue[]>();
@@ -292,9 +292,9 @@ public class SortedLogRecoveryTest {
   @Test
   public void testSkipSuccessfulCompaction() throws IOException {
     // Create a test log
-    Mutation ignored = new Mutation(new Text("ignored"));
+    Mutation ignored = new ServerMutation(new Text("ignored"));
     ignored.put(cf, cq, value);
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
         createKeyValue(COMPACTION_START, 3, 1, "somefile"), createKeyValue(COMPACTION_FINISH, 4, 1, null), createKeyValue(MUTATION, 2, 1, ignored),
@@ -311,9 +311,9 @@ public class SortedLogRecoveryTest {
   @Test
   public void testSkipSuccessfulCompactionAcrossFiles() throws IOException {
     // Create a test log
-    Mutation ignored = new Mutation(new Text("ignored"));
+    Mutation ignored = new ServerMutation(new Text("ignored"));
     ignored.put(cf, cq, value);
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
         createKeyValue(COMPACTION_START, 3, 1, "somefile"), createKeyValue(MUTATION, 2, 1, ignored),};
@@ -332,11 +332,11 @@ public class SortedLogRecoveryTest {
   @Test
   public void testGetMutationsAfterCompactionStart() throws IOException {
     // Create a test log
-    Mutation ignored = new Mutation(new Text("ignored"));
+    Mutation ignored = new ServerMutation(new Text("ignored"));
     ignored.put(cf, cq, value);
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
-    Mutation m2 = new Mutation(new Text("row2"));
+    Mutation m2 = new ServerMutation(new Text("row2"));
     m2.put(cf, cq, new Value("123".getBytes()));
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
         createKeyValue(COMPACTION_START, 3, 1, "somefile"), createKeyValue(MUTATION, 2, 1, ignored), createKeyValue(MUTATION, 4, 1, m),};
@@ -356,11 +356,11 @@ public class SortedLogRecoveryTest {
   @Test
   public void testDoubleFinish() throws IOException {
     // Create a test log
-    Mutation ignored = new Mutation(new Text("ignored"));
+    Mutation ignored = new ServerMutation(new Text("ignored"));
     ignored.put(cf, cq, value);
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
-    Mutation m2 = new Mutation(new Text("row2"));
+    Mutation m2 = new ServerMutation(new Text("row2"));
     m2.put(cf, cq, new Value("123".getBytes()));
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
         createKeyValue(COMPACTION_FINISH, 2, 1, null), createKeyValue(COMPACTION_START, 4, 1, "somefile"), createKeyValue(COMPACTION_FINISH, 6, 1, null),
@@ -378,13 +378,13 @@ public class SortedLogRecoveryTest {
   @Test
   public void testCompactionCrossesLogs2() throws IOException {
     // Create a test log
-    Mutation ignored = new Mutation(new Text("ignored"));
+    Mutation ignored = new ServerMutation(new Text("ignored"));
     ignored.put(cf, cq, value);
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
-    Mutation m2 = new Mutation(new Text("row2"));
+    Mutation m2 = new ServerMutation(new Text("row2"));
     m2.put(cf, cq, value);
-    Mutation m3 = new Mutation(new Text("row3"));
+    Mutation m3 = new ServerMutation(new Text("row3"));
     m3.put(cf, cq, value);
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
         createKeyValue(COMPACTION_START, 3, 1, "somefile"), createKeyValue(MUTATION, 2, 1, ignored), createKeyValue(MUTATION, 4, 1, m),};
@@ -407,9 +407,9 @@ public class SortedLogRecoveryTest {
   @Test
   public void testBug1() throws IOException {
     // this unit test reproduces a bug that occurred, nothing should recover
-    Mutation m1 = new Mutation(new Text("row1"));
+    Mutation m1 = new ServerMutation(new Text("row1"));
     m1.put(cf, cq, value);
-    Mutation m2 = new Mutation(new Text("row2"));
+    Mutation m2 = new ServerMutation(new Text("row2"));
     m2.put(cf, cq, value);
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
         createKeyValue(COMPACTION_START, 30, 1, "somefile"), createKeyValue(COMPACTION_FINISH, 32, 1, "somefile"), createKeyValue(MUTATION, 29, 1, m1),
@@ -425,13 +425,13 @@ public class SortedLogRecoveryTest {
   @Test
   public void testBug2() throws IOException {
     // Create a test log
-    Mutation ignored = new Mutation(new Text("ignored"));
+    Mutation ignored = new ServerMutation(new Text("ignored"));
     ignored.put(cf, cq, value);
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
-    Mutation m2 = new Mutation(new Text("row2"));
+    Mutation m2 = new ServerMutation(new Text("row2"));
     m2.put(cf, cq, value);
-    Mutation m3 = new Mutation(new Text("row3"));
+    Mutation m3 = new ServerMutation(new Text("row3"));
     m3.put(cf, cq, value);
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
         createKeyValue(COMPACTION_START, 2, 1, "somefile"), createKeyValue(COMPACTION_FINISH, 4, 1, null), createKeyValue(MUTATION, 3, 1, m),};
@@ -452,19 +452,19 @@ public class SortedLogRecoveryTest {
   @Test
   public void testCompactionCrossesLogs4() throws IOException {
     // Create a test log
-    Mutation ignored = new Mutation(new Text("ignored"));
+    Mutation ignored = new ServerMutation(new Text("ignored"));
     ignored.put(cf, cq, value);
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
-    Mutation m2 = new Mutation(new Text("row2"));
+    Mutation m2 = new ServerMutation(new Text("row2"));
     m2.put(cf, cq, value);
-    Mutation m3 = new Mutation(new Text("row3"));
+    Mutation m3 = new ServerMutation(new Text("row3"));
     m3.put(cf, cq, value);
-    Mutation m4 = new Mutation(new Text("row4"));
+    Mutation m4 = new ServerMutation(new Text("row4"));
     m4.put(cf, cq, value);
-    Mutation m5 = new Mutation(new Text("row5"));
+    Mutation m5 = new ServerMutation(new Text("row5"));
     m5.put(cf, cq, value);
-    Mutation m6 = new Mutation(new Text("row6"));
+    Mutation m6 = new ServerMutation(new Text("row6"));
     m6.put(cf, cq, value);
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
         createKeyValue(COMPACTION_START, 4, 1, "somefile"),
@@ -501,17 +501,17 @@ public class SortedLogRecoveryTest {
   
   @Test
   public void testLookingForBug3() throws IOException {
-    Mutation ignored = new Mutation(new Text("ignored"));
+    Mutation ignored = new ServerMutation(new Text("ignored"));
     ignored.put(cf, cq, value);
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put(cf, cq, value);
-    Mutation m2 = new Mutation(new Text("row2"));
+    Mutation m2 = new ServerMutation(new Text("row2"));
     m2.put(cf, cq, value);
-    Mutation m3 = new Mutation(new Text("row3"));
+    Mutation m3 = new ServerMutation(new Text("row3"));
     m3.put(cf, cq, value);
-    Mutation m4 = new Mutation(new Text("row4"));
+    Mutation m4 = new ServerMutation(new Text("row4"));
     m4.put(cf, cq, value);
-    Mutation m5 = new Mutation(new Text("row5"));
+    Mutation m5 = new ServerMutation(new Text("row5"));
     m5.put(cf, cq, value);
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
         createKeyValue(COMPACTION_START, 2, 1, "somefile"), createKeyValue(COMPACTION_FINISH, 3, 1, null), createKeyValue(MUTATION, 1, 1, ignored),
@@ -538,9 +538,9 @@ public class SortedLogRecoveryTest {
     // there was a bug where the oldest tablet id was used instead
     // of the newest
     
-    Mutation ignored = new Mutation("row1");
+    Mutation ignored = new ServerMutation(new Text("row1"));
     ignored.put("foo", "bar", "v1");
-    Mutation m = new Mutation(new Text("row1"));
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put("foo", "bar", "v1");
     
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 1, extent),
@@ -562,7 +562,7 @@ public class SortedLogRecoveryTest {
   public void testNoFinish0() throws Exception {
     // its possible that a minor compaction finishes successfully, but the process dies before writing the compaction event
     
-    Mutation ignored = new Mutation("row1");
+    Mutation ignored = new ServerMutation(new Text("row1"));
     ignored.put("foo", "bar", "v1");
     
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 2, extent),
@@ -581,9 +581,9 @@ public class SortedLogRecoveryTest {
   public void testNoFinish1() throws Exception {
     // its possible that a minor compaction finishes successfully, but the process dies before writing the compaction event
     
-    Mutation ignored = new Mutation("row1");
+    Mutation ignored = new ServerMutation(new Text("row1"));
     ignored.put("foo", "bar", "v1");
-    Mutation m = new Mutation("row1");
+    Mutation m = new ServerMutation(new Text("row1"));
     m.put("foo", "bar", "v2");
     
     KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"), createKeyValue(DEFINE_TABLET, 1, 2, extent),

Propchange: accumulo/branches/ACCUMULO-259/src/
------------------------------------------------------------------------------
  Merged /accumulo/trunk/src:r1391755-1398536
  Merged /accumulo/branches/1.4/src:r1393868,1396065,1396572,1396616,1396758,1396772,1397048,1397113,1397117,1397176,1397189,1397383,1397700,1397921,1398286,1398308,1398393,1398399,1398438
  Merged /accumulo/branches/1.4/src/src:r1396772,1397048,1397113,1397117,1397176,1397189,1397383,1397700,1397921,1398286,1398308,1398359,1398393,1398399,1398438

Modified: accumulo/branches/ACCUMULO-259/test/system/continuous/continuous-env.sh.example
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/continuous/continuous-env.sh.example?rev=1398539&r1=1398538&r2=1398539&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/continuous/continuous-env.sh.example (original)
+++ accumulo/branches/ACCUMULO-259/test/system/continuous/continuous-env.sh.example Mon Oct 15 22:17:22 2012
@@ -47,6 +47,11 @@ MAX_CF=32767
 #the maximum number of random column qualifiers to generate
 MAX_CQ=32767
 
+#an optional file in hdfs containing visibilites.  If left blank, then column
+#visibility will not be set.  If specified then a random line will be selected
+#from the file and used for column visibility for each linked list.
+VISIBILITIES=''
+
 #the max memory (in bytes) each ingester will use to buffer writes
 MAX_MEM=100000000
 
@@ -59,6 +64,11 @@ NUM_THREADS=4
 #the amount of time (in millis) to sleep between each query
 SLEEP_TIME=10
 
+#an optional file in hdfs containing line of comma seperated auths.  If
+#specified, walkers will randomly select lines from this file and use that to
+#set auths.
+AUTHS=''
+
 #time amount of time (in minutes) the agitator should sleep before killing
 KILL_SLEEP_TIME=20
 
@@ -81,6 +91,8 @@ VERFIY_OUT=/tmp/continuous_verify
 VERIFY_MAX_MAPS=64
 VERIFY_REDUCERS=64
 SCAN_OFFLINE=false
+#comma separated list of auths to use for verify
+VERIFY_AUTHS=''
 
 #settings related to the batch walker
 BATCH_WALKER_SLEEP=180000

Modified: accumulo/branches/ACCUMULO-259/test/system/continuous/run-verify.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/continuous/run-verify.sh?rev=1398539&r1=1398538&r2=1398539&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/continuous/run-verify.sh (original)
+++ accumulo/branches/ACCUMULO-259/test/system/continuous/run-verify.sh Mon Oct 15 22:17:22 2012
@@ -18,5 +18,11 @@
 
 . mapred-setup.sh
 
-$ACCUMULO_HOME/bin/tool.sh "$SERVER_LIBJAR" org.apache.accumulo.server.test.continuous.ContinuousVerify -libjars "$SERVER_LIBJAR" $INSTANCE_NAME $ZOO_KEEPERS $USER $PASS $TABLE $VERFIY_OUT $VERIFY_MAX_MAPS $VERIFY_REDUCERS $SCAN_OFFLINE
+AUTH_OPT="";
+
+if [ -n "$VERIFY_AUTHS" ] ; then
+	AUTH_OPT="--auths $VERIFY_AUTHS";
+fi
+
+$ACCUMULO_HOME/bin/tool.sh "$SERVER_LIBJAR" org.apache.accumulo.server.test.continuous.ContinuousVerify -libjars "$SERVER_LIBJAR" $AUTH_OPT $INSTANCE_NAME $ZOO_KEEPERS $USER $PASS $TABLE $VERFIY_OUT $VERIFY_MAX_MAPS $VERIFY_REDUCERS $SCAN_OFFLINE
 

Modified: accumulo/branches/ACCUMULO-259/test/system/continuous/start-batchwalkers.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/continuous/start-batchwalkers.sh?rev=1398539&r1=1398538&r2=1398539&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/continuous/start-batchwalkers.sh (original)
+++ accumulo/branches/ACCUMULO-259/test/system/continuous/start-batchwalkers.sh Mon Oct 15 22:17:22 2012
@@ -19,10 +19,16 @@
 . continuous-env.sh
 
 DEBUG_OPT="";
+AUTH_OPT="";
 
 if [ "$DEBUG_BATCH_WALKER" = "on" ] ; then
 	DEBUG_OPT="--debug $CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_batch_walk.log";
 fi
 
-pssh -h batch_walkers.txt "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.test.continuous.ContinuousBatchWalker $DEBUG_OPT $INSTANCE_NAME $ZOO_KEEPERS $USER $PASS $TABLE $MIN $MAX $BATCH_WALKER_SLEEP $BATCH_WALKER_BATCH_SIZE $BATCH_WALKER_THREADS >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_batch_walk.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_batch_walk.err &" < /dev/null
+if [ -n "$AUTHS" ] ; then
+	AUTH_OPT="--auths \"$AUTHS\"";
+fi
+
+
+pssh -h batch_walkers.txt "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.test.continuous.ContinuousBatchWalker $DEBUG_OPT $AUTH_OPT $INSTANCE_NAME $ZOO_KEEPERS $USER $PASS $TABLE $MIN $MAX $BATCH_WALKER_SLEEP $BATCH_WALKER_BATCH_SIZE $BATCH_WALKER_THREADS >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_batch_walk.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_batch_walk.err &" < /dev/null
 

Modified: accumulo/branches/ACCUMULO-259/test/system/continuous/start-ingest.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/continuous/start-ingest.sh?rev=1398539&r1=1398538&r2=1398539&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/continuous/start-ingest.sh (original)
+++ accumulo/branches/ACCUMULO-259/test/system/continuous/start-ingest.sh Mon Oct 15 22:17:22 2012
@@ -19,10 +19,15 @@
 . continuous-env.sh
 
 DEBUG_OPT="";
+VIS_OPT="";
 
 if [ "$DEBUG_INGEST" = "on" ] ; then
 	DEBUG_OPT="--debug $CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_ingest.log";
 fi
 
-pssh -h ingesters.txt "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.test.continuous.ContinuousIngest $DEBUG_OPT $INSTANCE_NAME $ZOO_KEEPERS $USER $PASS $TABLE $NUM $MIN $MAX $MAX_CF $MAX_CQ $MAX_MEM $MAX_LATENCY $NUM_THREADS $CHECKSUM >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_ingest.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_ingest.err &" < /dev/null
+if [ -n "$VISIBILITIES" ] ; then
+	VIS_OPT="--visibilities \"$VISIBILITIES\"";
+fi
+
+pssh -h ingesters.txt "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.test.continuous.ContinuousIngest $DEBUG_OPT $VIS_OPT $INSTANCE_NAME $ZOO_KEEPERS $USER $PASS $TABLE $NUM $MIN $MAX $MAX_CF $MAX_CQ $MAX_MEM $MAX_LATENCY $NUM_THREADS $CHECKSUM >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_ingest.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_ingest.err &" < /dev/null
 

Modified: accumulo/branches/ACCUMULO-259/test/system/continuous/start-scanners.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/continuous/start-scanners.sh?rev=1398539&r1=1398538&r2=1398539&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/continuous/start-scanners.sh (original)
+++ accumulo/branches/ACCUMULO-259/test/system/continuous/start-scanners.sh Mon Oct 15 22:17:22 2012
@@ -19,10 +19,16 @@
 . continuous-env.sh
 
 DEBUG_OPT="";
+AUTH_OPT="";
 
 if [ "$DEBUG_SCANNER" = "on" ] ; then
 	DEBUG_OPT="--debug $CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_scanner.log";
 fi
 
-pssh -h scanners.txt "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.test.continuous.ContinuousScanner $DEBUG_SCANNERS $INSTANCE_NAME $ZOO_KEEPERS $USER $PASS $TABLE $MIN $MAX $SCANNER_SLEEP_TIME $SCANNER_ENTRIES >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_scanner.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_scanner.err &" < /dev/null
+if [ -n "$AUTHS" ] ; then
+	AUTH_OPT="--auths \"$AUTHS\"";
+fi
+
+
+pssh -h scanners.txt "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.test.continuous.ContinuousScanner $DEBUG_OPT $AUTH_OPT $INSTANCE_NAME $ZOO_KEEPERS $USER $PASS $TABLE $MIN $MAX $SCANNER_SLEEP_TIME $SCANNER_ENTRIES >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_scanner.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_scanner.err &" < /dev/null
 

Modified: accumulo/branches/ACCUMULO-259/test/system/continuous/start-walkers.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/continuous/start-walkers.sh?rev=1398539&r1=1398538&r2=1398539&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/continuous/start-walkers.sh (original)
+++ accumulo/branches/ACCUMULO-259/test/system/continuous/start-walkers.sh Mon Oct 15 22:17:22 2012
@@ -19,10 +19,16 @@
 . continuous-env.sh
 
 DEBUG_OPT="";
+AUTH_OPT="";
 
 if [ "$DEBUG_WALKER" = "on" ] ; then
 	DEBUG_OPT="--debug $CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_walk.log";
 fi
 
-pssh -h walkers.txt "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.test.continuous.ContinuousWalk $DEBUG_OPT $INSTANCE_NAME $ZOO_KEEPERS $USER $PASS $TABLE $MIN $MAX $SLEEP_TIME >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_walk.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_walk.err &" < /dev/null
+if [ -n "$AUTHS" ] ; then
+	AUTH_OPT="--auths \"$AUTHS\"";
+fi
+
+
+pssh -h walkers.txt "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.server.test.continuous.ContinuousWalk $DEBUG_OPT $AUTH_OPT $INSTANCE_NAME $ZOO_KEEPERS $USER $PASS $TABLE $MIN $MAX $SLEEP_TIME >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_walk.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_walk.err &" < /dev/null
 

Propchange: accumulo/branches/ACCUMULO-259/test/system/randomwalk/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Mon Oct 15 22:17:22 2012
@@ -0,0 +1,2 @@
+*-tmp
+logs

Modified: accumulo/branches/ACCUMULO-259/test/system/randomwalk/bin/start-local.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/test/system/randomwalk/bin/start-local.sh?rev=1398539&r1=1398538&r2=1398539&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/test/system/randomwalk/bin/start-local.sh (original)
+++ accumulo/branches/ACCUMULO-259/test/system/randomwalk/bin/start-local.sh Mon Oct 15 22:17:22 2012
@@ -57,9 +57,10 @@ cd "$RW_HOME"
 "$HADOOP_HOME/bin/hadoop" fs -get /randomwalk/config.tgz config.tgz
 
 # extract config to a tmp directory
-rm -rf tmp/
-mkdir tmp/
-tar xzf config.tgz -C tmp/
+TEMP="`hostname`-tmp"
+rm -rf "$TEMP"
+mkdir "$TEMP"
+tar xzf config.tgz -C "$TEMP"
 rm config.tgz
 
 # config the logging
@@ -71,4 +72,4 @@ fi
 LOG_ID=`hostname -s`_`date +%Y%m%d_%H%M%S`
 
 # run the local walker
-"$ACCUMULO_HOME/bin/accumulo" org.apache.accumulo.server.test.randomwalk.Framework "$RW_HOME/tmp/conf/" "$RW_LOGS" "$LOG_ID" "$1" >"$RW_LOGS/$LOG_ID.out" 2>"$RW_LOGS/$LOG_ID.err" &
+"$ACCUMULO_HOME/bin/accumulo" org.apache.accumulo.server.test.randomwalk.Framework "$RW_HOME/$TEMP/conf/" "$RW_LOGS" "$LOG_ID" "$1" >"$RW_LOGS/$LOG_ID.out" 2>"$RW_LOGS/$LOG_ID.err" &