You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2012/10/13 03:07:57 UTC

svn commit: r1397757 - in /accumulo/branches/1.4: ./ src/ src/core/ src/server/ src/server/src/main/java/org/apache/accumulo/server/test/continuous/ src/server/src/main/java/org/apache/accumulo/server/test/scalability/ test/system/continuous/

Author: kturner
Date: Sat Oct 13 01:07:57 2012
New Revision: 1397757

URL: http://svn.apache.org/viewvc?rev=1397757&view=rev
Log:
ACCUMULO-810 added authorizations to continuous ingest test (merged from trunk)

Modified:
    accumulo/branches/1.4/   (props changed)
    accumulo/branches/1.4/src/   (props changed)
    accumulo/branches/1.4/src/core/   (props changed)
    accumulo/branches/1.4/src/server/   (props changed)
    accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousBatchWalker.java
    accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousIngest.java
    accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousMoru.java
    accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousScanner.java
    accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousVerify.java
    accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousWalk.java
    accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/scalability/Ingest.java
    accumulo/branches/1.4/test/system/continuous/continuous-env.sh.example
    accumulo/branches/1.4/test/system/continuous/run-verify.sh
    accumulo/branches/1.4/test/system/continuous/start-batchwalkers.sh
    accumulo/branches/1.4/test/system/continuous/start-ingest.sh
    accumulo/branches/1.4/test/system/continuous/start-scanners.sh
    accumulo/branches/1.4/test/system/continuous/start-walkers.sh

Propchange: accumulo/branches/1.4/
------------------------------------------------------------------------------
  Merged /accumulo/trunk:r1397746

Propchange: accumulo/branches/1.4/src/
------------------------------------------------------------------------------
  Merged /accumulo/trunk/src:r1397746
  Merged /accumulo/trunk:r1397746

Propchange: accumulo/branches/1.4/src/core/
------------------------------------------------------------------------------
  Merged /accumulo/trunk/src/core:r1397746
  Merged /accumulo/trunk/core:r1397746

Propchange: accumulo/branches/1.4/src/server/
------------------------------------------------------------------------------
  Merged /accumulo/trunk/src/server:r1397746
  Merged /accumulo/trunk/server:r1397746

Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousBatchWalker.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousBatchWalker.java?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousBatchWalker.java (original)
+++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousBatchWalker.java Sat Oct 13 01:07:57 2012
@@ -32,7 +32,9 @@ import org.apache.accumulo.core.client.Z
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.server.test.continuous.ContinuousWalk.RandomAuths;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.FileAppender;
 import org.apache.log4j.Level;
@@ -41,6 +43,7 @@ import org.apache.log4j.PatternLayout;
 
 public class ContinuousBatchWalker {
   private static String debugLog = null;
+  private static String authsFile = null;
   
   private static String[] processOptions(String[] args) {
     ArrayList<String> al = new ArrayList<String>();
@@ -48,6 +51,8 @@ public class ContinuousBatchWalker {
     for (int i = 0; i < args.length; i++) {
       if (args[i].equals("--debug")) {
         debugLog = args[++i];
+      } else if (args[i].equals("--auths")) {
+        authsFile = args[++i];
       } else {
         al.add(args[i]);
       }
@@ -62,7 +67,7 @@ public class ContinuousBatchWalker {
     
     if (args.length != 10) {
       throw new IllegalArgumentException("usage : " + ContinuousBatchWalker.class.getName()
-          + " [--debug <debug log>] <instance name> <zookeepers> <user> <pass> <table> <min> <max> <sleep time> <batch size> <query threads>");
+          + " [--debug <debug log>] [--auths <file>] <instance name> <zookeepers> <user> <pass> <table> <min> <max> <sleep time> <batch size> <query threads>");
     }
     
     if (debugLog != null) {
@@ -88,12 +93,14 @@ public class ContinuousBatchWalker {
     int batchSize = Integer.parseInt(args[8]);
     int numQueryThreads = Integer.parseInt(args[9]);
     
-    Connector conn = new ZooKeeperInstance(instanceName, zooKeepers).getConnector(user, password.getBytes());
-    Scanner scanner = conn.createScanner(table, Constants.NO_AUTHS);
-    BatchScanner bs = conn.createBatchScanner(table, Constants.NO_AUTHS, numQueryThreads);
-    
     Random r = new Random();
-    
+    RandomAuths randomAuths = new RandomAuths(authsFile);
+    Authorizations auths = randomAuths.getAuths(r);
+
+    Connector conn = new ZooKeeperInstance(instanceName, zooKeepers).getConnector(user, password.getBytes());
+    Scanner scanner = conn.createScanner(table, auths);
+    BatchScanner bs = conn.createBatchScanner(table, auths, numQueryThreads);
+
     while (true) {
       Set<Text> batch = getBatch(scanner, min, max, batchSize, r);
       

Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousIngest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousIngest.java?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousIngest.java (original)
+++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousIngest.java Sat Oct 13 01:07:57 2012
@@ -16,8 +16,12 @@
  */
 package org.apache.accumulo.server.test.continuous;
 
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
 import java.net.InetAddress;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 import java.util.Random;
 import java.util.UUID;
 import java.util.zip.CRC32;
@@ -36,8 +40,12 @@ import org.apache.accumulo.core.client.T
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.test.FastFormat;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.FileAppender;
 import org.apache.log4j.Level;
@@ -47,15 +55,44 @@ import org.apache.log4j.PatternLayout;
 
 public class ContinuousIngest {
   
+  private static String visFile = null;
   private static String debugLog = null;
   private static final byte[] EMPTY_BYTES = new byte[0];
   
+  private static List<ColumnVisibility> visibilities;
+  
+  private static void initVisibilities() throws Exception {
+    if (visFile == null) {
+      visibilities = Collections.singletonList(new ColumnVisibility());
+      return;
+    }
+    
+    visibilities = new ArrayList<ColumnVisibility>();
+    
+    FileSystem fs = FileSystem.get(new Configuration());
+    BufferedReader in = new BufferedReader(new InputStreamReader(fs.open(new Path(visFile))));
+    
+    String line;
+    
+    while ((line = in.readLine()) != null) {
+      visibilities.add(new ColumnVisibility(line));
+    }
+    
+    in.close();
+  }
+
+  private static ColumnVisibility getVisibility(Random rand) {
+    return visibilities.get(rand.nextInt(visibilities.size()));
+  }
+
   private static String[] processOptions(String[] args) {
     ArrayList<String> al = new ArrayList<String>();
     
     for (int i = 0; i < args.length; i++) {
       if (args[i].equals("--debug")) {
         debugLog = args[++i];
+      } else if (args[i].equals("--visibilities")) {
+        visFile = args[++i];
       } else {
         al.add(args[i]);
       }
@@ -72,7 +109,7 @@ public class ContinuousIngest {
       throw new IllegalArgumentException(
           "usage : "
               + ContinuousIngest.class.getName()
-              + " [--debug <debug log>] <instance name> <zookeepers> <user> <pass> <table> <num> <min> <max> <max colf> <max colq> <max mem> <max latency> <max threads> <enable checksum>");
+              + " [--debug <debug log>] [--visibilities <file>] <instance name> <zookeepers> <user> <pass> <table> <num> <min> <max> <max colf> <max colq> <max mem> <max latency> <max threads> <enable checksum>");
     }
     
     if (debugLog != null) {
@@ -82,6 +119,8 @@ public class ContinuousIngest {
       logger.addAppender(new FileAppender(new PatternLayout("%d{dd HH:mm:ss,SSS} [%-8c{2}] %-5p: %m%n"), debugLog, true));
     }
     
+    initVisibilities();
+
     String instanceName = args[0];
     String zooKeepers = args[1];
     
@@ -142,6 +181,8 @@ public class ContinuousIngest {
     
     out: while (true) {
       // generate first set of nodes
+      ColumnVisibility cv = getVisibility(r);
+
       for (int index = 0; index < flushInterval; index++) {
         long rowLong = genLong(min, max, r);
         prevRows[index] = rowLong;
@@ -153,7 +194,7 @@ public class ContinuousIngest {
         firstColFams[index] = cf;
         firstColQuals[index] = cq;
         
-        Mutation m = genMutation(rowLong, cf, cq, ingestInstanceId, count, null, r, checksum);
+        Mutation m = genMutation(rowLong, cf, cq, cv, ingestInstanceId, count, null, r, checksum);
         count++;
         bw.addMutation(m);
       }
@@ -168,7 +209,7 @@ public class ContinuousIngest {
           long rowLong = genLong(min, max, r);
           byte[] prevRow = genRow(prevRows[index]);
           prevRows[index] = rowLong;
-          Mutation m = genMutation(rowLong, r.nextInt(maxColF), r.nextInt(maxColQ), ingestInstanceId, count, prevRow, r, checksum);
+          Mutation m = genMutation(rowLong, r.nextInt(maxColF), r.nextInt(maxColQ), cv, ingestInstanceId, count, prevRow, r, checksum);
           count++;
           bw.addMutation(m);
         }
@@ -181,7 +222,8 @@ public class ContinuousIngest {
       // create one big linked list, this makes all of the first inserts
       // point to something
       for (int index = 0; index < flushInterval - 1; index++) {
-        Mutation m = genMutation(firstRows[index], firstColFams[index], firstColQuals[index], ingestInstanceId, count, genRow(prevRows[index + 1]), r, checksum);
+        Mutation m = genMutation(firstRows[index], firstColFams[index], firstColQuals[index], cv, ingestInstanceId, count, genRow(prevRows[index + 1]), r,
+            checksum);
         count++;
         bw.addMutation(m);
       }
@@ -192,7 +234,7 @@ public class ContinuousIngest {
     
     bw.close();
   }
-  
+
   private static long flush(BatchWriter bw, long count, final int flushInterval, long lastFlushTime) throws MutationsRejectedException {
     long t1 = System.currentTimeMillis();
     bw.flush();
@@ -202,7 +244,8 @@ public class ContinuousIngest {
     return lastFlushTime;
   }
   
-  public static Mutation genMutation(long rowLong, int cfInt, int cqInt, byte[] ingestInstanceId, long count, byte[] prevRow, Random r, boolean checksum) {
+  public static Mutation genMutation(long rowLong, int cfInt, int cqInt, ColumnVisibility cv, byte[] ingestInstanceId, long count, byte[] prevRow, Random r,
+      boolean checksum) {
     // Adler32 is supposed to be faster, but according to wikipedia is not good for small data.... so used CRC32 instead
     CRC32 cksum = null;
     
@@ -216,11 +259,12 @@ public class ContinuousIngest {
       cksum.update(rowString);
       cksum.update(cfString);
       cksum.update(cqString);
+      cksum.update(cv.getExpression());
     }
     
     Mutation m = new Mutation(new Text(rowString));
     
-    m.put(new Text(cfString), new Text(cqString), createValue(ingestInstanceId, count, prevRow, cksum));
+    m.put(new Text(cfString), new Text(cqString), cv, createValue(ingestInstanceId, count, prevRow, cksum));
     return m;
   }
   

Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousMoru.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousMoru.java?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousMoru.java (original)
+++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousMoru.java Sat Oct 13 01:07:57 2012
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
@@ -64,6 +65,8 @@ public class ContinuousMoru extends Conf
     private byte[] iiId;
     private long count;
     
+    private static final ColumnVisibility EMPTY_VIS = new ColumnVisibility();
+
     public void setup(Context context) throws IOException, InterruptedException {
       int max_cf = context.getConfiguration().getInt(MAX_CF, -1);
       int max_cq = context.getConfiguration().getInt(MAX_CQ, -1);
@@ -92,7 +95,8 @@ public class ContinuousMoru extends Conf
         int offset = ContinuousWalk.getPrevRowOffset(val);
         if (offset > 0) {
           long rowLong = Long.parseLong(new String(val, offset, 16), 16);
-          Mutation m = ContinuousIngest.genMutation(rowLong, random.nextInt(max_cf), random.nextInt(max_cq), iiId, count++, key.getRowData().toArray(), random,
+          Mutation m = ContinuousIngest.genMutation(rowLong, random.nextInt(max_cf), random.nextInt(max_cq), EMPTY_VIS, iiId, count++, key.getRowData()
+              .toArray(), random,
               true);
           context.write(null, m);
         }

Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousScanner.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousScanner.java?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousScanner.java (original)
+++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousScanner.java Sat Oct 13 01:07:57 2012
@@ -29,7 +29,9 @@ import org.apache.accumulo.core.client.Z
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.server.test.continuous.ContinuousWalk.RandomAuths;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.FileAppender;
 import org.apache.log4j.Level;
@@ -38,6 +40,7 @@ import org.apache.log4j.PatternLayout;
 
 public class ContinuousScanner {
   private static String debugLog = null;
+  private static String authsFile = null;
   
   private static String[] processOptions(String[] args) {
     ArrayList<String> al = new ArrayList<String>();
@@ -45,6 +48,8 @@ public class ContinuousScanner {
     for (int i = 0; i < args.length; i++) {
       if (args[i].equals("--debug")) {
         debugLog = args[++i];
+      } else if (args[i].equals("--auths")) {
+        authsFile = args[++i];
       } else {
         al.add(args[i]);
       }
@@ -59,7 +64,7 @@ public class ContinuousScanner {
     
     if (args.length != 9) {
       throw new IllegalArgumentException("usage : " + ContinuousScanner.class.getName()
-          + " [--debug <debug log>] <instance name> <zookeepers> <user> <pass> <table> <min> <max> <sleep time> <num to scan>");
+          + " [--debug <debug log>] [--auths <file>] <instance name> <zookeepers> <user> <pass> <table> <min> <max> <sleep time> <num to scan>");
     }
     
     if (debugLog != null) {
@@ -69,6 +74,8 @@ public class ContinuousScanner {
       logger.addAppender(new FileAppender(new PatternLayout("%d{dd HH:mm:ss,SSS} [%-8c{2}] %-5p: %m%n"), debugLog, true));
     }
     
+    Random r = new Random();
+
     String instanceName = args[0];
     String zooKeepers = args[1];
     
@@ -85,11 +92,12 @@ public class ContinuousScanner {
     
     int numToScan = Integer.parseInt(args[8]);
     
+    RandomAuths randomAuths = new RandomAuths(authsFile);
+
     Instance instance = new ZooKeeperInstance(instanceName, zooKeepers);
     Connector conn = instance.getConnector(user, password.getBytes());
-    Scanner scanner = conn.createScanner(table, Constants.NO_AUTHS);
-    
-    Random r = new Random();
+    Authorizations auths = randomAuths.getAuths(r);
+    Scanner scanner = conn.createScanner(table, auths);
     
     double delta = Math.min(.05, .05 / (numToScan / 1000.0));
     // System.out.println("Delta "+delta);

Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousVerify.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousVerify.java?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousVerify.java (original)
+++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousVerify.java Sat Oct 13 01:07:57 2012
@@ -23,6 +23,7 @@ import java.util.HashSet;
 import java.util.Random;
 import java.util.Set;
 
+import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
@@ -135,6 +136,20 @@ public class ContinuousVerify extends Co
   
   @Override
   public int run(String[] args) throws Exception {
+    
+    String auths = "";
+    ArrayList<String> argsList = new ArrayList<String>();
+    
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals("--auths")) {
+        auths = args[++i];
+      } else {
+        argsList.add(args[i]);
+      }
+    }
+    
+    args = argsList.toArray(new String[0]);
+
     if (args.length != 9) {
       throw new IllegalArgumentException("Usage : " + ContinuousVerify.class.getName()
           + " <instance name> <zookeepers> <user> <pass> <table> <output dir> <max mappers> <num reducers> <scan offline>");
@@ -165,9 +180,16 @@ public class ContinuousVerify extends Co
     }
 
     job.setInputFormatClass(AccumuloInputFormat.class);
-    AccumuloInputFormat.setInputInfo(job.getConfiguration(), user, pass.getBytes(), clone, new Authorizations());
+    Authorizations authorizations;
+    if (auths == null || auths.trim().equals(""))
+      authorizations = Constants.NO_AUTHS;
+    else
+      authorizations = new Authorizations(auths.split(","));
+
+    AccumuloInputFormat.setInputInfo(job.getConfiguration(), user, pass.getBytes(), clone, authorizations);
     AccumuloInputFormat.setZooKeeperInstance(job.getConfiguration(), instance, zookeepers);
     AccumuloInputFormat.setScanOffline(job.getConfiguration(), scanOffline);
+
     // set up ranges
     try {
       Set<Range> ranges = new ZooKeeperInstance(instance, zookeepers).getConnector(user, pass.getBytes()).tableOperations()

Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousWalk.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousWalk.java?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousWalk.java (original)
+++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousWalk.java Sat Oct 13 01:07:57 2012
@@ -16,8 +16,13 @@
  */
 package org.apache.accumulo.server.test.continuous;
 
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
 import java.net.InetAddress;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map.Entry;
 import java.util.Random;
 import java.util.zip.CRC32;
@@ -37,6 +42,9 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.Accumulo;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.FileAppender;
 import org.apache.log4j.Level;
@@ -47,6 +55,7 @@ import org.apache.log4j.PatternLayout;
 public class ContinuousWalk {
   
   private static String debugLog = null;
+  private static String authsFile = null;
   
   static class BadChecksumException extends RuntimeException {
     
@@ -64,6 +73,8 @@ public class ContinuousWalk {
     for (int i = 0; i < args.length; i++) {
       if (args[i].equals("--debug")) {
         debugLog = args[++i];
+      } else if (args[i].equals("--auths")) {
+        authsFile = args[++i];
       } else {
         al.add(args[i]);
       }
@@ -72,13 +83,41 @@ public class ContinuousWalk {
     return al.toArray(new String[al.size()]);
   }
   
+  static class RandomAuths {
+    private List<Authorizations> auths;
+    
+    RandomAuths(String file) throws IOException {
+      if (file == null) {
+        auths = Collections.singletonList(Constants.NO_AUTHS);
+        return;
+      }
+      
+      auths = new ArrayList<Authorizations>();
+      
+      FileSystem fs = FileSystem.get(new Configuration());
+      BufferedReader in = new BufferedReader(new InputStreamReader(fs.open(new Path(file))));
+      
+      String line;
+      
+      while ((line = in.readLine()) != null) {
+        auths.add(new Authorizations(line.split(",")));
+      }
+      
+      in.close();
+    }
+    
+    Authorizations getAuths(Random r) {
+      return auths.get(r.nextInt(auths.size()));
+    }
+  }
+
   public static void main(String[] args) throws Exception {
     
     args = processOptions(args);
     
     if (args.length != 8) {
       throw new IllegalArgumentException("usage : " + ContinuousWalk.class.getName()
-          + " [--debug <debug log>] <instance name> <zookeepers> <user> <pass> <table> <min> <max> <sleep time>");
+          + " [--debug <debug log>] [--auths <file>] <instance name> <zookeepers> <user> <pass> <table> <min> <max> <sleep time>");
     }
     
     if (debugLog != null) {
@@ -108,13 +147,14 @@ public class ContinuousWalk {
     Tracer.getInstance().addReceiver(new ZooSpanClient(zooKeepers, path, localhost, "cwalk", 1000));
     Accumulo.enableTracing(localhost, "ContinuousWalk");
     Connector conn = instance.getConnector(user, password.getBytes());
-    Scanner scanner = conn.createScanner(table, new Authorizations());
     
     Random r = new Random();
+    RandomAuths randomAuths = new RandomAuths(authsFile);
     
     ArrayList<Value> values = new ArrayList<Value>();
     
     while (true) {
+      Scanner scanner = conn.createScanner(table, randomAuths.getAuths(r));
       String row = findAStartRow(min, max, scanner, r);
       
       while (row != null) {
@@ -228,6 +268,7 @@ public class ContinuousWalk {
     cksum.update(key.getRowData().toArray());
     cksum.update(key.getColumnFamilyData().toArray());
     cksum.update(key.getColumnQualifierData().toArray());
+    cksum.update(key.getColumnVisibilityData().toArray());
     cksum.update(value.get(), 0, ckOff);
     
     if (cksum.getValue() != storedCksum) {

Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/scalability/Ingest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/scalability/Ingest.java?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/scalability/Ingest.java (original)
+++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/scalability/Ingest.java Sat Oct 13 01:07:57 2012
@@ -24,6 +24,7 @@ import org.apache.accumulo.core.client.C
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.server.test.continuous.ContinuousIngest;
 
 public class Ingest extends ScaleTest {
@@ -88,6 +89,8 @@ public class Ingest extends ScaleTest {
     long count = 0;
     long totalBytes = 0;
     
+    ColumnVisibility cv = new ColumnVisibility();
+
     // start timer
     startTimer();
     
@@ -95,7 +98,7 @@ public class Ingest extends ScaleTest {
     while (count < numIngestEntries) {
       count++;
       long rowId = ContinuousIngest.genLong(minRow, maxRow, r);
-      Mutation m = ContinuousIngest.genMutation(rowId, r.nextInt(maxColF), r.nextInt(maxColQ), ingestInstanceId.getBytes(), count, null, r, false);
+      Mutation m = ContinuousIngest.genMutation(rowId, r.nextInt(maxColF), r.nextInt(maxColQ), cv, ingestInstanceId.getBytes(), count, null, r, false);
       totalBytes += m.numBytes();
       try {
         bw.addMutation(m);

Modified: accumulo/branches/1.4/test/system/continuous/continuous-env.sh.example
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/test/system/continuous/continuous-env.sh.example?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/test/system/continuous/continuous-env.sh.example (original)
+++ accumulo/branches/1.4/test/system/continuous/continuous-env.sh.example Sat Oct 13 01:07:57 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/1.4/test/system/continuous/run-verify.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/test/system/continuous/run-verify.sh?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/test/system/continuous/run-verify.sh (original)
+++ accumulo/branches/1.4/test/system/continuous/run-verify.sh Sat Oct 13 01:07:57 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/1.4/test/system/continuous/start-batchwalkers.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/test/system/continuous/start-batchwalkers.sh?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/test/system/continuous/start-batchwalkers.sh (original)
+++ accumulo/branches/1.4/test/system/continuous/start-batchwalkers.sh Sat Oct 13 01:07:57 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/1.4/test/system/continuous/start-ingest.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/test/system/continuous/start-ingest.sh?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/test/system/continuous/start-ingest.sh (original)
+++ accumulo/branches/1.4/test/system/continuous/start-ingest.sh Sat Oct 13 01:07:57 2012
@@ -19,6 +19,7 @@
 . 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";
@@ -29,5 +30,9 @@ if [ -z $NUM ]; then
 	NUM=9223372036854775807
 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/1.4/test/system/continuous/start-scanners.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/test/system/continuous/start-scanners.sh?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/test/system/continuous/start-scanners.sh (original)
+++ accumulo/branches/1.4/test/system/continuous/start-scanners.sh Sat Oct 13 01:07:57 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/1.4/test/system/continuous/start-walkers.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/test/system/continuous/start-walkers.sh?rev=1397757&r1=1397756&r2=1397757&view=diff
==============================================================================
--- accumulo/branches/1.4/test/system/continuous/start-walkers.sh (original)
+++ accumulo/branches/1.4/test/system/continuous/start-walkers.sh Sat Oct 13 01:07:57 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