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/24 00:52:09 UTC

[21/30] Merge branch '1.5.1-SNAPSHOT' into 1.6.1-SNAPSHOT

http://git-wip-us.apache.org/repos/asf/accumulo/blob/aec43807/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index 0000000,8ff3f0e..59cc8d8
mode 000000,100644..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@@ -1,0 -1,428 +1,486 @@@
+ /*
+  * 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.nio.charset.Charset;
+ import java.util.ArrayList;
++import java.util.Collection;
++import java.util.HashSet;
+ import java.util.List;
+ import java.util.Set;
+ 
 -import org.apache.accumulo.core.client.AccumuloSecurityException;
++import org.apache.accumulo.core.Constants;
++import org.apache.accumulo.core.client.ClientConfiguration;
+ import org.apache.accumulo.core.client.Instance;
+ import org.apache.accumulo.core.client.IteratorSetting;
+ import org.apache.accumulo.core.client.ZooKeeperInstance;
++import org.apache.accumulo.core.client.mapreduce.lib.util.ConfiguratorBase.TokenSource;
+ import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
+ import org.apache.accumulo.core.client.mock.MockInstance;
+ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
++import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
+ 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.security.CredentialHelper;
+ import org.apache.accumulo.core.util.Pair;
+ import org.apache.commons.codec.binary.Base64;
+ import org.apache.hadoop.io.Text;
+ import org.apache.hadoop.io.Writable;
+ import org.apache.hadoop.mapreduce.InputSplit;
 -import org.apache.hadoop.util.StringUtils;
+ import org.apache.log4j.Level;
+ 
+ /**
+  * 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, principal;
++  private String tableId, tableName, instanceName, zooKeepers, principal;
++  private TokenSource tokenSource;
++  private String tokenFile;
+   private AuthenticationToken token;
+   private Boolean offline, mockInstance, isolatedScan, localIterators;
+   private Authorizations auths;
+   private Set<Pair<Text,Text>> fetchedColumns;
+   private List<IteratorSetting> iterators;
+   private Level level;
+ 
+   public RangeInputSplit() {
+     range = new Range();
+     locations = new String[0];
++    tableName = "";
++    tableId = "";
+   }
+ 
 -  public RangeInputSplit(Range range, String[] locations) {
++  public RangeInputSplit(RangeInputSplit split) throws IOException {
++    this.setRange(split.getRange());
++    this.setLocations(split.getLocations());
++    this.setTableName(split.getTableName());
++    this.setTableId(split.getTableId());
++  }
++
++  protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
+     this.range = range;
+     this.locations = locations;
++    this.tableName = table;
++    this.tableId = tableId;
+   }
+ 
+   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;
+   }
+ 
+   /**
+    * 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);
++    tableName = in.readUTF();
++    tableId = in.readUTF();
+     int numLocs = in.readInt();
+     locations = new String[numLocs];
+     for (int i = 0; i < numLocs; ++i)
+       locations[i] = in.readUTF();
+     
+     if (in.readBoolean()) {
+       isolatedScan = in.readBoolean();
+     }
+     
+     if (in.readBoolean()) {
+       offline = in.readBoolean();
+     }
+     
+     if (in.readBoolean()) {
+       localIterators = in.readBoolean();
+     }
+     
+     if (in.readBoolean()) {
+       mockInstance = in.readBoolean();
+     }
+     
+     if (in.readBoolean()) {
+       int numColumns = in.readInt();
+       List<String> columns = new ArrayList<String>(numColumns);
+       for (int i = 0; i < numColumns; i++) {
+         columns.add(in.readUTF());
+       }
+       
+       fetchedColumns = InputConfigurator.deserializeFetchedColumns(columns);
+     }
+     
+     if (in.readBoolean()) {
+       String strAuths = in.readUTF();
 -      auths = new Authorizations(strAuths.getBytes(Charset.forName("UTF-8")));
++      auths = new Authorizations(strAuths.getBytes(Constants.UTF8));
+     }
+     
+     if (in.readBoolean()) {
+       principal = in.readUTF();
+     }
+     
+     if (in.readBoolean()) {
 -      String tokenClass = in.readUTF();
 -      byte[] base64TokenBytes = in.readUTF().getBytes(Charset.forName("UTF-8"));
 -      byte[] tokenBytes = Base64.decodeBase64(base64TokenBytes);
 -      
 -      try {
 -        token = CredentialHelper.extractToken(tokenClass, tokenBytes);
 -      } catch (AccumuloSecurityException e) {
 -        throw new IOException(e);
++      int ordinal = in.readInt();
++      this.tokenSource = TokenSource.values()[ordinal];
++
++      switch (this.tokenSource) {
++        case INLINE:
++          String tokenClass = in.readUTF();
++          byte[] base64TokenBytes = in.readUTF().getBytes(Constants.UTF8);
++          byte[] tokenBytes = Base64.decodeBase64(base64TokenBytes);
++          
++          this.token = AuthenticationTokenSerializer.deserialize(tokenClass, tokenBytes);
++          break;
++          
++        case FILE:
++          this.tokenFile = in.readUTF();
++          
++          break;
++        default:
++          throw new IOException("Cannot parse unknown TokenSource ordinal");      
+       }
+     }
+     
+     if (in.readBoolean()) {
+       instanceName = in.readUTF();
+     }
+     
+     if (in.readBoolean()) {
+       zooKeepers = in.readUTF();
+     }
+     
+     if (in.readBoolean()) {
+       level = Level.toLevel(in.readInt());
+     }
+   }
+ 
+   public void write(DataOutput out) throws IOException {
+     range.write(out);
++    out.writeUTF(tableName);
++    out.writeUTF(tableId);
+     out.writeInt(locations.length);
+     for (int i = 0; i < locations.length; ++i)
+       out.writeUTF(locations[i]);
+     
+     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.writeBoolean(null != fetchedColumns);
+     if (null != fetchedColumns) {
+       String[] cols = InputConfigurator.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 != principal);
+     if (null != principal) {
+       out.writeUTF(principal);
+     }
+     
 -    out.writeBoolean(null != token);
 -    if (null != token) {
 -      out.writeUTF(token.getClass().getCanonicalName());
 -      try {
 -        out.writeUTF(CredentialHelper.tokenAsBase64(token));
 -      } catch (AccumuloSecurityException e) {
 -        throw new IOException(e);
++    out.writeBoolean(null != tokenSource);
++    if (null != tokenSource) {
++      out.writeInt(tokenSource.ordinal());
++      
++      if (null != token && null != tokenFile) {
++        throw new IOException("Cannot use both inline AuthenticationToken and file-based AuthenticationToken");
++      } else if (null != token) {
++        out.writeUTF(token.getClass().getCanonicalName());
++        out.writeUTF(Base64.encodeBase64String(AuthenticationTokenSerializer.serialize(token)));
++      } else {
++        out.writeUTF(tokenFile);
+       }
+     }
+     
+     out.writeBoolean(null != instanceName);
+     if (null != instanceName) {
+       out.writeUTF(instanceName);
+     }
+     
+     out.writeBoolean(null != zooKeepers);
+     if (null != zooKeepers) {
+       out.writeUTF(zooKeepers);
+     }
+     
+     out.writeBoolean(null != level);
+     if (null != level) {
+       out.writeInt(level.toInt());
+     }
+   }
+ 
+   @Override
+   public String toString() {
+     StringBuilder sb = new StringBuilder(256);
+     sb.append("Range: ").append(range);
+     sb.append(" Locations: ").append(locations);
 -    sb.append(" Table: ").append(table);
++    sb.append(" Table: ").append(tableName);
++    sb.append(" TableID: ").append(tableId);
+     sb.append(" InstanceName: ").append(instanceName);
+     sb.append(" zooKeepers: ").append(zooKeepers);
+     sb.append(" principal: ").append(principal);
++    sb.append(" tokenSource: ").append(tokenSource);
+     sb.append(" authenticationToken: ").append(token);
++    sb.append(" authenticationTokenFile: ").append(tokenFile);
+     sb.append(" Authorizations: ").append(auths);
+     sb.append(" offlineScan: ").append(offline);
+     sb.append(" mockInstance: ").append(mockInstance);
+     sb.append(" isolatedScan: ").append(isolatedScan);
+     sb.append(" localIterators: ").append(localIterators);
+     sb.append(" fetchColumns: ").append(fetchedColumns);
+     sb.append(" iterators: ").append(iterators);
+     sb.append(" logLevel: ").append(level);
+     return sb.toString();
+   }
+ 
 -  public String getTable() {
 -    return table;
++  public String getTableName() {
++    return tableName;
+   }
+ 
 -  public void setTable(String table) {
 -    this.table = table;
++  public void setTableName(String table) {
++    this.tableName = table;
++  }
++
++  public void setTableId(String tableId) {
++    this.tableId = tableId;
++  }
++
++  public String getTableId() {
++    return tableId;
+   }
+   
+   public Instance getInstance() {
+     if (null == instanceName) {
+       return null;
+     }
+     
+     if (isMockInstance()) {  
+       return new MockInstance(getInstanceName());
+     }
+     
+     if (null == zooKeepers) {
+       return null;
+     }
+     
 -    return new ZooKeeperInstance(getInstanceName(), getZooKeepers());
++    return new ZooKeeperInstance(ClientConfiguration.loadDefault().withInstance(getInstanceName()).withZkHosts(getZooKeepers()));
+   }
+ 
+   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 getPrincipal() {
+     return principal;
+   }
+ 
+   public void setPrincipal(String principal) {
+     this.principal = principal;
+   }
+   
+   public AuthenticationToken getToken() {
+     return token;
+   }
+   
+   public void setToken(AuthenticationToken token) {
 -    this.token = token;;
++    this.tokenSource = TokenSource.INLINE;
++    this.token = token;
++  }
++  
++  public void setToken(String tokenFile) {
++    this.tokenSource = TokenSource.FILE;
++    this.tokenFile = tokenFile;
+   }
+ 
+   public Boolean isOffline() {
+     return offline;
+   }
+ 
+   public void setOffline(Boolean offline) {
+     this.offline = offline;
+   }
+ 
+   public void setLocations(String[] locations) {
+     this.locations = locations;
+   }
+ 
+   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 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(Collection<Pair<Text,Text>> fetchedColumns) {
++    this.fetchedColumns = new HashSet<Pair<Text,Text>>();
++    for (Pair<Text,Text> columns : fetchedColumns) {
++      this.fetchedColumns.add(columns);
++    }
++  }
+ 
+   public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
+     this.fetchedColumns = fetchedColumns;
+   }
+ 
+   public List<IteratorSetting> getIterators() {
+     return iterators;
+   }
+ 
+   public void setIterators(List<IteratorSetting> iterators) {
+     this.iterators = iterators;
+   }
+ 
+   public Level getLogLevel() {
+     return level;
+   }
+   
+   public void setLogLevel(Level level) {
+     this.level = level;
+   }
+ }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/aec43807/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
index 0fbba98,ab99f56..cf861ce
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
@@@ -50,21 -42,7 +50,21 @@@ public class ConfiguratorBase 
     * @since 1.5.0
     */
    public static enum ConnectorInfo {
 -    IS_CONFIGURED, PRINCIPAL, TOKEN, TOKEN_CLASS
 +    IS_CONFIGURED, PRINCIPAL, TOKEN,
 +  }
 +
-   private static enum TokenSource {
++  public static enum TokenSource {
 +    FILE, INLINE;
 +
 +    private String prefix;
 +
 +    private TokenSource() {
 +      prefix = name().toLowerCase() + ":";
 +    }
 +
 +    public String prefix() {
 +      return prefix;
 +    }
    }
  
    /**
@@@ -72,8 -50,8 +72,8 @@@
     * 
     * @since 1.5.0
     */
--  protected static enum InstanceOpts {
 -    TYPE, NAME, ZOO_KEEPERS;
++  public static enum InstanceOpts {
 +    TYPE, NAME, ZOO_KEEPERS, CLIENT_CONFIG;
    }
  
    /**
@@@ -81,7 -59,7 +81,7 @@@
     * 
     * @since 1.5.0
     */
--  protected static enum GeneralOpts {
++  public static enum GeneralOpts {
      LOG_LEVEL
    }
  
@@@ -223,47 -153,19 +223,47 @@@
    }
  
    /**
 -   * 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.
 +   * Reads from the token file in distributed cache. Currently, the token file stores data separated by colons e.g. principal:token_class:token
     * 
 -   * @param implementingClass
 -   *          the class whose name will be used as a prefix for the property configuration key
     * @param conf
 -   *          the Hadoop configuration object to configure
 -   * @return the decoded principal's authentication token
 -   * @since 1.5.0
 +   *          the Hadoop context for the configured job
 +   * @return path to the token file as a String
 +   * @since 1.6.0
     * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
     */
-   private static AuthenticationToken getTokenFromFile(Configuration conf, String principal, String tokenFile) {
 -  public static byte[] getToken(Class<?> implementingClass, Configuration conf) {
 -    return Base64.decodeBase64(conf.get(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), "").getBytes(Charset.forName("UTF-8")));
++  public static AuthenticationToken getTokenFromFile(Configuration conf, String principal, String tokenFile) {
 +    FSDataInputStream in = null;
 +    try {
 +      URI[] uris = DistributedCacheHelper.getCacheFiles(conf);
 +      Path path = null;
 +      for (URI u : uris) {
 +        if (u.toString().equals(tokenFile)) {
 +          path = new Path(u);
 +        }
 +      }
 +      if (path == null) {
 +        throw new IllegalArgumentException("Couldn't find password file called \"" + tokenFile + "\" in cache.");
 +      }
 +      FileSystem fs = FileSystem.get(conf);
 +      in = fs.open(path);
 +    } catch (IOException e) {
 +      throw new IllegalArgumentException("Couldn't open password file called \"" + tokenFile + "\".");
 +    }
 +    java.util.Scanner fileScanner = new java.util.Scanner(in);
 +    try {
 +      while (fileScanner.hasNextLine()) {
 +        Credentials creds = Credentials.deserialize(fileScanner.nextLine());
 +        if (principal.equals(creds.getPrincipal())) {
 +          return creds.getToken();
 +        }
 +      }
 +      throw new IllegalArgumentException("Couldn't find token for user \"" + principal + "\" in file \"" + tokenFile + "\"");
 +    } finally {
 +      if (fileScanner != null && fileScanner.ioException() == null)
 +        fileScanner.close();
 +      else if (fileScanner.ioException() != null)
 +        throw new RuntimeException(fileScanner.ioException());
 +    }
    }
  
    /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/aec43807/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
index 11a1619,ff14107..7b17d11
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
@@@ -259,21 -204,27 +259,28 @@@ public class InputConfigurator extends 
     * @since 1.5.0
     */
    public static void fetchColumns(Class<?> implementingClass, Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 -    ArgumentChecker.notNull(columnFamilyColumnQualifierPairs);
 +    notNull(columnFamilyColumnQualifierPairs);
-     ArrayList<String> columnStrings = new ArrayList<String>();
+     String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
+     conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings);
+   }
+ 
+   public static String[] serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 -    ArgumentChecker.notNull(columnFamilyColumnQualifierPairs);
++    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())));
 +      String col = new String(Base64.encodeBase64(TextUtil.getBytes(column.getFirst())), Constants.UTF8);
        if (column.getSecond() != null)
 -        col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())));
 +        col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())), Constants.UTF8);
        columnStrings.add(col);
      }
-     conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings.toArray(new String[0]));
+ 
+     return columnStrings.toArray(new String[0]);
    }
 -  
 +
    /**
     * Gets the columns to be mapped over from this job.
     * 
@@@ -286,10 -237,21 +293,21 @@@
     * @see #fetchColumns(Class, Configuration, Collection)
     */
    public static Set<Pair<Text,Text>> getFetchedColumns(Class<?> implementingClass, Configuration conf) {
 -    ArgumentChecker.notNull(conf);
++    notNull(conf);
+ 
+     return deserializeFetchedColumns(conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.COLUMNS)));
+   }
+ 
+   public static Set<Pair<Text,Text>> deserializeFetchedColumns(Collection<String> serialized) {
      Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
-     for (String col : conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.COLUMNS))) {
+ 
+     if (null == serialized) {
+       return columns;
+     }
+ 
+     for (String col : serialized) {
        int idx = col.indexOf(":");
 -      Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes(Charset.forName("UTF-8"))) : Base64.decodeBase64(col.substring(0, idx).getBytes(Charset.forName("UTF-8"))));
 +      Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes(Constants.UTF8)) : Base64.decodeBase64(col.substring(0, idx).getBytes(Constants.UTF8)));
        Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes()));
        columns.add(new Pair<Text,Text>(cf, cq));
      }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/aec43807/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
index 68f88cb,0000000..2864016
mode 100644,000000..100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
@@@ -1,188 -1,0 +1,188 @@@
 +/*
 + * 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.mapred;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertNull;
 +
 +import java.io.IOException;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.Map;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
 +import org.apache.accumulo.core.client.mock.MockInstance;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.hadoop.conf.Configured;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.mapred.JobClient;
 +import org.apache.hadoop.mapred.JobConf;
 +import org.apache.hadoop.mapred.Mapper;
 +import org.apache.hadoop.mapred.OutputCollector;
 +import org.apache.hadoop.mapred.Reporter;
 +import org.apache.hadoop.mapred.lib.NullOutputFormat;
 +import org.apache.hadoop.util.Tool;
 +import org.apache.hadoop.util.ToolRunner;
 +import org.junit.Test;
 +
 +public class AccumuloMultiTableInputFormatTest {
 +
 +  private static final String PREFIX = AccumuloMultiTableInputFormatTest.class.getSimpleName();
 +  private static final String INSTANCE_NAME = PREFIX + "_mapred_instance";
 +  private static final String TEST_TABLE_1 = PREFIX + "_mapred_table_1";
 +  private static final String TEST_TABLE_2 = PREFIX + "_mapred_table_2";
 +
 +  private static AssertionError e1 = null;
 +  private static AssertionError e2 = null;
 +
 +  private static class MRTester extends Configured implements Tool {
 +    private static class TestMapper implements Mapper<Key,Value,Key,Value> {
 +      Key key = null;
 +      int count = 0;
 +
 +      @Override
 +      public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
 +        try {
-           String tableName = ((InputFormatBase.RangeInputSplit) reporter.getInputSplit()).getTableName();
++          String tableName = ((RangeInputSplit) reporter.getInputSplit()).getTableName();
 +          if (key != null)
 +            assertEquals(key.getRow().toString(), new String(v.get()));
 +          assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
 +          assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
 +        } catch (AssertionError e) {
 +          e1 = e;
 +        }
 +        key = new Key(k);
 +        count++;
 +      }
 +
 +      @Override
 +      public void configure(JobConf job) {}
 +
 +      @Override
 +      public void close() throws IOException {
 +        try {
 +          assertEquals(100, count);
 +        } catch (AssertionError e) {
 +          e2 = e;
 +        }
 +      }
 +
 +    }
 +
 +    @Override
 +    public int run(String[] args) throws Exception {
 +
 +      if (args.length != 4) {
 +        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table1> <table2>");
 +      }
 +
 +      String user = args[0];
 +      String pass = args[1];
 +      String table1 = args[2];
 +      String table2 = args[3];
 +
 +      JobConf job = new JobConf(getConf());
 +      job.setJarByClass(this.getClass());
 +
 +      job.setInputFormat(AccumuloInputFormat.class);
 +
 +      AccumuloMultiTableInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
 +      AccumuloMultiTableInputFormat.setMockInstance(job, INSTANCE_NAME);
 +
 +      InputTableConfig tableConfig1 = new InputTableConfig();
 +      InputTableConfig tableConfig2 = new InputTableConfig();
 +
 +      Map<String,InputTableConfig> configMap = new HashMap<String,InputTableConfig>();
 +      configMap.put(table1, tableConfig1);
 +      configMap.put(table2, tableConfig2);
 +
 +      AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
 +
 +      job.setMapperClass(TestMapper.class);
 +      job.setMapOutputKeyClass(Key.class);
 +      job.setMapOutputValueClass(Value.class);
 +      job.setOutputFormat(NullOutputFormat.class);
 +
 +      job.setNumReduceTasks(0);
 +
 +      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
 +    }
 +
 +    public static void main(String[] args) throws Exception {
 +      assertEquals(0, ToolRunner.run(CachedConfiguration.getInstance(), new MRTester(), args));
 +    }
 +  }
 +
 +  @Test
 +  public void testMap() throws Exception {
 +    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
 +    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
 +    c.tableOperations().create(TEST_TABLE_1);
 +    c.tableOperations().create(TEST_TABLE_2);
 +    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
 +    BatchWriter bw2 = c.createBatchWriter(TEST_TABLE_2, new BatchWriterConfig());
 +    for (int i = 0; i < 100; i++) {
 +      Mutation t1m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_1, i + 1)));
 +      t1m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_1, i).getBytes()));
 +      bw.addMutation(t1m);
 +      Mutation t2m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_2, i + 1)));
 +      t2m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_2, i).getBytes()));
 +      bw2.addMutation(t2m);
 +    }
 +    bw.close();
 +    bw2.close();
 +
 +    MRTester.main(new String[] {"root", "", TEST_TABLE_1, TEST_TABLE_2});
 +    assertNull(e1);
 +    assertNull(e2);
 +  }
 +
 +  /**
 +   * Verify {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects get correctly serialized in the JobContext.
 +   */
 +  @Test
 +  public void testTableQueryConfigSerialization() throws IOException {
 +
 +    JobConf job = new JobConf();
 +
 +    InputTableConfig table1 = new InputTableConfig().setRanges(Collections.singletonList(new Range("a", "b")))
 +        .fetchColumns(Collections.singleton(new Pair<Text,Text>(new Text("CF1"), new Text("CQ1"))))
 +        .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
 +
 +    InputTableConfig table2 = new InputTableConfig().setRanges(Collections.singletonList(new Range("a", "b")))
 +        .fetchColumns(Collections.singleton(new Pair<Text,Text>(new Text("CF1"), new Text("CQ1"))))
 +        .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
 +
 +    Map<String,InputTableConfig> configMap = new HashMap<String,InputTableConfig>();
 +    configMap.put(TEST_TABLE_1, table1);
 +    configMap.put(TEST_TABLE_2, table2);
 +    AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
 +
-     assertEquals(table1, AccumuloMultiTableInputFormat.getInputTableConfigs(job, TEST_TABLE_1));
-     assertEquals(table2, AccumuloMultiTableInputFormat.getInputTableConfigs(job, TEST_TABLE_2));
++    assertEquals(table1, AccumuloMultiTableInputFormat.getInputTableConfig(job, TEST_TABLE_1));
++    assertEquals(table2, AccumuloMultiTableInputFormat.getInputTableConfig(job, TEST_TABLE_2));
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/aec43807/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
index e342705,f9ccdf1..6a97d67
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
@@@ -23,6 -23,8 +23,9 @@@ import static org.junit.Assert.assertTr
  import java.io.ByteArrayOutputStream;
  import java.io.DataOutputStream;
  import java.io.IOException;
++import java.net.URI;
+ import java.util.Collection;
+ import java.util.Collections;
  import java.util.List;
  
  import org.apache.accumulo.core.client.BatchWriter;
@@@ -36,16 -39,24 +40,37 @@@ import org.apache.accumulo.core.data.Mu
  import org.apache.accumulo.core.data.Value;
  import org.apache.accumulo.core.iterators.user.RegExFilter;
  import org.apache.accumulo.core.iterators.user.WholeRowIterator;
+ import org.apache.accumulo.core.security.Authorizations;
  import org.apache.accumulo.core.util.CachedConfiguration;
+ import org.apache.accumulo.core.util.Pair;
  import org.apache.commons.codec.binary.Base64;
  import org.apache.hadoop.conf.Configuration;
++import org.apache.hadoop.conf.Configuration.IntegerRanges;
  import org.apache.hadoop.conf.Configured;
++import org.apache.hadoop.fs.Path;
++import org.apache.hadoop.io.RawComparator;
  import org.apache.hadoop.io.Text;
++import org.apache.hadoop.mapreduce.Counter;
++import org.apache.hadoop.mapreduce.InputFormat;
+ import org.apache.hadoop.mapreduce.InputSplit;
  import org.apache.hadoop.mapreduce.Job;
++import org.apache.hadoop.mapreduce.JobID;
  import org.apache.hadoop.mapreduce.Mapper;
++import org.apache.hadoop.mapreduce.Mapper.Context;
++import org.apache.hadoop.mapreduce.OutputCommitter;
++import org.apache.hadoop.mapreduce.OutputFormat;
++import org.apache.hadoop.mapreduce.Partitioner;
+ import org.apache.hadoop.mapreduce.RecordReader;
++import org.apache.hadoop.mapreduce.Reducer;
+ import org.apache.hadoop.mapreduce.TaskAttemptContext;
+ import org.apache.hadoop.mapreduce.TaskAttemptID;
  import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
++import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
++import org.apache.hadoop.security.Credentials;
  import org.apache.hadoop.util.Tool;
  import org.apache.hadoop.util.ToolRunner;
+ import org.apache.log4j.Level;
+ import org.junit.Assert;
  import org.junit.Test;
  
  public class AccumuloInputFormatTest {
@@@ -284,4 -329,152 +309,410 @@@
      assertNull(e1);
      assertNull(e2);
    }
+   
+   @SuppressWarnings("deprecation")
+   @Test
+   public void testCorrectRangeInputSplits() throws Exception {
+     Job job = new Job(new Configuration(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
+ 
+     String username = "user", table = "table", instance = "instance";
+     PasswordToken password = new PasswordToken("password");
+     Authorizations auths = new Authorizations("foo");
+     Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
+     boolean isolated = true, localIters = true;
+     Level level = Level.WARN;
+ 
+     Instance inst = new MockInstance(instance);
+     Connector connector = inst.getConnector(username, password);
+     connector.tableOperations().create(table);
+ 
+     AccumuloInputFormat.setConnectorInfo(job, username, password);
+     AccumuloInputFormat.setInputTableName(job, table);
+     AccumuloInputFormat.setScanAuthorizations(job, auths);
+     AccumuloInputFormat.setMockInstance(job, instance);
+     AccumuloInputFormat.setScanIsolation(job, isolated);
+     AccumuloInputFormat.setLocalIterators(job, localIters);
+     AccumuloInputFormat.fetchColumns(job, fetchColumns);
+     AccumuloInputFormat.setLogLevel(job, level);
+     
+     AccumuloInputFormat aif = new AccumuloInputFormat();
+     
+     List<InputSplit> splits = aif.getSplits(job);
+     
+     Assert.assertEquals(1, splits.size());
+     
+     InputSplit split = splits.get(0);
+     
+     Assert.assertEquals(RangeInputSplit.class, split.getClass());
+     
+     RangeInputSplit risplit = (RangeInputSplit) split;
+     
+     Assert.assertEquals(username, risplit.getPrincipal());
 -    Assert.assertEquals(table, risplit.getTable());
++    Assert.assertEquals(table, risplit.getTableName());
+     Assert.assertEquals(password, risplit.getToken());
+     Assert.assertEquals(auths, risplit.getAuths());
+     Assert.assertEquals(instance, risplit.getInstanceName());
+     Assert.assertEquals(isolated, risplit.isIsolatedScan());
+     Assert.assertEquals(localIters, risplit.usesLocalIterators());
+     Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());
+     Assert.assertEquals(level, risplit.getLogLevel());
+   }
+   
+   static class TestMapper extends Mapper<Key,Value,Key,Value> {
+     Key key = null;
+     int count = 0;
+ 
+     @Override
+     protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
+       if (key != null)
+         assertEquals(key.getRow().toString(), new String(v.get()));
+       assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
+       assertEquals(new String(v.get()), String.format("%09x", count));
+       key = new Key(k);
+       count++;
+     }
+   }
+ 
+   @Test
+   public void testPartialInputSplitDelegationToConfiguration() throws Exception {
+     String user = "testPartialInputSplitUser";
+     PasswordToken password = new PasswordToken("");
+     
+     MockInstance mockInstance = new MockInstance("testPartialInputSplitDelegationToConfiguration");
+     Connector c = mockInstance.getConnector(user, password);
+     c.tableOperations().create("testtable");
+     BatchWriter bw = c.createBatchWriter("testtable", new BatchWriterConfig());
+     for (int i = 0; i < 100; i++) {
+       Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+       m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+       bw.addMutation(m);
+     }
+     bw.close();
+ 
 -    Job job = new Job(new Configuration());
++    Job job = Job.getInstance();
+     job.setInputFormatClass(AccumuloInputFormat.class);
+     job.setMapperClass(TestMapper.class);
+     job.setNumReduceTasks(0);
+     AccumuloInputFormat.setConnectorInfo(job, user, password);
+     AccumuloInputFormat.setInputTableName(job, "testtable");
+     AccumuloInputFormat.setScanAuthorizations(job, new Authorizations());
+     AccumuloInputFormat.setMockInstance(job, "testPartialInputSplitDelegationToConfiguration");
+ 
+     AccumuloInputFormat input = new AccumuloInputFormat();
+     List<InputSplit> splits = input.getSplits(job);
+     assertEquals(splits.size(), 1);
+ 
+     TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
+     
+     RangeInputSplit emptySplit = new RangeInputSplit();
++    emptySplit.setTableName("testtable");
++    emptySplit.setTableId(c.tableOperations().tableIdMap().get("testtable"));
+     
+     // Using an empty split should fall back to the information in the Job's Configuration
+     TaskAttemptID id = new TaskAttemptID();
 -    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
++    TaskAttemptContext attempt = new TaskAttemptContextImpl(job.getConfiguration(), id);
+     RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
 -    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
 -    reader.initialize(emptySplit, context);
 -    mapper.run(context);
++
++    reader.initialize(emptySplit, attempt);
++    Context nullContext = mapper.new Context() {
++
++      @Override
++      public InputSplit getInputSplit() {
++        return null;
++      }
++
++      @Override
++      public boolean nextKeyValue() throws IOException, InterruptedException {
++        return false;
++      }
++
++      @Override
++      public Key getCurrentKey() throws IOException, InterruptedException {
++        return null;
++      }
++
++      @Override
++      public Value getCurrentValue() throws IOException, InterruptedException {
++        return null;
++      }
++
++      @Override
++      public void write(Key key, Value value) throws IOException, InterruptedException {
++        
++      }
++
++      @Override
++      public OutputCommitter getOutputCommitter() {
++        return null;
++      }
++
++      @Override
++      public TaskAttemptID getTaskAttemptID() {
++        return null;
++      }
++
++      @Override
++      public void setStatus(String msg) {
++        
++      }
++
++      @Override
++      public String getStatus() {
++        return null;
++      }
++
++      @Override
++      public float getProgress() {
++        return 0;
++      }
++
++      @Override
++      public Counter getCounter(Enum<?> counterName) {
++        return null;
++      }
++
++      @Override
++      public Counter getCounter(String groupName, String counterName) {
++        return null;
++      }
++
++      @Override
++      public Configuration getConfiguration() {
++        return null;
++      }
++
++      @Override
++      public Credentials getCredentials() {
++        return null;
++      }
++
++      @Override
++      public JobID getJobID() {
++        return null;
++      }
++
++      @Override
++      public int getNumReduceTasks() {
++        return 0;
++      }
++
++      @Override
++      public Path getWorkingDirectory() throws IOException {
++        return null;
++      }
++
++      @Override
++      public Class<?> getOutputKeyClass() {
++        return null;
++      }
++
++      @Override
++      public Class<?> getOutputValueClass() {
++        return null;
++      }
++
++      @Override
++      public Class<?> getMapOutputKeyClass() {
++        return null;
++      }
++
++      @Override
++      public Class<?> getMapOutputValueClass() {
++        return null;
++      }
++
++      @Override
++      public String getJobName() {
++        return null;
++      }
++
++      @Override
++      public Class<? extends InputFormat<?,?>> getInputFormatClass() throws ClassNotFoundException {
++        return null;
++      }
++
++      @Override
++      public Class<? extends Mapper<?,?,?,?>> getMapperClass() throws ClassNotFoundException {
++        return null;
++      }
++
++      @Override
++      public Class<? extends Reducer<?,?,?,?>> getCombinerClass() throws ClassNotFoundException {
++        return null;
++      }
++
++      @Override
++      public Class<? extends Reducer<?,?,?,?>> getReducerClass() throws ClassNotFoundException {
++        return null;
++      }
++
++      @Override
++      public Class<? extends OutputFormat<?,?>> getOutputFormatClass() throws ClassNotFoundException {
++        return null;
++      }
++
++      @Override
++      public Class<? extends Partitioner<?,?>> getPartitionerClass() throws ClassNotFoundException {
++        return null;
++      }
++
++      @Override
++      public RawComparator<?> getSortComparator() {
++        return null;
++      }
++
++      @Override
++      public String getJar() {
++        return null;
++      }
++
++      @Override
++      public RawComparator<?> getGroupingComparator() {
++        return null;
++      }
++
++      @Override
++      public boolean getJobSetupCleanupNeeded() {
++        return false;
++      }
++
++      @Override
++      public boolean getTaskCleanupNeeded() {
++        return false;
++      }
++
++      @Override
++      public boolean getProfileEnabled() {
++        return false;
++      }
++
++      @Override
++      public String getProfileParams() {
++        return null;
++      }
++
++      @Override
++      public IntegerRanges getProfileTaskRange(boolean isMap) {
++        return null;
++      }
++
++      @Override
++      public String getUser() {
++        return null;
++      }
++
++      @Override
++      public boolean getSymlink() {
++        return false;
++      }
++
++      @Override
++      public Path[] getArchiveClassPaths() {
++        return null;
++      }
++
++      @Override
++      public URI[] getCacheArchives() throws IOException {
++        return null;
++      }
++
++      @Override
++      public URI[] getCacheFiles() throws IOException {
++        return null;
++      }
++
++      @Override
++      public Path[] getLocalCacheArchives() throws IOException {
++        return null;
++      }
++
++      @Override
++      public Path[] getLocalCacheFiles() throws IOException {
++        return null;
++      }
++
++      @Override
++      public Path[] getFileClassPaths() {
++        return null;
++      }
++
++      @Override
++      public String[] getArchiveTimestamps() {
++        return null;
++      }
++
++      @Override
++      public String[] getFileTimestamps() {
++        return null;
++      }
++
++      @Override
++      public int getMaxMapAttempts() {
++        return 0;
++      }
++
++      @Override
++      public int getMaxReduceAttempts() {
++        return 0;
++      }
++
++      @Override
++      public void progress() {
++        
++      }
++
++    };
++    
++    while (reader.nextKeyValue()) {
++      Key key = reader.getCurrentKey();
++      Value value = reader.getCurrentValue();
++      
++      mapper.map(key, value, nullContext);
++    }
+   }
+ 
+   @Test(expected = IOException.class)
+   public void testPartialFailedInputSplitDelegationToConfiguration() throws Exception {
+     String user = "testPartialFailedInputSplit";
+     PasswordToken password = new PasswordToken("");
+     
+     MockInstance mockInstance = new MockInstance("testPartialFailedInputSplitDelegationToConfiguration");
+     Connector c = mockInstance.getConnector(user, password);
+     c.tableOperations().create("testtable");
+     BatchWriter bw = c.createBatchWriter("testtable", new BatchWriterConfig());
+     for (int i = 0; i < 100; i++) {
+       Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+       m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+       bw.addMutation(m);
+     }
+     bw.close();
+ 
 -    Job job = new Job(new Configuration());
++    Job job = Job.getInstance();
+     job.setInputFormatClass(AccumuloInputFormat.class);
+     job.setMapperClass(TestMapper.class);
+     job.setNumReduceTasks(0);
+     AccumuloInputFormat.setConnectorInfo(job, user, password);
+     AccumuloInputFormat.setInputTableName(job, "testtable");
+     AccumuloInputFormat.setMockInstance(job, "testPartialFailedInputSplitDelegationToConfiguration");
+ 
+     AccumuloInputFormat input = new AccumuloInputFormat();
+     List<InputSplit> splits = input.getSplits(job);
+     assertEquals(splits.size(), 1);
+ 
+     TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
+     
+     RangeInputSplit emptySplit = new RangeInputSplit();
++    emptySplit.setTableName("testtable");
++    emptySplit.setTableId(c.tableOperations().tableIdMap().get("testtable"));
+     emptySplit.setPrincipal("root");
+     emptySplit.setToken(new PasswordToken("anythingelse"));
+     
+     // Using an empty split should fall back to the information in the Job's Configuration
+     TaskAttemptID id = new TaskAttemptID();
 -    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
++    TaskAttemptContext attempt = new TaskAttemptContextImpl(job.getConfiguration(), id);
+     RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
 -    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
 -    reader.initialize(emptySplit, context);
 -    mapper.run(context);
++    
++    reader.initialize(emptySplit, attempt);
++
+   }
  }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/aec43807/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
index a9bff63,0000000..05fbbb4
mode 100644,000000..100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
@@@ -1,185 -1,0 +1,185 @@@
 +/*
 + * 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 static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertNull;
 +
 +import java.io.IOException;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.Map;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.client.mock.MockInstance;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.hadoop.conf.Configured;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.mapreduce.Job;
 +import org.apache.hadoop.mapreduce.Mapper;
 +import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 +import org.apache.hadoop.util.Tool;
 +import org.apache.hadoop.util.ToolRunner;
 +import org.junit.Test;
 +
 +public class AccumuloMultiTableInputFormatTest {
 +
 +  private static final String PREFIX = AccumuloMultiTableInputFormatTest.class.getSimpleName();
 +  private static final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
 +  private static final String TEST_TABLE_1 = PREFIX + "_mapreduce_table_1";
 +  private static final String TEST_TABLE_2 = PREFIX + "_mapreduce_table_2";
 +
 +  private static AssertionError e1 = null;
 +  private static AssertionError e2 = null;
 +
 +  private static class MRTester extends Configured implements Tool {
 +
 +    private static class TestMapper extends Mapper<Key,Value,Key,Value> {
 +      Key key = null;
 +      int count = 0;
 +
 +      @Override
 +      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
 +        try {
-           String tableName = ((InputFormatBase.RangeInputSplit) context.getInputSplit()).getTableName();
++          String tableName = ((RangeInputSplit) context.getInputSplit()).getTableName();
 +          if (key != null)
 +            assertEquals(key.getRow().toString(), new String(v.get()));
 +          assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
 +          assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
 +        } catch (AssertionError e) {
 +          e1 = e;
 +        }
 +        key = new Key(k);
 +        count++;
 +      }
 +
 +      @Override
 +      protected void cleanup(Context context) throws IOException, InterruptedException {
 +        try {
 +          assertEquals(100, count);
 +        } catch (AssertionError e) {
 +          e2 = e;
 +        }
 +      }
 +    }
 +
 +    @Override
 +    public int run(String[] args) throws Exception {
 +
 +      if (args.length != 4) {
 +        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table1> <table2>");
 +      }
 +
 +      String user = args[0];
 +      String pass = args[1];
 +      String table1 = args[2];
 +      String table2 = args[3];
 +
 +      @SuppressWarnings("deprecation")
 +      Job job = new Job(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
 +      job.setJarByClass(this.getClass());
 +
 +      job.setInputFormatClass(AccumuloMultiTableInputFormat.class);
 +
 +      AccumuloMultiTableInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
 +
 +      InputTableConfig tableConfig1 = new InputTableConfig();
 +      InputTableConfig tableConfig2 = new InputTableConfig();
 +
 +      Map<String,InputTableConfig> configMap = new HashMap<String,InputTableConfig>();
 +      configMap.put(table1, tableConfig1);
 +      configMap.put(table2, tableConfig2);
 +
 +      AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
 +      AccumuloMultiTableInputFormat.setMockInstance(job, INSTANCE_NAME);
 +
 +      job.setMapperClass(TestMapper.class);
 +      job.setMapOutputKeyClass(Key.class);
 +      job.setMapOutputValueClass(Value.class);
 +      job.setOutputFormatClass(NullOutputFormat.class);
 +
 +      job.setNumReduceTasks(0);
 +
 +      job.waitForCompletion(true);
 +
 +      return job.isSuccessful() ? 0 : 1;
 +    }
 +
 +    public static void main(String[] args) throws Exception {
 +      assertEquals(0, ToolRunner.run(CachedConfiguration.getInstance(), new MRTester(), args));
 +    }
 +  }
 +
 +  /**
 +   * Generate incrementing counts and attach table name to the key/value so that order and multi-table data can be verified.
 +   */
 +  @Test
 +  public void testMap() throws Exception {
 +    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
 +    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
 +    c.tableOperations().create(TEST_TABLE_1);
 +    c.tableOperations().create(TEST_TABLE_2);
 +    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
 +    BatchWriter bw2 = c.createBatchWriter(TEST_TABLE_2, new BatchWriterConfig());
 +    for (int i = 0; i < 100; i++) {
 +      Mutation t1m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_1, i + 1)));
 +      t1m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_1, i).getBytes()));
 +      bw.addMutation(t1m);
 +      Mutation t2m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_2, i + 1)));
 +      t2m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_2, i).getBytes()));
 +      bw2.addMutation(t2m);
 +    }
 +    bw.close();
 +    bw2.close();
 +
 +    MRTester.main(new String[] {"root", "", TEST_TABLE_1, TEST_TABLE_2});
 +    assertNull(e1);
 +    assertNull(e2);
 +  }
 +
 +  /**
 +   * Verify {@link InputTableConfig} objects get correctly serialized in the JobContext.
 +   */
 +  @Test
 +  public void testInputTableConfigSerialization() throws IOException {
 +    @SuppressWarnings("deprecation")
 +    Job job = new Job();
 +
 +    InputTableConfig tableConfig = new InputTableConfig().setRanges(Collections.singletonList(new Range("a", "b")))
 +        .fetchColumns(Collections.singleton(new Pair<Text,Text>(new Text("CF1"), new Text("CQ1"))))
 +        .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
 +
 +    Map<String,InputTableConfig> configMap = new HashMap<String,InputTableConfig>();
 +    configMap.put(TEST_TABLE_1, tableConfig);
 +    configMap.put(TEST_TABLE_2, tableConfig);
 +
 +    AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
 +
 +    assertEquals(tableConfig, AccumuloMultiTableInputFormat.getInputTableConfig(job, TEST_TABLE_1));
 +    assertEquals(tableConfig, AccumuloMultiTableInputFormat.getInputTableConfig(job, TEST_TABLE_2));
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/aec43807/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
index 0000000,6da07a5..cb8b773
mode 000000,100644..100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
@@@ -1,0 -1,89 +1,91 @@@
+ 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.client.security.tokens.PasswordToken;
+ 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.apache.log4j.Level;
+ 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"});
++    RangeInputSplit split = new RangeInputSplit("table", "1", 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.getTableName(), newSplit.getTableName());
++    Assert.assertEquals(split.getTableId(), newSplit.getTableId());
+     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"});
++    RangeInputSplit split = new RangeInputSplit("table", "1", 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.setFetchedColumns(fetchedColumns);
+     split.setToken(new PasswordToken("password"));
+     split.setPrincipal("root");
+     split.setInstanceName("instance");
+     split.setMockInstance(true);
+     split.setZooKeepers("localhost");
+     split.setLogLevel(Level.WARN);
+     
+     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.getFetchedColumns(), newSplit.getFetchedColumns());
+     Assert.assertEquals(split.getToken(), newSplit.getToken());
+     Assert.assertEquals(split.getPrincipal(), newSplit.getPrincipal());
+     Assert.assertEquals(split.getInstanceName(), newSplit.getInstanceName());
+     Assert.assertEquals(split.isMockInstance(), newSplit.isMockInstance());
+     Assert.assertEquals(split.getZooKeepers(), newSplit.getZooKeepers());
+     Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel());
+   }
+   
+ }