You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2013/11/09 04:17:22 UTC

[1/3] git commit: ACCUMULO-1854 Move RangeInputSplit into its own file and store all connection information into it.

Updated Branches:
  refs/heads/ACCUMULO-1854-info-in-splits [created] a9644f5b9


ACCUMULO-1854 Move RangeInputSplit into its own file and store all
connection information into it.


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

Branch: refs/heads/ACCUMULO-1854-info-in-splits
Commit: cdbed432400da7b17b177e4a53d9440ff23bff9f
Parents: 6648056
Author: Josh Elser <el...@apache.org>
Authored: Fri Nov 8 15:46:39 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Nov 8 15:46:39 2013 -0500

----------------------------------------------------------------------
 .../core/client/mapreduce/InputFormatBase.java  | 491 +++++++++----------
 .../core/client/mapreduce/RangeInputSplit.java  | 402 +++++++++++++++
 .../mapreduce/AccumuloInputFormatTest.java      |   1 -
 .../mapreduce/AccumuloRowInputFormatTest.java   |   1 -
 4 files changed, 622 insertions(+), 273 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/cdbed432/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index 8e238f1..d3ebd21 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -110,7 +110,7 @@ import org.apache.log4j.Logger;
 
 public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static final Logger log = Logger.getLogger(InputFormatBase.class);
-  
+
   private static final String PREFIX = AccumuloInputFormat.class.getSimpleName();
   private static final String INPUT_INFO_HAS_BEEN_SET = PREFIX + ".configured";
   private static final String INSTANCE_HAS_BEEN_SET = PREFIX + ".instanceConfigured";
@@ -118,34 +118,34 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   private static final String PASSWORD = PREFIX + ".password";
   private static final String TABLE_NAME = PREFIX + ".tablename";
   private static final String AUTHORIZATIONS = PREFIX + ".authorizations";
-  
+
   private static final String INSTANCE_NAME = PREFIX + ".instanceName";
   private static final String ZOOKEEPERS = PREFIX + ".zooKeepers";
   private static final String MOCK = ".useMockInstance";
-  
+
   private static final String RANGES = PREFIX + ".ranges";
   private static final String AUTO_ADJUST_RANGES = PREFIX + ".ranges.autoAdjust";
-  
+
   private static final String ROW_REGEX = PREFIX + ".regex.row";
   private static final String COLUMN_FAMILY_REGEX = PREFIX + ".regex.cf";
   private static final String COLUMN_QUALIFIER_REGEX = PREFIX + ".regex.cq";
   private static final String VALUE_REGEX = PREFIX + ".regex.value";
-  
+
   private static final String COLUMNS = PREFIX + ".columns";
   private static final String LOGLEVEL = PREFIX + ".loglevel";
-  
+
   private static final String ISOLATED = PREFIX + ".isolated";
-  
+
   private static final String LOCAL_ITERATORS = PREFIX + ".localiters";
-  
+
   // Used to specify the maximum # of versions of an Accumulo cell value to return
   private static final String MAX_VERSIONS = PREFIX + ".maxVersions";
-  
+
   // Used for specifying the iterators to be applied
   private static final String ITERATORS = PREFIX + ".iterators";
   private static final String ITERATORS_OPTIONS = PREFIX + ".iterators.options";
   private static final String ITERATORS_DELIM = ",";
-  
+
   private static final String READ_OFFLINE = PREFIX + ".read.offline";
 
   /**
@@ -154,7 +154,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIsolated(JobContext job, boolean enable) {
     setIsolated(job.getConfiguration(), enable);
   }
-  
+
   /**
    * Enable or disable use of the {@link IsolatedScanner} in this configuration object. By default it is not enabled.
    * 
@@ -166,14 +166,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIsolated(Configuration conf, boolean enable) {
     conf.setBoolean(ISOLATED, enable);
   }
-  
+
   /**
    * @deprecated Use {@link #setLocalIterators(Configuration,boolean)} instead
    */
   public static void setLocalIterators(JobContext job, boolean enable) {
     setLocalIterators(job.getConfiguration(), enable);
   }
-  
+
   /**
    * Enable or disable use of the {@link ClientSideIteratorScanner} in this Configuration object. By default it is not enabled.
    * 
@@ -185,14 +185,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setLocalIterators(Configuration conf, boolean enable) {
     conf.setBoolean(LOCAL_ITERATORS, enable);
   }
-  
+
   /**
    * @deprecated Use {@link #setInputInfo(Configuration,String,byte[],String,Authorizations)} instead
    */
   public static void setInputInfo(JobContext job, String user, byte[] passwd, String table, Authorizations auths) {
     setInputInfo(job.getConfiguration(), user, passwd, table, auths);
   }
-  
+
   /**
    * Initialize the user, table, and authorization information for the configuration object that will be used with an Accumulo InputFormat.
    * 
@@ -211,7 +211,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     if (conf.getBoolean(INPUT_INFO_HAS_BEEN_SET, false))
       throw new IllegalStateException("Input info can only be set once per job");
     conf.setBoolean(INPUT_INFO_HAS_BEEN_SET, true);
-    
+
     ArgumentChecker.notNull(user, passwd, table);
     conf.set(USERNAME, user);
     conf.set(PASSWORD, new String(Base64.encodeBase64(passwd)));
@@ -219,14 +219,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     if (auths != null && !auths.isEmpty())
       conf.set(AUTHORIZATIONS, auths.serialize());
   }
-  
+
   /**
    * @deprecated Use {@link #setZooKeeperInstance(Configuration,String,String)} instead
    */
   public static void setZooKeeperInstance(JobContext job, String instanceName, String zooKeepers) {
     setZooKeeperInstance(job.getConfiguration(), instanceName, zooKeepers);
   }
-  
+
   /**
    * Configure a {@link ZooKeeperInstance} for this configuration object.
    * 
@@ -241,19 +241,19 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     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);
   }
-  
+
   /**
    * @deprecated Use {@link #setMockInstance(Configuration,String)} instead
    */
   public static void setMockInstance(JobContext job, String instanceName) {
     setMockInstance(job.getConfiguration(), instanceName);
   }
-  
+
   /**
    * Configure a {@link MockInstance} for this configuration object.
    * 
@@ -267,14 +267,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     conf.setBoolean(MOCK, true);
     conf.set(INSTANCE_NAME, instanceName);
   }
-  
+
   /**
    * @deprecated Use {@link #setRanges(Configuration,Collection)} instead
    */
   public static void setRanges(JobContext job, Collection<Range> ranges) {
     setRanges(job.getConfiguration(), ranges);
   }
-  
+
   /**
    * Set the ranges to map over for this configuration object.
    * 
@@ -297,14 +297,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     conf.setStrings(RANGES, rangeStrings.toArray(new String[0]));
   }
-  
+
   /**
    * @deprecated Use {@link #disableAutoAdjustRanges(Configuration)} instead
    */
   public static void disableAutoAdjustRanges(JobContext job) {
     disableAutoAdjustRanges(job.getConfiguration());
   }
-  
+
   /**
    * Disables the adjustment of ranges for this configuration object. By default, overlapping ranges will be merged and ranges will be fit to existing tablet
    * boundaries. Disabling this adjustment will cause there to be exactly one mapper per range set using {@link #setRanges(Configuration, Collection)}.
@@ -315,14 +315,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void disableAutoAdjustRanges(Configuration conf) {
     conf.setBoolean(AUTO_ADJUST_RANGES, false);
   }
-  
+
   /**
    * @deprecated since 1.4 use {@link org.apache.accumulo.core.iterators.user.RegExFilter} and {@link #addIterator(Configuration, IteratorSetting)}
    */
   public static enum RegexType {
     ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VALUE
   }
-  
+
   /**
    * @deprecated since 1.4 use {@link #addIterator(Configuration, IteratorSetting)}
    * @see org.apache.accumulo.core.iterators.user.RegExFilter#setRegexs(IteratorSetting, String, String, String, String, boolean)
@@ -356,14 +356,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * @deprecated Use {@link #setMaxVersions(Configuration,int)} instead
    */
   public static void setMaxVersions(JobContext job, int maxVersions) throws IOException {
     setMaxVersions(job.getConfiguration(), maxVersions);
   }
-  
+
   /**
    * Sets the max # of values that may be returned for an individual Accumulo cell. By default, applied before all other Accumulo iterators (highest priority)
    * leveraged in the scan by the record reader. To adjust priority use setIterator() & setIteratorOptions() w/ the VersioningIterator type explicitly.
@@ -380,7 +380,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       throw new IOException("Invalid maxVersions: " + maxVersions + ".  Must be >= 1");
     conf.setInt(MAX_VERSIONS, maxVersions);
   }
-  
+
   /**
    * <p>
    * Enable reading offline tables. This will make the map reduce job directly read the tables files. If the table is not offline, then the job will fail. If
@@ -407,18 +407,18 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @param scanOff
    *          pass true to read offline tables
    */
-  
+
   public static void setScanOffline(Configuration conf, boolean scanOff) {
     conf.setBoolean(READ_OFFLINE, scanOff);
   }
-  
+
   /**
    * @deprecated Use {@link #fetchColumns(Configuration,Collection)} instead
    */
   public static void fetchColumns(JobContext job, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
     fetchColumns(job.getConfiguration(), columnFamilyColumnQualifierPairs);
   }
-  
+
   /**
    * Restricts the columns that will be mapped over for this configuration object.
    * 
@@ -429,27 +429,33 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    *          selected. An empty set is the default and is equivalent to scanning the all columns.
    */
   public static void fetchColumns(Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
+    String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
+    conf.setStrings(COLUMNS, columnStrings);
+  }
+
+  public static String[] serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
     ArgumentChecker.notNull(columnFamilyColumnQualifierPairs);
     ArrayList<String> columnStrings = new ArrayList<String>(columnFamilyColumnQualifierPairs.size());
     for (Pair<Text,Text> column : columnFamilyColumnQualifierPairs) {
       if (column.getFirst() == null)
         throw new IllegalArgumentException("Column family can not be null");
-      
+
       String col = new String(Base64.encodeBase64(TextUtil.getBytes(column.getFirst())));
       if (column.getSecond() != null)
         col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())));
       columnStrings.add(col);
     }
-    conf.setStrings(COLUMNS, columnStrings.toArray(new String[0]));
+
+    return columnStrings.toArray(new String[0]);
   }
-  
+
   /**
    * @deprecated Use {@link #setLogLevel(Configuration,Level)} instead
    */
   public static void setLogLevel(JobContext job, Level level) {
     setLogLevel(job.getConfiguration(), level);
   }
-  
+
   /**
    * Sets the log level for this configuration object.
    * 
@@ -463,14 +469,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     log.setLevel(level);
     conf.setInt(LOGLEVEL, level.toInt());
   }
-  
+
   /**
    * @deprecated Use {@link #addIterator(Configuration,IteratorSetting)} instead
    */
   public static void addIterator(JobContext job, IteratorSetting cfg) {
     addIterator(job.getConfiguration(), cfg);
   }
-  
+
   /**
    * Encode an iterator on the input for this configuration object.
    * 
@@ -482,7 +488,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void addIterator(Configuration conf, IteratorSetting cfg) {
     // First check to see if anything has been set already
     String iterators = conf.get(ITERATORS);
-    
+
     // No iterators specified yet, create a new string
     if (iterators == null || iterators.isEmpty()) {
       iterators = new AccumuloIterator(cfg.getPriority(), cfg.getIteratorClass(), cfg.getName()).toString();
@@ -495,9 +501,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     for (Entry<String,String> entry : cfg.getOptions().entrySet()) {
       if (entry.getValue() == null)
         continue;
-      
+
       String iteratorOptions = conf.get(ITERATORS_OPTIONS);
-      
+
       // No options specified yet, create a new string
       if (iteratorOptions == null || iteratorOptions.isEmpty()) {
         iteratorOptions = new AccumuloIteratorOption(cfg.getName(), entry.getKey(), entry.getValue()).toString();
@@ -505,12 +511,12 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         // append the next option & reset
         iteratorOptions = iteratorOptions.concat(ITERATORS_DELIM + new AccumuloIteratorOption(cfg.getName(), entry.getKey(), entry.getValue()));
       }
-      
+
       // Store the options w/ the job
       conf.set(ITERATORS_OPTIONS, iteratorOptions);
     }
   }
-  
+
   /**
    * Specify an Accumulo iterator type to manage the behavior of the underlying table scan this InputFormat's RecordReader will conduct, w/ priority dictating
    * the order in which specified iterators are applied. Repeat calls to specify multiple iterators are allowed.
@@ -529,7 +535,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIterator(JobContext job, int priority, String iteratorClass, String iteratorName) {
     // First check to see if anything has been set already
     String iterators = job.getConfiguration().get(ITERATORS);
-    
+
     // No iterators specified yet, create a new string
     if (iterators == null || iterators.isEmpty()) {
       iterators = new AccumuloIterator(priority, iteratorClass, iteratorName).toString();
@@ -539,9 +545,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     // Store the iterators w/ the job
     job.getConfiguration().set(ITERATORS, iterators);
-    
+
   }
-  
+
   /**
    * Specify an option for a named Accumulo iterator, further specifying that iterator's behavior.
    * 
@@ -559,9 +565,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIteratorOption(JobContext job, String iteratorName, String key, String value) {
     if (iteratorName == null || key == null || value == null)
       return;
-    
+
     String iteratorOptions = job.getConfiguration().get(ITERATORS_OPTIONS);
-    
+
     // No options specified yet, create a new string
     if (iteratorOptions == null || iteratorOptions.isEmpty()) {
       iteratorOptions = new AccumuloIteratorOption(iteratorName, key, value).toString();
@@ -569,18 +575,18 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       // append the next option & reset
       iteratorOptions = iteratorOptions.concat(ITERATORS_DELIM + new AccumuloIteratorOption(iteratorName, key, value));
     }
-    
+
     // Store the options w/ the job
     job.getConfiguration().set(ITERATORS_OPTIONS, iteratorOptions);
   }
-  
+
   /**
    * @deprecated Use {@link #isIsolated(Configuration)} instead
    */
   protected static boolean isIsolated(JobContext job) {
     return isIsolated(job.getConfiguration());
   }
-  
+
   /**
    * Determines whether a configuration has isolation enabled.
    * 
@@ -592,14 +598,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean isIsolated(Configuration conf) {
     return conf.getBoolean(ISOLATED, false);
   }
-  
+
   /**
    * @deprecated Use {@link #usesLocalIterators(Configuration)} instead
    */
   protected static boolean usesLocalIterators(JobContext job) {
     return usesLocalIterators(job.getConfiguration());
   }
-  
+
   /**
    * Determines whether a configuration uses local iterators.
    * 
@@ -611,14 +617,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean usesLocalIterators(Configuration conf) {
     return conf.getBoolean(LOCAL_ITERATORS, false);
   }
-  
+
   /**
    * @deprecated Use {@link #getUsername(Configuration)} instead
    */
   protected static String getUsername(JobContext job) {
     return getUsername(job.getConfiguration());
   }
-  
+
   /**
    * Gets the user name from the configuration.
    * 
@@ -630,7 +636,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static String getUsername(Configuration conf) {
     return conf.get(USERNAME);
   }
-  
+
   /**
    * 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.
@@ -640,7 +646,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static byte[] getPassword(JobContext job) {
     return getPassword(job.getConfiguration());
   }
-  
+
   /**
    * Gets the password from the configuration. 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.
@@ -653,14 +659,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static byte[] getPassword(Configuration conf) {
     return Base64.decodeBase64(conf.get(PASSWORD, "").getBytes());
   }
-  
+
   /**
    * @deprecated Use {@link #getTablename(Configuration)} instead
    */
   protected static String getTablename(JobContext job) {
     return getTablename(job.getConfiguration());
   }
-  
+
   /**
    * Gets the table name from the configuration.
    * 
@@ -672,14 +678,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static String getTablename(Configuration conf) {
     return conf.get(TABLE_NAME);
   }
-  
+
   /**
    * @deprecated Use {@link #getAuthorizations(Configuration)} instead
    */
   protected static Authorizations getAuthorizations(JobContext job) {
     return getAuthorizations(job.getConfiguration());
   }
-  
+
   /**
    * Gets the authorizations to set for the scans from the configuration.
    * 
@@ -692,14 +698,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     String authString = conf.get(AUTHORIZATIONS);
     return authString == null ? Constants.NO_AUTHS : new Authorizations(authString.split(","));
   }
-  
+
   /**
    * @deprecated Use {@link #getInstance(Configuration)} instead
    */
   protected static Instance getInstance(JobContext job) {
     return getInstance(job.getConfiguration());
   }
-  
+
   /**
    * Initializes an Accumulo {@link Instance} based on the configuration.
    * 
@@ -714,14 +720,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       return new MockInstance(conf.get(INSTANCE_NAME));
     return new ZooKeeperInstance(conf.get(INSTANCE_NAME), conf.get(ZOOKEEPERS));
   }
-  
+
   /**
    * @deprecated Use {@link #getTabletLocator(Configuration)} instead
    */
   protected static TabletLocator getTabletLocator(JobContext job) throws TableNotFoundException {
     return getTabletLocator(job.getConfiguration());
   }
-  
+
   /**
    * Initializes an Accumulo {@link TabletLocator} based on the configuration.
    * 
@@ -741,14 +747,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     return TabletLocator.getInstance(instance, new AuthInfo(username, ByteBuffer.wrap(password), instance.getInstanceID()),
         new Text(Tables.getTableId(instance, tableName)));
   }
-  
+
   /**
    * @deprecated Use {@link #getRanges(Configuration)} instead
    */
   protected static List<Range> getRanges(JobContext job) throws IOException {
     return getRanges(job.getConfiguration());
   }
-  
+
   /**
    * Gets the ranges to scan over from a configuration object.
    * 
@@ -769,7 +775,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return ranges;
   }
-  
+
   /**
    * @deprecated since 1.4 use {@link org.apache.accumulo.core.iterators.user.RegExFilter} and {@link #addIterator(Configuration, IteratorSetting)}
    * @see #setRegex(JobContext, RegexType, String)
@@ -802,14 +808,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * @deprecated Use {@link #getFetchedColumns(Configuration)} instead
    */
   protected static Set<Pair<Text,Text>> getFetchedColumns(JobContext job) {
     return getFetchedColumns(job.getConfiguration());
   }
-  
+
   /**
    * Gets the columns to be mapped over from this configuration object.
    * 
@@ -819,8 +825,19 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @see #fetchColumns(Configuration, Collection)
    */
   protected static Set<Pair<Text,Text>> getFetchedColumns(Configuration conf) {
+    ArgumentChecker.notNull(conf);
+    
+    return deserializeFetchedColumns(conf.getStrings(COLUMNS));
+  }
+
+  public static Set<Pair<Text,Text>> deserializeFetchedColumns(String[] serialized) {
     Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
-    for (String col : conf.getStringCollection(COLUMNS)) {
+    
+    if (null == serialized) {
+      return columns;
+    }
+    
+    for (String col : serialized) {
       int idx = col.indexOf(":");
       Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes()) : Base64.decodeBase64(col.substring(0, idx).getBytes()));
       Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes()));
@@ -828,14 +845,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return columns;
   }
-  
+
   /**
    * @deprecated Use {@link #getAutoAdjustRanges(Configuration)} instead
    */
   protected static boolean getAutoAdjustRanges(JobContext job) {
     return getAutoAdjustRanges(job.getConfiguration());
   }
-  
+
   /**
    * Determines whether a configuration has auto-adjust ranges enabled.
    * 
@@ -847,14 +864,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean getAutoAdjustRanges(Configuration conf) {
     return conf.getBoolean(AUTO_ADJUST_RANGES, true);
   }
-  
+
   /**
    * @deprecated Use {@link #getLogLevel(Configuration)} instead
    */
   protected static Level getLogLevel(JobContext job) {
     return getLogLevel(job.getConfiguration());
   }
-  
+
   /**
    * Gets the log level from this configuration.
    * 
@@ -866,7 +883,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static Level getLogLevel(Configuration conf) {
     return Level.toLevel(conf.getInt(LOGLEVEL, Level.INFO.toInt()));
   }
-  
+
   // InputFormat doesn't have the equivalent of OutputFormat's
   // checkOutputSpecs(JobContext job)
   /**
@@ -875,7 +892,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static void validateOptions(JobContext job) throws IOException {
     validateOptions(job.getConfiguration());
   }
-  
+
   // InputFormat doesn't have the equivalent of OutputFormat's
   // checkOutputSpecs(JobContext job)
   /**
@@ -898,7 +915,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new IOException("Unable to authenticate user");
       if (!c.securityOperations().hasTablePermission(getUsername(conf), getTablename(conf), TablePermission.READ))
         throw new IOException("Unable to access table");
-      
+
       if (!usesLocalIterators(conf)) {
         // validate that any scan-time iterators can be loaded by the the tablet servers
         for (AccumuloIterator iter : getIterators(conf)) {
@@ -906,21 +923,21 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
             throw new AccumuloException("Servers are unable to load " + iter.getIteratorClass() + " as a " + SortedKeyValueIterator.class.getName());
         }
       }
-      
+
     } catch (AccumuloException e) {
       throw new IOException(e);
     } catch (AccumuloSecurityException e) {
       throw new IOException(e);
     }
   }
-  
+
   /**
    * @deprecated Use {@link #getMaxVersions(Configuration)} instead
    */
   protected static int getMaxVersions(JobContext job) {
     return getMaxVersions(job.getConfiguration());
   }
-  
+
   /**
    * Gets the maxVersions to use for the {@link VersioningIterator} from this configuration.
    * 
@@ -932,7 +949,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static int getMaxVersions(Configuration conf) {
     return conf.getInt(MAX_VERSIONS, -1);
   }
-  
+
   protected static boolean isOfflineScan(Configuration conf) {
     return conf.getBoolean(READ_OFFLINE, false);
   }
@@ -945,7 +962,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static List<AccumuloIterator> getIterators(JobContext job) {
     return getIterators(job.getConfiguration());
   }
-  
+
   /**
    * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
    * 
@@ -955,13 +972,13 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @see #addIterator(Configuration, IteratorSetting)
    */
   protected static List<AccumuloIterator> getIterators(Configuration conf) {
-    
+
     String iterators = conf.get(ITERATORS);
-    
+
     // If no iterators are present, return an empty list
     if (iterators == null || iterators.isEmpty())
       return new ArrayList<AccumuloIterator>();
-    
+
     // Compose the set of iterators encoded in the job configuration
     StringTokenizer tokens = new StringTokenizer(conf.get(ITERATORS), ITERATORS_DELIM);
     List<AccumuloIterator> list = new ArrayList<AccumuloIterator>();
@@ -971,14 +988,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return list;
   }
-  
+
   /**
    * @deprecated Use {@link #getIteratorOptions(Configuration)} instead
    */
   protected static List<AccumuloIteratorOption> getIteratorOptions(JobContext job) {
     return getIteratorOptions(job.getConfiguration());
   }
-  
+
   /**
    * Gets a list of the iterator options specified on this configuration.
    * 
@@ -989,11 +1006,11 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    */
   protected static List<AccumuloIteratorOption> getIteratorOptions(Configuration conf) {
     String iteratorOptions = conf.get(ITERATORS_OPTIONS);
-    
+
     // If no options are present, return an empty list
     if (iteratorOptions == null || iteratorOptions.isEmpty())
       return new ArrayList<AccumuloIteratorOption>();
-    
+
     // Compose the set of options encoded in the job configuration
     StringTokenizer tokens = new StringTokenizer(conf.get(ITERATORS_OPTIONS), ITERATORS_DELIM);
     List<AccumuloIteratorOption> list = new ArrayList<AccumuloIteratorOption>();
@@ -1003,13 +1020,13 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return list;
   }
-  
+
   protected abstract static class RecordReaderBase<K,V> extends RecordReader<K,V> {
     protected long numKeysRead;
     protected Iterator<Entry<Key,Value>> scannerIterator;
     private boolean scannerRegexEnabled = false;
     protected RangeInputSplit split;
-    
+
     /**
      * @deprecated since 1.4, configure {@link org.apache.accumulo.core.iterators.user.RegExFilter} instead.
      */
@@ -1024,7 +1041,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         log.info("Setting " + methodName + " to " + regex);
       }
     }
-    
+
     /**
      * @deprecated since 1.4, configure {@link org.apache.accumulo.core.iterators.user.RegExFilter} instead.
      */
@@ -1039,7 +1056,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new AccumuloException("Can't set up regex for scanner");
       }
     }
-    
+
     // Apply the configured iterators from the job to the scanner
     /**
      * @deprecated Use {@link #setupIterators(Configuration,Scanner)} instead
@@ -1047,7 +1064,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     protected void setupIterators(TaskAttemptContext attempt, Scanner scanner) throws AccumuloException {
       setupIterators(attempt.getConfiguration(), scanner);
     }
-    
+
     /**
      * Apply the configured iterators from the configuration to the scanner.
      * 
@@ -1060,7 +1077,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     protected void setupIterators(Configuration conf, Scanner scanner) throws AccumuloException {
       List<AccumuloIterator> iterators = getIterators(conf);
       List<AccumuloIteratorOption> options = getIteratorOptions(conf);
-      
+
       Map<String,IteratorSetting> scanIterators = new HashMap<String,IteratorSetting>();
       for (AccumuloIterator iterator : iterators) {
         scanIterators.put(iterator.getIteratorName(), new IteratorSetting(iterator.getPriority(), iterator.getIteratorName(), iterator.getIteratorClass()));
@@ -1072,14 +1089,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         scanner.addScanIterator(scanIterators.get(iterator.getIteratorName()));
       }
     }
-    
+
     /**
      * @deprecated Use {@link #setupMaxVersions(Configuration,Scanner)} instead
      */
     protected void setupMaxVersions(TaskAttemptContext attempt, Scanner scanner) {
       setupMaxVersions(attempt.getConfiguration(), scanner);
     }
-    
+
     /**
      * If maxVersions has been set, configure a {@link VersioningIterator} at priority 0 for this scanner.
      * 
@@ -1097,20 +1114,20 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         scanner.addScanIterator(vers);
       }
     }
-    
+
     /**
      * Initialize a scanner over the given input split using this task attempt configuration.
      */
     public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
       Scanner scanner;
       split = (RangeInputSplit) inSplit;
-      log.debug("Initializing input split: " + split.range);
+      log.debug("Initializing input split: " + split.getRange());
       Configuration conf = attempt.getConfiguration();
       Instance instance = getInstance(conf);
       String user = getUsername(conf);
       byte[] password = getPassword(conf);
       Authorizations authorizations = getAuthorizations(conf);
-      
+
       try {
         log.debug("Creating connector with user: " + user);
         Connector conn = instance.getConnector(user, password);
@@ -1131,18 +1148,16 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           scanner = new ClientSideIteratorScanner(scanner);
         }
         setupMaxVersions(conf, scanner);
-        if (conf.get(ROW_REGEX) != null || conf.get(COLUMN_FAMILY_REGEX) != null || conf.get(COLUMN_QUALIFIER_REGEX) != null ||
-            conf.get(VALUE_REGEX) != null) {
+        if (conf.get(ROW_REGEX) != null || conf.get(COLUMN_FAMILY_REGEX) != null || conf.get(COLUMN_QUALIFIER_REGEX) != null || conf.get(VALUE_REGEX) != null) {
           IteratorSetting is = new IteratorSetting(50, RegExFilter.class);
-          RegExFilter.setRegexs(is, conf.get(ROW_REGEX), conf.get(COLUMN_FAMILY_REGEX), conf.get(COLUMN_QUALIFIER_REGEX),
-            conf.get(VALUE_REGEX), false);
+          RegExFilter.setRegexs(is, conf.get(ROW_REGEX), conf.get(COLUMN_FAMILY_REGEX), conf.get(COLUMN_QUALIFIER_REGEX), conf.get(VALUE_REGEX), false);
           scanner.addScanIterator(is);
         }
         setupIterators(conf, scanner);
       } catch (Exception e) {
         throw new IOException(e);
       }
-      
+
       // setup a scanner within the bounds of this split
       for (Pair<Text,Text> c : getFetchedColumns(conf)) {
         if (c.getSecond() != null) {
@@ -1153,48 +1168,48 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           scanner.fetchColumnFamily(c.getFirst());
         }
       }
-      
-      scanner.setRange(split.range);
-      
+
+      scanner.setRange(split.getRange());
+
       numKeysRead = 0;
-      
+
       // do this last after setting all scanner options
       scannerIterator = scanner.iterator();
     }
-    
+
     public void close() {}
-    
+
     public float getProgress() throws IOException {
       if (numKeysRead > 0 && currentKey == null)
         return 1.0f;
       return split.getProgress(currentKey);
     }
-    
+
     protected K currentK = null;
     protected V currentV = null;
     protected Key currentKey = null;
     protected Value currentValue = null;
-    
+
     @Override
     public K getCurrentKey() throws IOException, InterruptedException {
       return currentK;
     }
-    
+
     @Override
     public V getCurrentValue() throws IOException, InterruptedException {
       return currentV;
     }
   }
-  
+
   Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext job, String tableName, List<Range> ranges) throws TableNotFoundException,
       AccumuloException, AccumuloSecurityException {
-    
+
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 
     Instance instance = getInstance(job.getConfiguration());
     Connector conn = instance.getConnector(getUsername(job.getConfiguration()), getPassword(job.getConfiguration()));
     String tableId = Tables.getTableId(instance, tableName);
-    
+
     if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
       Tables.clearCache(instance);
       if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
@@ -1204,12 +1219,12 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
 
     for (Range range : ranges) {
       Text startRow;
-      
+
       if (range.getStartKey() != null)
         startRow = range.getStartKey().getRow();
       else
         startRow = new Text();
-      
+
       Range metadataRange = new Range(new KeyExtent(new Text(tableId), startRow, null).getMetadataEntry(), true, null, false);
       Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
       ColumnFQ.fetch(scanner, Constants.METADATA_PREV_ROW_COLUMN);
@@ -1217,9 +1232,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
       scanner.fetchColumnFamily(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY);
       scanner.setRange(metadataRange);
-      
+
       RowIterator rowIter = new RowIterator(scanner);
-      
+
       // TODO check that extents match prev extent
 
       KeyExtent lastExtent = null;
@@ -1229,15 +1244,15 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         String last = "";
         KeyExtent extent = null;
         String location = null;
-        
+
         while (row.hasNext()) {
           Entry<Key,Value> entry = row.next();
           Key key = entry.getKey();
-          
+
           if (key.getColumnFamily().equals(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY)) {
             last = entry.getValue().toString();
           }
-          
+
           if (key.getColumnFamily().equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY)
               || key.getColumnFamily().equals(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY)) {
             location = entry.getValue().toString();
@@ -1246,9 +1261,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           if (Constants.METADATA_PREV_ROW_COLUMN.hasColumns(key)) {
             extent = new KeyExtent(key.getRow(), entry.getValue());
           }
-          
+
         }
-        
+
         if (location != null)
           return null;
 
@@ -1265,24 +1280,24 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           tabletRanges = new HashMap<KeyExtent,List<Range>>();
           binnedRanges.put(last, tabletRanges);
         }
-        
+
         List<Range> rangeList = tabletRanges.get(extent);
         if (rangeList == null) {
           rangeList = new ArrayList<Range>();
           tabletRanges.put(extent, rangeList);
         }
-        
+
         rangeList.add(range);
 
         if (extent.getEndRow() == null || range.afterEndKey(new Key(extent.getEndRow()).followingKey(PartialKey.ROW))) {
           break;
         }
-        
+
         lastExtent = extent;
       }
 
     }
-    
+
     return binnedRanges;
   }
 
@@ -1290,18 +1305,33 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * Read the metadata table to get tablets and match up ranges to them.
    */
   public List<InputSplit> getSplits(JobContext job) throws IOException {
-    log.setLevel(getLogLevel(job.getConfiguration()));
-    validateOptions(job.getConfiguration());
-    
-    String tableName = getTablename(job.getConfiguration());
-    boolean autoAdjust = getAutoAdjustRanges(job.getConfiguration());
-    List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(job.getConfiguration())) : getRanges(job.getConfiguration());
-    
+    Configuration conf = job.getConfiguration();
+
+    log.setLevel(getLogLevel(conf));
+    validateOptions(conf);
+
+    String tableName = getTablename(conf);
+    boolean autoAdjust = getAutoAdjustRanges(conf);
+    List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(conf)) : getRanges(conf);
+    boolean offline = isOfflineScan(conf);
+    boolean isolated = isIsolated(conf);
+    boolean localIterators = usesLocalIterators(conf);
+    boolean mockInstance = conf.getBoolean(MOCK, false);
+    int maxVersions = getMaxVersions(conf);
+    String rowRegex = conf.get(ROW_REGEX), colfamRegex = conf.get(COLUMN_FAMILY_REGEX), colqualRegex = conf.get(COLUMN_QUALIFIER_REGEX), 
+        valueRegex = conf.get(VALUE_REGEX);
+    Set<Pair<Text,Text>> fetchedColumns = getFetchedColumns(conf);
+    Authorizations auths = getAuthorizations(conf);
+    byte[] password = getPassword(conf);
+    String username = getUsername(conf);
+    Instance instance = getInstance(conf);
+        
+
     if (ranges.isEmpty()) {
       ranges = new ArrayList<Range>(1);
       ranges.add(new Range());
     }
-    
+
     // get the metadata information for these ranges
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
     TabletLocator tl;
@@ -1314,7 +1344,6 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           binnedRanges = binOfflineTable(job, tableName, ranges);
         }
       } else {
-        Instance instance = getInstance(job.getConfiguration());
         String tableId = null;
         tl = getTabletLocator(job.getConfiguration());
         // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
@@ -1337,15 +1366,15 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     } catch (Exception e) {
       throw new IOException(e);
     }
-    
+
     ArrayList<InputSplit> splits = new ArrayList<InputSplit>(ranges.size());
     HashMap<Range,ArrayList<String>> splitsToAdd = null;
-    
+
     if (!autoAdjust)
       splitsToAdd = new HashMap<Range,ArrayList<String>>();
-    
+
     HashMap<String,String> hostNameCache = new HashMap<String,String>();
-    
+
     for (Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
       String ip = tserverBin.getKey().split(":", 2)[0];
       String location = hostNameCache.get(ip);
@@ -1354,14 +1383,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         location = inetAddress.getHostName();
         hostNameCache.put(ip, location);
       }
-      
+
       for (Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
         Range ke = extentRanges.getKey().toDataRange();
         for (Range r : extentRanges.getValue()) {
           if (autoAdjust) {
             // divide ranges into smaller ranges, based on the
             // tablets
-            splits.add(new RangeInputSplit(tableName, ke.clip(r), new String[] {location}));
+            splits.add(new RangeInputSplit(ke.clip(r), new String[] {location}));
           } else {
             // don't divide ranges
             ArrayList<String> locations = splitsToAdd.get(r);
@@ -1373,132 +1402,52 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         }
       }
     }
-    
+
     if (!autoAdjust)
       for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
-        splits.add(new RangeInputSplit(tableName, entry.getKey(), entry.getValue().toArray(new String[0])));
-    return splits;
-  }
-  
-  /**
-   * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
-   */
-  public static class RangeInputSplit extends InputSplit implements Writable {
-    private Range range;
-    private String[] locations;
-    
-    public RangeInputSplit() {
-      range = new Range();
-      locations = new String[0];
-    }
-    
-    public Range getRange() {
-      return range;
-    }
-    
-    private static byte[] extractBytes(ByteSequence seq, int numBytes) {
-      byte[] bytes = new byte[numBytes + 1];
-      bytes[0] = 0;
-      for (int i = 0; i < numBytes; i++) {
-        if (i >= seq.length())
-          bytes[i + 1] = 0;
-        else
-          bytes[i + 1] = seq.byteAt(i);
-      }
-      return bytes;
-    }
-    
-    public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
-      int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
-      BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
-      BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
-      BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
-      return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
-    }
-    
-    public float getProgress(Key currentKey) {
-      if (currentKey == null)
-        return 0f;
-      if (range.getStartKey() != null && range.getEndKey() != null) {
-        if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
-          // just look at the row progress
-          return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
-        } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
-          // just look at the column family progress
-          return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
-        } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL) != 0) {
-          // just look at the column qualifier progress
-          return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
-        }
-      }
-      // if we can't figure it out, then claim no progress
-      return 0f;
-    }
-    
-    RangeInputSplit(String table, Range range, String[] locations) {
-      this.range = range;
-      this.locations = locations;
-    }
-    
-    /**
-     * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
-     */
-    public long getLength() throws IOException {
-      Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
-      Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
-      int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
-      long diff = 0;
-      
-      byte[] start = startRow.getBytes();
-      byte[] stop = stopRow.getBytes();
-      for (int i = 0; i < maxCommon; ++i) {
-        diff |= 0xff & (start[i] ^ stop[i]);
-        diff <<= Byte.SIZE;
-      }
-      
-      if (startRow.getLength() != stopRow.getLength())
-        diff |= 0xff;
-      
-      return diff + 1;
-    }
-    
-    public String[] getLocations() throws IOException {
-      return locations;
-    }
-    
-    public void readFields(DataInput in) throws IOException {
-      range.readFields(in);
-      int numLocs = in.readInt();
-      locations = new String[numLocs];
-      for (int i = 0; i < numLocs; ++i)
-        locations[i] = in.readUTF();
-    }
-    
-    public void write(DataOutput out) throws IOException {
-      range.write(out);
-      out.writeInt(locations.length);
-      for (int i = 0; i < locations.length; ++i)
-        out.writeUTF(locations[i]);
+        splits.add(new RangeInputSplit(entry.getKey(), entry.getValue().toArray(new String[0])));
+
+    for (InputSplit inputSplit : splits) {
+      RangeInputSplit split = (RangeInputSplit) inputSplit;
+
+      split.setTable(tableName);
+      split.setOffline(offline);
+      split.setIsolatedScan(isolated);
+      split.setUsesLocalIterators(localIterators);
+      split.setMockInstance(mockInstance);
+      split.setMaxVersions(maxVersions);
+      split.setRowRegex(rowRegex);
+      split.setColfamRegex(colfamRegex);
+      split.setColqualRegex(colqualRegex);
+      split.setValueRegex(valueRegex);
+      split.setFetchedColumns(fetchedColumns);
+      split.setUsername(username);
+      split.setPassword(password);
+      split.setInstanceName(instance.getInstanceName());
+      split.setZooKeepers(instance.getZooKeepers());
+      split.setAuths(auths);
     }
+
+    return splits;
   }
-  
+
   /**
    * The Class IteratorSetting. Encapsulates specifics for an Accumulo iterator's name & priority.
    */
   static class AccumuloIterator {
-    
+
     private static final String FIELD_SEP = ":";
-    
+
     private int priority;
     private String iteratorClass;
     private String iteratorName;
-    
+
     public AccumuloIterator(int priority, String iteratorClass, String iteratorName) {
       this.priority = priority;
       this.iteratorClass = iteratorClass;
       this.iteratorName = iteratorName;
     }
-    
+
     // Parses out a setting given an string supplied from an earlier toString() call
     public AccumuloIterator(String iteratorSetting) {
       // Parse the string to expand the iterator
@@ -1507,42 +1456,42 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       iteratorClass = tokenizer.nextToken();
       iteratorName = tokenizer.nextToken();
     }
-    
+
     public int getPriority() {
       return priority;
     }
-    
+
     public String getIteratorClass() {
       return iteratorClass;
     }
-    
+
     public String getIteratorName() {
       return iteratorName;
     }
-    
+
     @Override
     public String toString() {
       return new String(priority + FIELD_SEP + iteratorClass + FIELD_SEP + iteratorName);
     }
-    
+
   }
-  
+
   /**
    * The Class AccumuloIteratorOption. Encapsulates specifics for an Accumulo iterator's optional configuration details - associated via the iteratorName.
    */
   static class AccumuloIteratorOption {
     private static final String FIELD_SEP = ":";
-    
+
     private String iteratorName;
     private String key;
     private String value;
-    
+
     public AccumuloIteratorOption(String iteratorName, String key, String value) {
       this.iteratorName = iteratorName;
       this.key = key;
       this.value = value;
     }
-    
+
     // Parses out an option given a string supplied from an earlier toString() call
     public AccumuloIteratorOption(String iteratorOption) {
       StringTokenizer tokenizer = new StringTokenizer(iteratorOption, FIELD_SEP);
@@ -1554,19 +1503,19 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new RuntimeException(e);
       }
     }
-    
+
     public String getIteratorName() {
       return iteratorName;
     }
-    
+
     public String getKey() {
       return key;
     }
-    
+
     public String getValue() {
       return value;
     }
-    
+
     @Override
     public String toString() {
       try {
@@ -1575,7 +1524,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new RuntimeException(e);
       }
     }
-    
+
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cdbed432/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
new file mode 100644
index 0000000..bba7cf0
--- /dev/null
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -0,0 +1,402 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapreduce;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Set;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
+ */
+public class RangeInputSplit extends InputSplit implements Writable {
+  private Range range;
+  private String[] locations;
+  private String table, instanceName, zooKeepers, username;
+  private String rowRegex, colfamRegex, colqualRegex, valueRegex;
+  private byte[] password;
+  private boolean offline = false, mockInstance = false, isolatedScan = false, localIterators = false;
+  private int maxVersions = 1;
+  private Authorizations auths;
+  private Set<Pair<Text,Text>> fetchedColumns;
+
+  public RangeInputSplit() {
+    range = new Range();
+    locations = new String[0];
+  }
+
+  public Range getRange() {
+    return range;
+  }
+
+  private static byte[] extractBytes(ByteSequence seq, int numBytes) {
+    byte[] bytes = new byte[numBytes + 1];
+    bytes[0] = 0;
+    for (int i = 0; i < numBytes; i++) {
+      if (i >= seq.length())
+        bytes[i + 1] = 0;
+      else
+        bytes[i + 1] = seq.byteAt(i);
+    }
+    return bytes;
+  }
+
+  public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
+    int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
+    BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
+    BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
+    BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
+    return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
+  }
+
+  public float getProgress(Key currentKey) {
+    if (currentKey == null)
+      return 0f;
+    if (range.getStartKey() != null && range.getEndKey() != null) {
+      if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
+        // just look at the row progress
+        return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
+      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
+        // just look at the column family progress
+        return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
+      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL) != 0) {
+        // just look at the column qualifier progress
+        return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
+      }
+    }
+    // if we can't figure it out, then claim no progress
+    return 0f;
+  }
+
+  public RangeInputSplit(Range range, String[] locations) {
+    this.range = range;
+    this.locations = locations;
+  }
+
+  /**
+   * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
+   */
+  public long getLength() throws IOException {
+    Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
+    Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
+    int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
+    long diff = 0;
+
+    byte[] start = startRow.getBytes();
+    byte[] stop = stopRow.getBytes();
+    for (int i = 0; i < maxCommon; ++i) {
+      diff |= 0xff & (start[i] ^ stop[i]);
+      diff <<= Byte.SIZE;
+    }
+
+    if (startRow.getLength() != stopRow.getLength())
+      diff |= 0xff;
+
+    return diff + 1;
+  }
+
+  public String[] getLocations() throws IOException {
+    return locations;
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    range.readFields(in);
+    int numLocs = in.readInt();
+    locations = new String[numLocs];
+    for (int i = 0; i < numLocs; ++i)
+      locations[i] = in.readUTF();
+    
+    isolatedScan = in.readBoolean();
+    offline = in.readBoolean();
+    localIterators = in.readBoolean();
+    mockInstance = in.readBoolean();
+    
+    maxVersions = in.readInt();
+    
+    if (in.readBoolean()) {
+      rowRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      colfamRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      colqualRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      valueRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      int numColumns = in.readInt();
+      String[] columns = new String[numColumns];
+      for (int i = 0; i < numColumns; i++) {
+        columns[i] = in.readUTF();
+      }
+      
+      fetchedColumns = InputFormatBase.deserializeFetchedColumns(columns);
+    }
+    
+    if (in.readBoolean()) {
+      auths = new Authorizations(StringUtils.split(in.readUTF()));
+    }
+    
+    if (in.readBoolean()) {
+      username = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      password = in.readUTF().getBytes();
+    }
+    
+    if (in.readBoolean()) {
+      instanceName = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      zooKeepers = in.readUTF();
+    }
+  }
+
+  public void write(DataOutput out) throws IOException {
+    range.write(out);
+    out.writeInt(locations.length);
+    for (int i = 0; i < locations.length; ++i)
+      out.writeUTF(locations[i]);
+    
+    out.writeBoolean(isIsolatedScan());
+    out.writeBoolean(isOffline());
+    out.writeBoolean(usesLocalIterators());
+    out.writeBoolean(isMockInstance());
+    
+    out.writeInt(getMaxVersions());
+    
+    out.writeBoolean(null != rowRegex);
+    if (null != rowRegex) {
+      out.writeUTF(rowRegex);
+    }
+    
+    out.writeBoolean(null != colfamRegex);
+    if (null != colfamRegex) {
+      out.writeUTF(colfamRegex);
+    }
+    
+    out.writeBoolean(null != colqualRegex);
+    if (null != colqualRegex) {
+      out.writeUTF(colqualRegex);
+    }
+    
+    out.writeBoolean(null != valueRegex);
+    if (null != valueRegex) {
+      out.writeUTF(valueRegex);
+    }
+    
+    out.writeBoolean(null != fetchedColumns);
+    if (null != fetchedColumns) {
+      String[] cols = InputFormatBase.serializeColumns(fetchedColumns);
+      out.writeInt(cols.length);
+      for (String col : cols) {
+        out.writeUTF(col);
+      }
+    }
+    
+    out.writeBoolean(null != auths);
+    if (null != auths) {
+      out.writeUTF(auths.serialize());
+    }
+    
+    out.writeBoolean(null != username);
+    if (null != username) {
+      out.writeUTF(username);
+    }
+    
+    out.writeBoolean(null != password);
+    if (null != password) {
+      out.writeUTF(new String(password));
+    }
+    
+    out.writeBoolean(null != instanceName);
+    if (null != instanceName) {
+      out.writeUTF(instanceName);
+    }
+    
+    out.writeBoolean(null != zooKeepers);
+    if (null != zooKeepers) {
+      out.writeUTF(zooKeepers);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder(128);
+    sb.append("Range: ").append(range);
+    sb.append(" Locations: ").append(locations);
+    sb.append(" Table: ").append(table);
+    return sb.toString();
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+
+  public String getInstanceName() {
+    return instanceName;
+  }
+
+  public void setInstanceName(String instanceName) {
+    this.instanceName = instanceName;
+  }
+
+  public String getZooKeepers() {
+    return zooKeepers;
+  }
+
+  public void setZooKeepers(String zooKeepers) {
+    this.zooKeepers = zooKeepers;
+  }
+
+  public String getUsername() {
+    return username;
+  }
+
+  public void setUsername(String username) {
+    this.username = username;
+  }
+
+  public byte[] getPassword() {
+    return password;
+  }
+
+  public void setPassword(byte[] password) {
+    this.password = password;
+  }
+
+  public boolean isOffline() {
+    return offline;
+  }
+
+  public void setOffline(boolean offline) {
+    this.offline = offline;
+  }
+
+  public void setLocations(String[] locations) {
+    this.locations = locations;
+  }
+
+  public String getRowRegex() {
+    return rowRegex;
+  }
+
+  public void setRowRegex(String rowRegex) {
+    this.rowRegex = rowRegex;
+  }
+
+  public String getColfamRegex() {
+    return colfamRegex;
+  }
+
+  public void setColfamRegex(String colfamRegex) {
+    this.colfamRegex = colfamRegex;
+  }
+
+  public String getColqualRegex() {
+    return colqualRegex;
+  }
+
+  public void setColqualRegex(String colqualRegex) {
+    this.colqualRegex = colqualRegex;
+  }
+
+  public String getValueRegex() {
+    return valueRegex;
+  }
+
+  public void setValueRegex(String valueRegex) {
+    this.valueRegex = valueRegex;
+  }
+
+  public boolean isMockInstance() {
+    return mockInstance;
+  }
+
+  public void setMockInstance(boolean mockInstance) {
+    this.mockInstance = mockInstance;
+  }
+
+  public boolean isIsolatedScan() {
+    return isolatedScan;
+  }
+
+  public void setIsolatedScan(boolean isolatedScan) {
+    this.isolatedScan = isolatedScan;
+  }
+
+  public int getMaxVersions() {
+    return maxVersions;
+  }
+
+  public void setMaxVersions(int maxVersions) {
+    this.maxVersions = maxVersions;
+  }
+
+  public Authorizations getAuths() {
+    return auths;
+  }
+
+  public void setAuths(Authorizations auths) {
+    this.auths = auths;
+  }
+
+  public void setRange(Range range) {
+    this.range = range;
+  }
+
+  public boolean usesLocalIterators() {
+    return localIterators;
+  }
+
+  public void setUsesLocalIterators(boolean localIterators) {
+    this.localIterators = localIterators;
+  }
+
+  public Set<Pair<Text,Text>> getFetchedColumns() {
+    return fetchedColumns;
+  }
+
+  public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
+    this.fetchedColumns = fetchedColumns;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cdbed432/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
index ba647e9..2f68dde 100644
--- a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
@@ -28,7 +28,6 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIterator;
 import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIteratorOption;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit;
 import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RegexType;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.data.Key;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cdbed432/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
index 0673f1b..d9f9da0 100644
--- a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
@@ -27,7 +27,6 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;


[2/3] git commit: ACCUMULO-1854 Clean up constructors. Add a test.

Posted by el...@apache.org.
ACCUMULO-1854 Clean up constructors. Add a test.


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

Branch: refs/heads/ACCUMULO-1854-info-in-splits
Commit: 7c549ab0d5a05a1f5daa36be21ece2e9e291aa3c
Parents: cdbed43
Author: Josh Elser <el...@apache.org>
Authored: Fri Nov 8 16:23:49 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Nov 8 16:23:49 2013 -0500

----------------------------------------------------------------------
 .../core/client/mapreduce/RangeInputSplit.java  | 10 +-
 .../client/mapreduce/RangeInputSplitTest.java   | 97 ++++++++++++++++++++
 2 files changed, 102 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c549ab0/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index bba7cf0..4cd16b2 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -52,6 +52,11 @@ public class RangeInputSplit extends InputSplit implements Writable {
     locations = new String[0];
   }
 
+  public RangeInputSplit(Range range, String[] locations) {
+    this.range = range;
+    this.locations = locations;
+  }
+
   public Range getRange() {
     return range;
   }
@@ -95,11 +100,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
     return 0f;
   }
 
-  public RangeInputSplit(Range range, String[] locations) {
-    this.range = range;
-    this.locations = locations;
-  }
-
   /**
    * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
    */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c549ab0/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
new file mode 100644
index 0000000..abcbde0
--- /dev/null
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
@@ -0,0 +1,97 @@
+package org.apache.accumulo.core.client.mapreduce;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class RangeInputSplitTest {
+
+  @Test
+  public void testSimpleWritable() throws IOException {
+    RangeInputSplit split = new RangeInputSplit(new Range(new Key("a"), new Key("b")), new String[]{"localhost"});
+    
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    split.write(dos);
+    
+    RangeInputSplit newSplit = new RangeInputSplit();
+    
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInputStream dis = new DataInputStream(bais);
+    newSplit.readFields(dis);
+    
+    Assert.assertEquals(split.getRange(), newSplit.getRange());
+    Assert.assertTrue(Arrays.equals(split.getLocations(), newSplit.getLocations()));
+  }
+
+
+
+  @Test
+  public void testAllFieldsWritable() throws IOException {
+    RangeInputSplit split = new RangeInputSplit(new Range(new Key("a"), new Key("b")), new String[]{"localhost"});
+    
+    Set<Pair<Text,Text>> fetchedColumns = new HashSet<Pair<Text,Text>>();
+    
+    fetchedColumns.add(new Pair<Text,Text>(new Text("colf1"), new Text("colq1")));
+    fetchedColumns.add(new Pair<Text,Text>(new Text("colf2"), new Text("colq2")));
+    
+    split.setAuths(new Authorizations("foo"));
+    split.setOffline(true);
+    split.setIsolatedScan(true);
+    split.setUsesLocalIterators(true);
+    split.setMaxVersions(5);
+    split.setRowRegex("row");
+    split.setColfamRegex("colf");
+    split.setColqualRegex("colq");
+    split.setValueRegex("value");
+    split.setFetchedColumns(fetchedColumns);
+    split.setPassword("password".getBytes());
+    split.setUsername("root");
+    split.setInstanceName("instance");
+    split.setMockInstance(true);
+    split.setZooKeepers("localhost");
+    
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    split.write(dos);
+    
+    RangeInputSplit newSplit = new RangeInputSplit();
+    
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInputStream dis = new DataInputStream(bais);
+    newSplit.readFields(dis);
+    
+    Assert.assertEquals(split.getRange(), newSplit.getRange());
+    Assert.assertArrayEquals(split.getLocations(), newSplit.getLocations());
+    
+    Assert.assertEquals(split.getAuths(), newSplit.getAuths());
+    Assert.assertEquals(split.isOffline(), newSplit.isOffline());
+    Assert.assertEquals(split.isIsolatedScan(), newSplit.isOffline());
+    Assert.assertEquals(split.usesLocalIterators(), newSplit.usesLocalIterators());
+    Assert.assertEquals(split.getMaxVersions(), newSplit.getMaxVersions());
+    Assert.assertEquals(split.getRowRegex(), newSplit.getRowRegex());
+    Assert.assertEquals(split.getColfamRegex(), newSplit.getColfamRegex());
+    Assert.assertEquals(split.getColqualRegex(), newSplit.getColqualRegex());
+    Assert.assertEquals(split.getValueRegex(), newSplit.getValueRegex());
+    Assert.assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
+    Assert.assertEquals(new String(split.getPassword()), new String(newSplit.getPassword()));
+    Assert.assertEquals(split.getUsername(), newSplit.getUsername());
+    Assert.assertEquals(split.getInstanceName(), newSplit.getInstanceName());
+    Assert.assertEquals(split.isMockInstance(), newSplit.isMockInstance());
+    Assert.assertEquals(split.getZooKeepers(), newSplit.getZooKeepers());
+  }
+  
+}


[3/3] git commit: ACCUMULO-1854 Fix up InputFormatBase to use the information stored on RangeInputSplit and fall back onto the Configuration.

Posted by el...@apache.org.
ACCUMULO-1854 Fix up InputFormatBase to use the information stored on
RangeInputSplit and fall back onto the Configuration.


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

Branch: refs/heads/ACCUMULO-1854-info-in-splits
Commit: a9644f5b94d74466db624fb06bd3d70fb5bf9cf9
Parents: 7c549ab
Author: Josh Elser <el...@apache.org>
Authored: Fri Nov 8 17:47:57 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Nov 8 17:47:57 2013 -0500

----------------------------------------------------------------------
 .../core/client/mapreduce/InputFormatBase.java  | 140 +++++++++++++++----
 .../core/client/mapreduce/RangeInputSplit.java  | 116 ++++++++++++---
 .../simple/filedata/ChunkInputFormatTest.java   |   4 +-
 3 files changed, 204 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a9644f5b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index d3ebd21..0fd2630 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -826,17 +826,17 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    */
   protected static Set<Pair<Text,Text>> getFetchedColumns(Configuration conf) {
     ArgumentChecker.notNull(conf);
-    
+
     return deserializeFetchedColumns(conf.getStrings(COLUMNS));
   }
 
   public static Set<Pair<Text,Text>> deserializeFetchedColumns(String[] serialized) {
     Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
-    
+
     if (null == serialized) {
       return columns;
     }
-    
+
     for (String col : serialized) {
       int idx = col.indexOf(":");
       Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes()) : Base64.decodeBase64(col.substring(0, idx).getBytes()));
@@ -1061,8 +1061,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     /**
      * @deprecated Use {@link #setupIterators(Configuration,Scanner)} instead
      */
-    protected void setupIterators(TaskAttemptContext attempt, Scanner scanner) throws AccumuloException {
-      setupIterators(attempt.getConfiguration(), scanner);
+    protected void setupIterators(TaskAttemptContext attempt, Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
+        throws AccumuloException {
+      setupIterators(attempt.getConfiguration(), scanner, iterators, options);
     }
 
     /**
@@ -1074,9 +1075,8 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
      *          the scanner to configure
      * @throws AccumuloException
      */
-    protected void setupIterators(Configuration conf, Scanner scanner) throws AccumuloException {
-      List<AccumuloIterator> iterators = getIterators(conf);
-      List<AccumuloIteratorOption> options = getIteratorOptions(conf);
+    protected void setupIterators(Configuration conf, Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
+        throws AccumuloException {
 
       Map<String,IteratorSetting> scanIterators = new HashMap<String,IteratorSetting>();
       for (AccumuloIterator iterator : iterators) {
@@ -1093,8 +1093,8 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     /**
      * @deprecated Use {@link #setupMaxVersions(Configuration,Scanner)} instead
      */
-    protected void setupMaxVersions(TaskAttemptContext attempt, Scanner scanner) {
-      setupMaxVersions(attempt.getConfiguration(), scanner);
+    protected void setupMaxVersions(TaskAttemptContext attempt, Scanner scanner, int maxVersions) {
+      setupMaxVersions(attempt.getConfiguration(), scanner, maxVersions);
     }
 
     /**
@@ -1105,8 +1105,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
      * @param scanner
      *          the scanner to configure
      */
-    protected void setupMaxVersions(Configuration conf, Scanner scanner) {
-      int maxVersions = getMaxVersions(conf);
+    protected void setupMaxVersions(Configuration conf, Scanner scanner, int maxVersions) {
       // Check to make sure its a legit value
       if (maxVersions >= 1) {
         IteratorSetting vers = new IteratorSetting(0, "vers", VersioningIterator.class);
@@ -1123,43 +1122,119 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       split = (RangeInputSplit) inSplit;
       log.debug("Initializing input split: " + split.getRange());
       Configuration conf = attempt.getConfiguration();
-      Instance instance = getInstance(conf);
-      String user = getUsername(conf);
-      byte[] password = getPassword(conf);
-      Authorizations authorizations = getAuthorizations(conf);
+
+      Instance instance = split.getInstance();
+      if (null == instance) {
+        instance = getInstance(conf);
+      }
+
+      String user = split.getUsername();
+      if (null == user) {
+        user = getUsername(conf);
+      }
+
+      byte[] password = split.getPassword();
+      if (null == password) {
+        password = getPassword(conf);
+      }
+
+      Authorizations authorizations = split.getAuths();
+      if (null == authorizations) {
+        authorizations = getAuthorizations(conf);
+      }
+
+      String table = split.getTable();
+      if (null == table) {
+        table = getTablename(conf);
+      }
+      
+      Boolean isOffline = split.isOffline();
+      if (null == isOffline) {
+        isOffline = isOfflineScan(conf);
+      }
+
+      Boolean isIsolated = split.isIsolatedScan();
+      if (null == isIsolated) {
+        isIsolated = isIsolated(conf);
+      }
+
+      Boolean usesLocalIterators = split.usesLocalIterators();
+      if (null == usesLocalIterators) {
+        usesLocalIterators = usesLocalIterators(conf);
+      }
+
+      String rowRegex = split.getRowRegex();
+      if (null == rowRegex) {
+        rowRegex = conf.get(ROW_REGEX);
+      }
+
+      String colfRegex = split.getColfamRegex();
+      if (null == colfRegex) {
+        colfRegex = conf.get(COLUMN_FAMILY_REGEX);
+      }
+
+      String colqRegex = split.getColqualRegex();
+      if (null == colqRegex) {
+        colqRegex = conf.get(COLUMN_QUALIFIER_REGEX);
+      }
+
+      String valueRegex = split.getValueRegex();
+      if (null == valueRegex) {
+        valueRegex = conf.get(VALUE_REGEX);
+      }
+
+      Integer maxVersions = split.getMaxVersions();
+      if (null == maxVersions) {
+        maxVersions = getMaxVersions(conf);
+      }
+      
+      List<AccumuloIterator> iterators = split.getIterators();
+      if (null == iterators) {
+        iterators = getIterators(conf);
+      }
+      
+      List<AccumuloIteratorOption> options = split.getOptions();
+      if (null == options) {
+        options = getIteratorOptions(conf);
+      }
+      
+      Set<Pair<Text,Text>> columns = split.getFetchedColumns();
+      if (null == columns) {
+        columns = getFetchedColumns(conf);
+      }
 
       try {
         log.debug("Creating connector with user: " + user);
         Connector conn = instance.getConnector(user, password);
-        log.debug("Creating scanner for table: " + getTablename(conf));
+        log.debug("Creating scanner for table: " + table);
         log.debug("Authorizations are: " + authorizations);
-        if (isOfflineScan(conf)) {
-          scanner = new OfflineScanner(instance, new AuthInfo(user, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(instance,
-              getTablename(conf)), authorizations);
+        if (isOffline) {
+          scanner = new OfflineScanner(instance, new AuthInfo(user, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(instance, table),
+              authorizations);
         } else {
-          scanner = conn.createScanner(getTablename(conf), authorizations);
+          scanner = conn.createScanner(table, authorizations);
         }
-        if (isIsolated(conf)) {
+        if (isIsolated) {
           log.info("Creating isolated scanner");
           scanner = new IsolatedScanner(scanner);
         }
-        if (usesLocalIterators(conf)) {
+        if (usesLocalIterators) {
           log.info("Using local iterators");
           scanner = new ClientSideIteratorScanner(scanner);
         }
-        setupMaxVersions(conf, scanner);
-        if (conf.get(ROW_REGEX) != null || conf.get(COLUMN_FAMILY_REGEX) != null || conf.get(COLUMN_QUALIFIER_REGEX) != null || conf.get(VALUE_REGEX) != null) {
+        setupMaxVersions(conf, scanner, maxVersions);
+        if (rowRegex != null || colfRegex != null || colqRegex != null || valueRegex != null) {
           IteratorSetting is = new IteratorSetting(50, RegExFilter.class);
-          RegExFilter.setRegexs(is, conf.get(ROW_REGEX), conf.get(COLUMN_FAMILY_REGEX), conf.get(COLUMN_QUALIFIER_REGEX), conf.get(VALUE_REGEX), false);
+          RegExFilter.setRegexs(is, rowRegex, colfRegex, colqRegex, valueRegex, false);
           scanner.addScanIterator(is);
         }
-        setupIterators(conf, scanner);
+        setupIterators(conf, scanner, iterators, options);
       } catch (Exception e) {
         throw new IOException(e);
       }
 
       // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : getFetchedColumns(conf)) {
+      for (Pair<Text,Text> c : columns) {
         if (c.getSecond() != null) {
           log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
           scanner.fetchColumn(c.getFirst(), c.getSecond());
@@ -1318,14 +1393,15 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     boolean localIterators = usesLocalIterators(conf);
     boolean mockInstance = conf.getBoolean(MOCK, false);
     int maxVersions = getMaxVersions(conf);
-    String rowRegex = conf.get(ROW_REGEX), colfamRegex = conf.get(COLUMN_FAMILY_REGEX), colqualRegex = conf.get(COLUMN_QUALIFIER_REGEX), 
-        valueRegex = conf.get(VALUE_REGEX);
+    String rowRegex = conf.get(ROW_REGEX), colfamRegex = conf.get(COLUMN_FAMILY_REGEX), colqualRegex = conf.get(COLUMN_QUALIFIER_REGEX), valueRegex = conf
+        .get(VALUE_REGEX);
     Set<Pair<Text,Text>> fetchedColumns = getFetchedColumns(conf);
     Authorizations auths = getAuthorizations(conf);
     byte[] password = getPassword(conf);
     String username = getUsername(conf);
     Instance instance = getInstance(conf);
-        
+    List<AccumuloIterator> iterators = getIterators(conf);
+    List<AccumuloIteratorOption> options = getIteratorOptions(conf);
 
     if (ranges.isEmpty()) {
       ranges = new ArrayList<Range>(1);
@@ -1426,6 +1502,8 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       split.setInstanceName(instance.getInstanceName());
       split.setZooKeepers(instance.getZooKeepers());
       split.setAuths(auths);
+      split.setIterators(iterators);
+      split.setOptions(options);
     }
 
     return splits;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a9644f5b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index 4cd16b2..3caa111 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -20,8 +20,14 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.math.BigInteger;
+import java.util.List;
 import java.util.Set;
 
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIterator;
+import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIteratorOption;
+import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
@@ -42,10 +48,12 @@ public class RangeInputSplit extends InputSplit implements Writable {
   private String table, instanceName, zooKeepers, username;
   private String rowRegex, colfamRegex, colqualRegex, valueRegex;
   private byte[] password;
-  private boolean offline = false, mockInstance = false, isolatedScan = false, localIterators = false;
-  private int maxVersions = 1;
+  private Boolean offline, mockInstance, isolatedScan, localIterators;
+  private Integer maxVersions;
   private Authorizations auths;
   private Set<Pair<Text,Text>> fetchedColumns;
+  private List<AccumuloIterator> iterators;
+  private List<AccumuloIteratorOption> options;
 
   public RangeInputSplit() {
     range = new Range();
@@ -133,12 +141,25 @@ public class RangeInputSplit extends InputSplit implements Writable {
     for (int i = 0; i < numLocs; ++i)
       locations[i] = in.readUTF();
     
-    isolatedScan = in.readBoolean();
-    offline = in.readBoolean();
-    localIterators = in.readBoolean();
-    mockInstance = in.readBoolean();
+    if (in.readBoolean()) {
+      isolatedScan = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      offline = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      localIterators = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      mockInstance = in.readBoolean();
+    }
     
-    maxVersions = in.readInt();
+    if (in.readBoolean()) {
+      maxVersions = in.readInt();
+    }
     
     if (in.readBoolean()) {
       rowRegex = in.readUTF();
@@ -193,12 +214,30 @@ public class RangeInputSplit extends InputSplit implements Writable {
     for (int i = 0; i < locations.length; ++i)
       out.writeUTF(locations[i]);
     
-    out.writeBoolean(isIsolatedScan());
-    out.writeBoolean(isOffline());
-    out.writeBoolean(usesLocalIterators());
-    out.writeBoolean(isMockInstance());
+    out.writeBoolean(null != isolatedScan);
+    if (null != isolatedScan) {
+      out.writeBoolean(isolatedScan);
+    }
+    
+    out.writeBoolean(null != offline);
+    if (null != offline) {
+      out.writeBoolean(offline);
+    }
+    
+    out.writeBoolean(null != localIterators);
+    if (null != localIterators) {
+      out.writeBoolean(localIterators);
+    }
+    
+    out.writeBoolean(null != mockInstance);
+    if (null != mockInstance) {
+      out.writeBoolean(mockInstance);
+    }
     
-    out.writeInt(getMaxVersions());
+    out.writeBoolean(null != maxVersions);
+    if (null != maxVersions) {
+      out.writeInt(getMaxVersions());
+    }
     
     out.writeBoolean(null != rowRegex);
     if (null != rowRegex) {
@@ -261,6 +300,7 @@ public class RangeInputSplit extends InputSplit implements Writable {
     sb.append("Range: ").append(range);
     sb.append(" Locations: ").append(locations);
     sb.append(" Table: ").append(table);
+    // TODO finish building of string
     return sb.toString();
   }
 
@@ -271,6 +311,22 @@ public class RangeInputSplit extends InputSplit implements Writable {
   public void setTable(String table) {
     this.table = table;
   }
+  
+  public Instance getInstance() {
+    if (null == instanceName) {
+      return null;
+    }
+    
+    if (isMockInstance()) {  
+      return new MockInstance(getInstanceName());
+    }
+    
+    if (null == zooKeepers) {
+      return null;
+    }
+    
+    return new ZooKeeperInstance(getInstanceName(), getZooKeepers());
+  }
 
   public String getInstanceName() {
     return instanceName;
@@ -304,11 +360,11 @@ public class RangeInputSplit extends InputSplit implements Writable {
     this.password = password;
   }
 
-  public boolean isOffline() {
+  public Boolean isOffline() {
     return offline;
   }
 
-  public void setOffline(boolean offline) {
+  public void setOffline(Boolean offline) {
     this.offline = offline;
   }
 
@@ -348,27 +404,27 @@ public class RangeInputSplit extends InputSplit implements Writable {
     this.valueRegex = valueRegex;
   }
 
-  public boolean isMockInstance() {
+  public Boolean isMockInstance() {
     return mockInstance;
   }
 
-  public void setMockInstance(boolean mockInstance) {
+  public void setMockInstance(Boolean mockInstance) {
     this.mockInstance = mockInstance;
   }
 
-  public boolean isIsolatedScan() {
+  public Boolean isIsolatedScan() {
     return isolatedScan;
   }
 
-  public void setIsolatedScan(boolean isolatedScan) {
+  public void setIsolatedScan(Boolean isolatedScan) {
     this.isolatedScan = isolatedScan;
   }
 
-  public int getMaxVersions() {
+  public Integer getMaxVersions() {
     return maxVersions;
   }
 
-  public void setMaxVersions(int maxVersions) {
+  public void setMaxVersions(Integer maxVersions) {
     this.maxVersions = maxVersions;
   }
 
@@ -384,11 +440,11 @@ public class RangeInputSplit extends InputSplit implements Writable {
     this.range = range;
   }
 
-  public boolean usesLocalIterators() {
+  public Boolean usesLocalIterators() {
     return localIterators;
   }
 
-  public void setUsesLocalIterators(boolean localIterators) {
+  public void setUsesLocalIterators(Boolean localIterators) {
     this.localIterators = localIterators;
   }
 
@@ -399,4 +455,20 @@ public class RangeInputSplit extends InputSplit implements Writable {
   public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
     this.fetchedColumns = fetchedColumns;
   }
+
+  public List<AccumuloIterator> getIterators() {
+    return iterators;
+  }
+
+  public void setIterators(List<AccumuloIterator> iterators) {
+    this.iterators = iterators;
+  }
+
+  public List<AccumuloIteratorOption> getOptions() {
+    return options;
+  }
+
+  public void setOptions(List<AccumuloIteratorOption> options) {
+    this.options = options;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a9644f5b/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
----------------------------------------------------------------------
diff --git a/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java b/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
index c31c738..af12302 100644
--- a/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
+++ b/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
@@ -30,15 +30,13 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit;
+import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
 import org.apache.accumulo.core.client.mock.MockInstance;
 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.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.examples.simple.filedata.ChunkInputFormat;
-import org.apache.accumulo.examples.simple.filedata.ChunkInputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobID;