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/01/12 20:16:29 UTC

svn commit: r1230709 - in /incubator/accumulo/trunk/src: core/src/main/java/org/apache/accumulo/core/client/mapreduce/ core/src/test/java/org/apache/accumulo/core/client/mapreduce/ examples/simple/src/main/java/org/apache/accumulo/examples/simple/hello...

Author: vines
Date: Thu Jan 12 19:16:28 2012
New Revision: 1230709

URL: http://svn.apache.org/viewvc?rev=1230709&view=rev
Log:
Fixes ACCUMULO-310 - clean up deprecations

Modified:
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
    incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
    incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
    incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/helloworld/InsertWithOutputFormat.java
    incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/RowHash.java
    incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TeraSortIngest.java
    incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/WordCount.java
    incubator/accumulo/trunk/src/examples/wikisearch/ingest/src/main/java/org/apache/accumulo/examples/wikisearch/ingest/WikipediaIngester.java
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousMoru.java
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/CopyTool.java
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/sequential/MapRedVerifyTool.java

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java Thu Jan 12 19:16:28 2012
@@ -27,7 +27,6 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.file.rfile.RFile;
-import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -62,7 +61,7 @@ public class AccumuloFileOutputFormat ex
     if (extension == null || extension.isEmpty())
       extension = RFile.EXTENSION;
     
-    handleBlockSize(job);
+    handleBlockSize(job.getConfiguration());
     final Path file = this.getDefaultWorkFile(job, "." + extension);
     
     final FileSKVWriter out = FileOperations.getInstance().openWriter(file.toString(), file.getFileSystem(conf), conf,
@@ -87,17 +86,11 @@ public class AccumuloFileOutputFormat ex
     };
   }
   
-  /**
-   * @deprecated Use {@link #handleBlockSize(Configuration)} instead
-   */
-  protected static void handleBlockSize(JobContext job) {
-    handleBlockSize(job.getConfiguration());
-  }
-
   protected static void handleBlockSize(Configuration conf) {
     int blockSize;
     if (conf.getBoolean(INSTANCE_HAS_BEEN_SET, false)) {
-      blockSize = (int) getInstance(conf).getConfiguration().getMemoryInBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE);
+      blockSize = (int) new ZooKeeperInstance(conf.get(INSTANCE_NAME), conf.get(ZOOKEEPERS)).getConfiguration().getMemoryInBytes(
+          Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE);
     } else {
       blockSize = getBlockSize(conf);
     }
@@ -105,24 +98,10 @@ public class AccumuloFileOutputFormat ex
     
   }
   
-  /**
-   * @deprecated Use {@link #setFileType(Configuration,String)} instead
-   */
-  public static void setFileType(JobContext job, String type) {
-    setFileType(job.getConfiguration(), type);
-  }
-
   public static void setFileType(Configuration conf, String type) {
     conf.set(FILE_TYPE, type);
   }
   
-  /**
-   * @deprecated Use {@link #setBlockSize(Configuration,int)} instead
-   */
-  public static void setBlockSize(JobContext job, int blockSize) {
-    setBlockSize(job.getConfiguration(), blockSize);
-  }
-
   public static void setBlockSize(Configuration conf, int blockSize) {
     conf.setInt(BLOCK_SIZE, blockSize);
   }
@@ -135,34 +114,6 @@ public class AccumuloFileOutputFormat ex
   /**
    * 
    * @param job
-   * @param instanceName
-   * @param zooKeepers
-   * @deprecated since 1.4, use {@link #setBlockSize(Configuration, int)} to set block size no other configurations are used by OutputFormat
-   */
-  public static void setZooKeeperInstance(JobContext job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job.getConfiguration(), instanceName, zooKeepers);
-  }
-
-  /**
-   * 
-   * @param conf
-   * @param instanceName
-   * @param zooKeepers
-   * @deprecated since 1.4, use {@link #setBlockSize(Configuration, int)} to set block size no other configurations are used by OutputFormat
-   */
-  public static void setZooKeeperInstance(Configuration conf, String instanceName, String zooKeepers) {
-    if (conf.getBoolean(INSTANCE_HAS_BEEN_SET, false))
-      throw new IllegalStateException("Instance info can only be set once per job");
-    conf.setBoolean(INSTANCE_HAS_BEEN_SET, true);
-    
-    ArgumentChecker.notNull(instanceName, zooKeepers);
-    conf.set(INSTANCE_NAME, instanceName);
-    conf.set(ZOOKEEPERS, zooKeepers);
-  }
-  
-  /**
-   * 
-   * @param job
    * @return
    * 
    * @deprecated since 1.4, use {@link #setBlockSize(Configuration, int)} no other configurations are used by OutputFormat

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java Thu Jan 12 19:16:28 2012
@@ -91,25 +91,6 @@ public class AccumuloOutputFormat extend
   /**
    * Configure the output format.
    * 
-   * @param job
-   *          the Map/Reduce job object
-   * @param user
-   *          the username, which must have the Table.CREATE permission to create tables
-   * @param passwd
-   *          the passwd for the username
-   * @param createTables
-   *          the output format will create new tables as necessary. Table names can only be alpha-numeric and underscores.
-   * @param defaultTable
-   *          the table to use when the tablename is null in the write call
-   * @deprecated Use {@link #setOutputInfo(Configuration,String,byte[],boolean,String)} instead
-   */
-  public static void setOutputInfo(JobContext job, String user, byte[] passwd, boolean createTables, String defaultTable) {
-    setOutputInfo(job.getConfiguration(), user, passwd, createTables, defaultTable);
-  }
-
-  /**
-   * Configure the output format.
-   * 
    * @param conf
    *          the Map/Reduce job object
    * @param user
@@ -134,13 +115,6 @@ public class AccumuloOutputFormat extend
       conf.set(DEFAULT_TABLE_NAME, defaultTable);
   }
   
-  /**
-   * @deprecated Use {@link #setZooKeeperInstance(Configuration,String,String)} instead
-   */
-  public static void setZooKeeperInstance(JobContext job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job.getConfiguration(), instanceName, zooKeepers);
-  }
-
   public static void setZooKeeperInstance(Configuration conf, String instanceName, String zooKeepers) {
     if (conf.getBoolean(INSTANCE_HAS_BEEN_SET, false))
       throw new IllegalStateException("Instance info can only be set once per job");
@@ -151,82 +125,33 @@ public class AccumuloOutputFormat extend
     conf.set(ZOOKEEPERS, zooKeepers);
   }
   
-  /**
-   * @deprecated Use {@link #setMockInstance(Configuration,String)} instead
-   */
-  public static void setMockInstance(JobContext job, String instanceName) {
-    setMockInstance(job.getConfiguration(), instanceName);
-  }
-
   public static void setMockInstance(Configuration conf, String instanceName) {
     conf.setBoolean(INSTANCE_HAS_BEEN_SET, true);
     conf.setBoolean(MOCK, true);
     conf.set(INSTANCE_NAME, instanceName);
   }
   
-  /**
-   * @deprecated Use {@link #setMaxMutationBufferSize(Configuration,long)} instead
-   */
-  public static void setMaxMutationBufferSize(JobContext job, long numberOfBytes) {
-    setMaxMutationBufferSize(job.getConfiguration(), numberOfBytes);
-  }
-
   public static void setMaxMutationBufferSize(Configuration conf, long numberOfBytes) {
     conf.setLong(MAX_MUTATION_BUFFER_SIZE, numberOfBytes);
   }
   
-  /**
-   * @deprecated Use {@link #setMaxLatency(Configuration,int)} instead
-   */
-  public static void setMaxLatency(JobContext job, int numberOfMilliseconds) {
-    setMaxLatency(job.getConfiguration(), numberOfMilliseconds);
-  }
-  
   public static void setMaxLatency(Configuration conf, int numberOfMilliseconds) {
     conf.setInt(MAX_LATENCY, numberOfMilliseconds);
   }
   
-  /**
-   * @deprecated Use {@link #setMaxWriteThreads(Configuration,int)} instead
-   */
-  public static void setMaxWriteThreads(JobContext job, int numberOfThreads) {
-    setMaxWriteThreads(job.getConfiguration(), numberOfThreads);
-  }
-  
   public static void setMaxWriteThreads(Configuration conf, int numberOfThreads) {
     conf.setInt(NUM_WRITE_THREADS, numberOfThreads);
   }
-  
-  /**
-   * @deprecated Use {@link #setLogLevel(Configuration,Level)} instead
-   */
-  public static void setLogLevel(JobContext job, Level level) {
-    setLogLevel(job.getConfiguration(), level);
-  }
-  
+
   public static void setLogLevel(Configuration conf, Level level) {
     ArgumentChecker.notNull(level);
     conf.setInt(LOGLEVEL, level.toInt());
   }
   
-  /**
-   * @deprecated Use {@link #setSimulationMode(Configurtion)} instead
-   */
-  public static void setSimulationMode(JobContext job) {
-    setSimulationMode(job.getConfiguration());
-  }
-  
   public static void setSimulationMode(Configuration conf) {
     conf.setBoolean(SIMULATE, true);
   }
   
-  /**
-   * @deprecated Use {@link #getUsername(Configuration)} instead
-   */
-  protected static String getUsername(JobContext job) {
-    return getUsername(job.getConfiguration());
-  }
-  
   protected static String getUsername(Configuration conf) {
     return conf.get(USERNAME);
   }
@@ -234,109 +159,43 @@ public class AccumuloOutputFormat extend
   /**
    * WARNING: The password is stored in the Configuration and shared with all MapReduce tasks; It is BASE64 encoded to provide a charset safe conversion to a
    * string, and is not intended to be secure.
-   * 
-   * @deprecated Use {@link #getPassword(Configuration)} instead
-   */
-  protected static byte[] getPassword(JobContext job) {
-    return getPassword(job.getConfiguration());
-  }
-  
-  /**
-   * WARNING: The password is stored in the Configuration and shared with all MapReduce tasks; It is BASE64 encoded to provide a charset safe conversion to a
-   * string, and is not intended to be secure.
    */
   protected static byte[] getPassword(Configuration conf) {
     return Base64.decodeBase64(conf.get(PASSWORD, "").getBytes());
   }
   
-  /**
-   * @deprecated Use {@link #canCreateTables(Configuration)} instead
-   */
-  protected static boolean canCreateTables(JobContext job) {
-    return canCreateTables(job.getConfiguration());
-  }
-  
   protected static boolean canCreateTables(Configuration conf) {
     return conf.getBoolean(CREATETABLES, false);
   }
   
-  /**
-   * @deprecated Use {@link #getDefaultTableName(Configuration)} instead
-   */
-  protected static String getDefaultTableName(JobContext job) {
-    return getDefaultTableName(job.getConfiguration());
-  }
-  
   protected static String getDefaultTableName(Configuration conf) {
     return conf.get(DEFAULT_TABLE_NAME);
   }
   
-  /**
-   * @deprecated Use {@link #getInstance(Configuration)} instead
-   */
-  protected static Instance getInstance(JobContext job) {
-    return getInstance(job.getConfiguration());
-  }
-  
   protected static Instance getInstance(Configuration conf) {
     if (conf.getBoolean(MOCK, false))
       return new MockInstance(conf.get(INSTANCE_NAME));
     return new ZooKeeperInstance(conf.get(INSTANCE_NAME), conf.get(ZOOKEEPERS));
   }
   
-  /**
-   * @deprecated Use {@link #getMaxMutationBufferSize(Configuration)} instead
-   */
-  protected static long getMaxMutationBufferSize(JobContext job) {
-    return getMaxMutationBufferSize(job.getConfiguration());
-  }
-  
   protected static long getMaxMutationBufferSize(Configuration conf) {
     return conf.getLong(MAX_MUTATION_BUFFER_SIZE, DEFAULT_MAX_MUTATION_BUFFER_SIZE);
   }
   
-  /**
-   * @deprecated Use {@link #getMaxLatency(Configuration)} instead
-   */
-  protected static int getMaxLatency(JobContext job) {
-    return getMaxLatency(job.getConfiguration());
-  }
-  
   protected static int getMaxLatency(Configuration conf) {
     return conf.getInt(MAX_LATENCY, DEFAULT_MAX_LATENCY);
   }
   
-  /**
-   * @deprecated Use {@link #getMaxWriteThreads(Configuration)} instead
-   */
-  protected static int getMaxWriteThreads(JobContext job) {
-    return getMaxWriteThreads(job.getConfiguration());
-  }
-  
   protected static int getMaxWriteThreads(Configuration conf) {
     return conf.getInt(NUM_WRITE_THREADS, DEFAULT_NUM_WRITE_THREADS);
   }
   
-  /**
-   * @deprecated Use {@link #getLogLevel(Configuration)} instead
-   */
-  protected static Level getLogLevel(JobContext job) {
-    return getLogLevel(job.getConfiguration());
-  }
-  
   protected static Level getLogLevel(Configuration conf) {
     if (conf.get(LOGLEVEL) != null)
       return Level.toLevel(conf.getInt(LOGLEVEL, Level.INFO.toInt()));
     return null;
   }
   
-  /**
-   * @deprecated Use {@link #getSimulationMode(Configuration)} instead
-   */
-  protected static boolean getSimulationMode(JobContext job) {
-    return getSimulationMode(job.getConfiguration());
-  }
-  
   protected static boolean getSimulationMode(Configuration conf) {
     return conf.getBoolean(SIMULATE, false);
   }
@@ -355,23 +214,24 @@ public class AccumuloOutputFormat extend
     private Connector conn;
     
     AccumuloRecordWriter(TaskAttemptContext attempt) throws AccumuloException, AccumuloSecurityException {
-      Level l = getLogLevel(attempt);
+      Level l = getLogLevel(attempt.getConfiguration());
       if (l != null)
-        log.setLevel(getLogLevel(attempt));
-      this.simulate = getSimulationMode(attempt);
-      this.createTables = canCreateTables(attempt);
+        log.setLevel(getLogLevel(attempt.getConfiguration()));
+      this.simulate = getSimulationMode(attempt.getConfiguration());
+      this.createTables = canCreateTables(attempt.getConfiguration());
       
       if (simulate)
         log.info("Simulating output only. No writes to tables will occur");
       
       this.bws = new HashMap<Text,BatchWriter>();
       
-      String tname = getDefaultTableName(attempt);
+      String tname = getDefaultTableName(attempt.getConfiguration());
       this.defaultTableName = (tname == null) ? null : new Text(tname);
       
       if (!simulate) {
-        this.conn = getInstance(attempt).getConnector(getUsername(attempt), getPassword(attempt));
-        mtbw = conn.createMultiTableBatchWriter(getMaxMutationBufferSize(attempt), getMaxLatency(attempt), getMaxWriteThreads(attempt));
+        this.conn = getInstance(attempt.getConfiguration()).getConnector(getUsername(attempt.getConfiguration()), getPassword(attempt.getConfiguration()));
+        mtbw = conn.createMultiTableBatchWriter(getMaxMutationBufferSize(attempt.getConfiguration()), getMaxLatency(attempt.getConfiguration()),
+            getMaxWriteThreads(attempt.getConfiguration()));
       }
     }
     
@@ -501,8 +361,8 @@ public class AccumuloOutputFormat extend
     if (!conf.getBoolean(INSTANCE_HAS_BEEN_SET, false))
       throw new IOException("Instance info has not been set.");
     try {
-      Connector c = getInstance(job).getConnector(getUsername(job), getPassword(job));
-      if (!c.securityOperations().authenticateUser(getUsername(job), getPassword(job)))
+      Connector c = getInstance(job.getConfiguration()).getConnector(getUsername(job.getConfiguration()), getPassword(job.getConfiguration()));
+      if (!c.securityOperations().authenticateUser(getUsername(job.getConfiguration()), getPassword(job.getConfiguration())))
         throw new IOException("Unable to authenticate user");
     } catch (AccumuloException e) {
       throw new IOException(e);

Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java (original)
+++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java Thu Jan 12 19:16:28 2012
@@ -60,7 +60,7 @@ public class AccumuloFileOutputFormatTes
   
   @Test
   public void testSet() throws IOException, InterruptedException {
-    AccumuloFileOutputFormat.setBlockSize(job, 300);
+    AccumuloFileOutputFormat.setBlockSize(job.getConfiguration(), 300);
     validate(300);
   }
   
@@ -98,7 +98,7 @@ public class AccumuloFileOutputFormatTes
   }
   
   public void validate(int size) throws IOException, InterruptedException {
-    AccumuloFileOutputFormat.handleBlockSize(job);
+    AccumuloFileOutputFormat.handleBlockSize(job.getConfiguration());
     int detSize = job.getConfiguration().getInt("io.seqfile.compress.blocksize", -1);
     assertEquals(size, detSize);
   }

Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java (original)
+++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java Thu Jan 12 19:16:28 2012
@@ -95,8 +95,8 @@ public class AccumuloOutputFormatTest {
     job.setNumReduceTasks(0);
     AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable1", new Authorizations());
     AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmrinstance");
-    AccumuloOutputFormat.setOutputInfo(job, "root", "".getBytes(), false, "testtable2");
-    AccumuloOutputFormat.setMockInstance(job, "testmrinstance");
+    AccumuloOutputFormat.setOutputInfo(job.getConfiguration(), "root", "".getBytes(), false, "testtable2");
+    AccumuloOutputFormat.setMockInstance(job.getConfiguration(), "testmrinstance");
     
     AccumuloInputFormat input = new AccumuloInputFormat();
     List<InputSplit> splits = input.getSplits(job);

Modified: incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/helloworld/InsertWithOutputFormat.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/helloworld/InsertWithOutputFormat.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/helloworld/InsertWithOutputFormat.java (original)
+++ incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/helloworld/InsertWithOutputFormat.java Thu Jan 12 19:16:28 2012
@@ -44,8 +44,8 @@ public class InsertWithOutputFormat exte
     Text tableName = new Text(args[2]);
     Job job = new Job(getConf());
     Configuration conf = job.getConfiguration();
-    AccumuloOutputFormat.setZooKeeperInstance(job, args[0], args[1]);
-    AccumuloOutputFormat.setOutputInfo(job, args[3], args[4].getBytes(), true, null);
+    AccumuloOutputFormat.setZooKeeperInstance(job.getConfiguration(), args[0], args[1]);
+    AccumuloOutputFormat.setOutputInfo(job.getConfiguration(), args[3], args[4].getBytes(), true, null);
     job.setOutputFormatClass(AccumuloOutputFormat.class);
     
     // when running a mapreduce, you won't need to instantiate the output

Modified: incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/RowHash.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/RowHash.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/RowHash.java (original)
+++ incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/RowHash.java Thu Jan 12 19:16:28 2012
@@ -77,8 +77,8 @@ public class RowHash extends Configured 
     job.setNumReduceTasks(0);
     
     job.setOutputFormatClass(AccumuloOutputFormat.class);
-    AccumuloOutputFormat.setZooKeeperInstance(job, args[0], args[1]);
-    AccumuloOutputFormat.setOutputInfo(job, args[2], args[3].getBytes(), true, args[6]);
+    AccumuloOutputFormat.setZooKeeperInstance(job.getConfiguration(), args[0], args[1]);
+    AccumuloOutputFormat.setOutputInfo(job.getConfiguration(), args[2], args[3].getBytes(), true, args[6]);
     // AccumuloOutputFormat.setLogLevel(job, Level.TRACE);
     
     job.waitForCompletion(true);

Modified: incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TeraSortIngest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TeraSortIngest.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TeraSortIngest.java (original)
+++ incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TeraSortIngest.java Thu Jan 12 19:16:28 2012
@@ -356,9 +356,9 @@ public class TeraSortIngest extends Conf
     job.setNumReduceTasks(0);
     
     job.setOutputFormatClass(AccumuloOutputFormat.class);
-    AccumuloOutputFormat.setZooKeeperInstance(job, args[6], args[7]);
-    AccumuloOutputFormat.setOutputInfo(job, args[8], args[9].getBytes(), true, null);
-    AccumuloOutputFormat.setMaxMutationBufferSize(job, 10L * 1000 * 1000);
+    AccumuloOutputFormat.setZooKeeperInstance(job.getConfiguration(), args[6], args[7]);
+    AccumuloOutputFormat.setOutputInfo(job.getConfiguration(), args[8], args[9].getBytes(), true, null);
+    AccumuloOutputFormat.setMaxMutationBufferSize(job.getConfiguration(), 10L * 1000 * 1000);
     
     Configuration conf = job.getConfiguration();
     conf.setLong(NUMROWS, Long.parseLong(args[0]));

Modified: incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/WordCount.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/WordCount.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/WordCount.java (original)
+++ incubator/accumulo/trunk/src/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/WordCount.java Thu Jan 12 19:16:28 2012
@@ -104,8 +104,8 @@ public class WordCount extends Configure
     job.setOutputFormatClass(AccumuloOutputFormat.class);
     job.setOutputKeyClass(Text.class);
     job.setOutputValueClass(Mutation.class);
-    AccumuloOutputFormat.setOutputInfo(job, username, password.getBytes(), true, args[3]);
-    AccumuloOutputFormat.setZooKeeperInstance(job, args[0], args[1]);
+    AccumuloOutputFormat.setOutputInfo(job.getConfiguration(), username, password.getBytes(), true, args[3]);
+    AccumuloOutputFormat.setZooKeeperInstance(job.getConfiguration(), args[0], args[1]);
     job.waitForCompletion(true);
     return 0;
   }

Modified: incubator/accumulo/trunk/src/examples/wikisearch/ingest/src/main/java/org/apache/accumulo/examples/wikisearch/ingest/WikipediaIngester.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/examples/wikisearch/ingest/src/main/java/org/apache/accumulo/examples/wikisearch/ingest/WikipediaIngester.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/examples/wikisearch/ingest/src/main/java/org/apache/accumulo/examples/wikisearch/ingest/WikipediaIngester.java (original)
+++ incubator/accumulo/trunk/src/examples/wikisearch/ingest/src/main/java/org/apache/accumulo/examples/wikisearch/ingest/WikipediaIngester.java Thu Jan 12 19:16:28 2012
@@ -171,8 +171,8 @@ public class WikipediaIngester extends C
     job.setMapOutputKeyClass(Text.class);
     job.setMapOutputValueClass(Mutation.class);
     job.setOutputFormatClass(AccumuloOutputFormat.class);
-    AccumuloOutputFormat.setOutputInfo(job, user, password, true, tablename);
-    AccumuloOutputFormat.setZooKeeperInstance(job, instanceName, zookeepers);
+    AccumuloOutputFormat.setOutputInfo(job.getConfiguration(), user, password, true, tablename);
+    AccumuloOutputFormat.setZooKeeperInstance(job.getConfiguration(), instanceName, zookeepers);
     
     return job.waitForCompletion(true) ? 0 : 1;
   }

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousMoru.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousMoru.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousMoru.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousMoru.java Thu Jan 12 19:16:28 2012
@@ -145,11 +145,11 @@ public class ContinuousMoru extends Conf
     job.setNumReduceTasks(0);
     
     job.setOutputFormatClass(AccumuloOutputFormat.class);
-    AccumuloOutputFormat.setOutputInfo(job, user, pass.getBytes(), false, table);
-    AccumuloOutputFormat.setZooKeeperInstance(job, instance, zookeepers);
-    AccumuloOutputFormat.setMaxLatency(job, (int) (Integer.parseInt(maxLatency) / 1000.0));
-    AccumuloOutputFormat.setMaxMutationBufferSize(job, Long.parseLong(maxMem));
-    AccumuloOutputFormat.setMaxWriteThreads(job, Integer.parseInt(numThreads));
+    AccumuloOutputFormat.setOutputInfo(job.getConfiguration(), user, pass.getBytes(), false, table);
+    AccumuloOutputFormat.setZooKeeperInstance(job.getConfiguration(), instance, zookeepers);
+    AccumuloOutputFormat.setMaxLatency(job.getConfiguration(), (int) (Integer.parseInt(maxLatency) / 1000.0));
+    AccumuloOutputFormat.setMaxMutationBufferSize(job.getConfiguration(), Long.parseLong(maxMem));
+    AccumuloOutputFormat.setMaxWriteThreads(job.getConfiguration(), Integer.parseInt(numThreads));
     
     Configuration conf = job.getConfiguration();
     conf.setLong(MIN, Long.parseLong(min));

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/CopyTool.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/CopyTool.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/CopyTool.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/CopyTool.java Thu Jan 12 19:16:28 2012
@@ -62,8 +62,8 @@ public class CopyTool extends Configured
     job.setNumReduceTasks(0);
     
     job.setOutputFormatClass(AccumuloOutputFormat.class);
-    AccumuloOutputFormat.setOutputInfo(job, args[0], args[1].getBytes(), true, args[5]);
-    AccumuloOutputFormat.setZooKeeperInstance(job, args[3], args[4]);
+    AccumuloOutputFormat.setOutputInfo(job.getConfiguration(), args[0], args[1].getBytes(), true, args[5]);
+    AccumuloOutputFormat.setZooKeeperInstance(job.getConfiguration(), args[3], args[4]);
     
     job.waitForCompletion(true);
     return job.isSuccessful() ? 0 : 1;

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/sequential/MapRedVerifyTool.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/sequential/MapRedVerifyTool.java?rev=1230709&r1=1230708&r2=1230709&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/sequential/MapRedVerifyTool.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/sequential/MapRedVerifyTool.java Thu Jan 12 19:16:28 2012
@@ -95,8 +95,8 @@ public class MapRedVerifyTool extends Co
     job.setNumReduceTasks(1);
     
     job.setOutputFormatClass(AccumuloOutputFormat.class);
-    AccumuloOutputFormat.setOutputInfo(job, args[0], args[1].getBytes(), true, args[5]);
-    AccumuloOutputFormat.setZooKeeperInstance(job, args[3], args[4]);
+    AccumuloOutputFormat.setOutputInfo(job.getConfiguration(), args[0], args[1].getBytes(), true, args[5]);
+    AccumuloOutputFormat.setZooKeeperInstance(job.getConfiguration(), args[3], args[4]);
     
     job.waitForCompletion(true);
     return job.isSuccessful() ? 0 : 1;