You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2014/04/09 19:58:02 UTC

[31/64] [abbrv] Merge branch '1.4.6-SNAPSHOT' into 1.5.2-SNAPSHOT

http://git-wip-us.apache.org/repos/asf/accumulo/blob/92613388/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java
index f123289,0000000..d2113fb
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java
@@@ -1,671 -1,0 +1,683 @@@
 +/*
 + * 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.impl;
 +
 +import java.io.IOException;
 +import java.net.InetSocketAddress;
 +import java.security.SecurityPermission;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.LinkedList;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Random;
 +import java.util.Set;
 +import java.util.concurrent.CountDownLatch;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.util.AddressUtil;
 +import org.apache.accumulo.core.util.Daemon;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.TTimeoutTransport;
 +import org.apache.accumulo.core.util.ThriftUtil;
 +import org.apache.log4j.Logger;
 +import org.apache.thrift.transport.TTransport;
 +import org.apache.thrift.transport.TTransportException;
 +
 +public class ThriftTransportPool {
 +  private static SecurityPermission TRANSPORT_POOL_PERMISSION = new SecurityPermission("transportPoolPermission");
 +  
 +  private static final Random random = new Random();
 +  private long killTime = 1000 * 3;
 +  
 +  private Map<ThriftTransportKey,List<CachedConnection>> cache = new HashMap<ThriftTransportKey,List<CachedConnection>>();
 +  private Map<ThriftTransportKey,Long> errorCount = new HashMap<ThriftTransportKey,Long>();
 +  private Map<ThriftTransportKey,Long> errorTime = new HashMap<ThriftTransportKey,Long>();
 +  private Set<ThriftTransportKey> serversWarnedAbout = new HashSet<ThriftTransportKey>();
 +
 +  private CountDownLatch closerExitLatch;
 +  
 +  private static final Logger log = Logger.getLogger(ThriftTransportPool.class);
 +  
 +  private static final Long ERROR_THRESHOLD = 20l;
 +  private static final int STUCK_THRESHOLD = 2 * 60 * 1000;
 +  
 +  private static class CachedConnection {
 +    
 +    public CachedConnection(CachedTTransport t) {
 +      this.transport = t;
 +    }
 +    
 +    void setReserved(boolean reserved) {
 +      this.transport.setReserved(reserved);
 +    }
 +    
 +    boolean isReserved() {
 +      return this.transport.reserved;
 +    }
 +    
 +    CachedTTransport transport;
 +    
 +    long lastReturnTime;
 +  }
 +  
 +  public static class TransportPoolShutdownException extends RuntimeException {
 +    private static final long serialVersionUID = 1L;
 +  }
 +
 +  private static class Closer implements Runnable {
 +    final ThriftTransportPool pool;
 +    private CountDownLatch closerExitLatch;
 +    
 +    public Closer(ThriftTransportPool pool, CountDownLatch closerExitLatch) {
 +      this.pool = pool;
 +      this.closerExitLatch = closerExitLatch;
 +    }
 +    
 +    private void closeConnections() {
 +      while (true) {
 +        
 +        ArrayList<CachedConnection> connectionsToClose = new ArrayList<CachedConnection>();
 +        
 +        synchronized (pool) {
 +          for (List<CachedConnection> ccl : pool.getCache().values()) {
 +            Iterator<CachedConnection> iter = ccl.iterator();
 +            while (iter.hasNext()) {
 +              CachedConnection cachedConnection = iter.next();
 +              
 +              if (!cachedConnection.isReserved() && System.currentTimeMillis() - cachedConnection.lastReturnTime > pool.killTime) {
 +                connectionsToClose.add(cachedConnection);
 +                iter.remove();
 +              }
 +            }
 +          }
 +          
 +          for (List<CachedConnection> ccl : pool.getCache().values()) {
 +            for (CachedConnection cachedConnection : ccl) {
 +              cachedConnection.transport.checkForStuckIO(STUCK_THRESHOLD);
 +            }
 +          }
 +          
 +          Iterator<Entry<ThriftTransportKey,Long>> iter = pool.errorTime.entrySet().iterator();
 +          while (iter.hasNext()) {
 +            Entry<ThriftTransportKey,Long> entry = iter.next();
 +            long delta = System.currentTimeMillis() - entry.getValue();
 +            if (delta >= STUCK_THRESHOLD) {
 +              pool.errorCount.remove(entry.getKey());
 +              iter.remove();
 +            }
 +          }
 +        }
 +        
 +        // close connections outside of sync block
 +        for (CachedConnection cachedConnection : connectionsToClose) {
 +          cachedConnection.transport.close();
 +        }
 +        
 +        try {
 +          Thread.sleep(500);
 +        } catch (InterruptedException e) {
 +          e.printStackTrace();
 +        }
 +      }
 +    }
 +
++    @Override
 +    public void run() {
 +      try {
 +        closeConnections();
 +      } catch (TransportPoolShutdownException e) {
 +      } finally {
 +        closerExitLatch.countDown();
 +      }
 +    }
 +  }
 +  
 +  static class CachedTTransport extends TTransport {
 +    
 +    private ThriftTransportKey cacheKey;
 +    private TTransport wrappedTransport;
 +    private boolean sawError = false;
 +    
 +    private volatile String ioThreadName = null;
 +    private volatile long ioStartTime = 0;
 +    private volatile boolean reserved = false;
 +    
 +    private String stuckThreadName = null;
 +    
 +    int ioCount = 0;
 +    int lastIoCount = -1;
 +    
 +    private void sawError(Exception e) {
 +      sawError = true;
 +    }
 +    
 +    final void setReserved(boolean reserved) {
 +      this.reserved = reserved;
 +      if (reserved) {
 +        ioThreadName = Thread.currentThread().getName();
 +        ioCount = 0;
 +        lastIoCount = -1;
 +      } else {
 +        if ((ioCount & 1) == 1) {
 +          // connection unreserved, but it seems io may still be
 +          // happening
 +          log.warn("Connection returned to thrift connection pool that may still be in use " + ioThreadName + " " + Thread.currentThread().getName(),
 +              new Exception());
 +        }
 +        
 +        ioCount = 0;
 +        lastIoCount = -1;
 +        ioThreadName = null;
 +      }
 +      checkForStuckIO(STUCK_THRESHOLD);
 +    }
 +    
 +    final void checkForStuckIO(long threshold) {
 +      /*
 +       * checking for stuck io needs to be light weight.
 +       * 
 +       * Tried to call System.currentTimeMillis() and Thread.currentThread() before every io operation.... this dramatically slowed things down. So switched to
 +       * incrementing a counter before and after each io operation.
 +       */
 +      
 +      if ((ioCount & 1) == 1) {
 +        // when ioCount is odd, it means I/O is currently happening
 +        if (ioCount == lastIoCount) {
 +          // still doing same I/O operation as last time this
 +          // functions was called
 +          long delta = System.currentTimeMillis() - ioStartTime;
 +          if (delta >= threshold && stuckThreadName == null) {
 +            stuckThreadName = ioThreadName;
 +            log.warn("Thread \"" + ioThreadName + "\" stuck on IO  to " + cacheKey + " for at least " + delta + " ms");
 +          }
 +        } else {
 +          // remember this ioCount and the time we saw it, need to see
 +          // if it changes
 +          lastIoCount = ioCount;
 +          ioStartTime = System.currentTimeMillis();
 +          
 +          if (stuckThreadName != null) {
 +            // doing I/O, but ioCount changed so no longer stuck
 +            log.info("Thread \"" + stuckThreadName + "\" no longer stuck on IO  to " + cacheKey + " sawError = " + sawError);
 +            stuckThreadName = null;
 +          }
 +        }
 +      } else {
 +        // I/O is not currently happening
 +        if (stuckThreadName != null) {
 +          // no longer stuck, and was stuck in the past
 +          log.info("Thread \"" + stuckThreadName + "\" no longer stuck on IO  to " + cacheKey + " sawError = " + sawError);
 +          stuckThreadName = null;
 +        }
 +      }
 +    }
 +    
 +    public CachedTTransport(TTransport transport, ThriftTransportKey cacheKey2) {
 +      this.wrappedTransport = transport;
 +      this.cacheKey = cacheKey2;
 +    }
 +    
++    @Override
 +    public boolean isOpen() {
 +      return wrappedTransport.isOpen();
 +    }
 +    
++    @Override
 +    public void open() throws TTransportException {
 +      try {
 +        ioCount++;
 +        wrappedTransport.open();
 +      } catch (TTransportException tte) {
 +        sawError(tte);
 +        throw tte;
 +      } finally {
 +        ioCount++;
 +      }
 +    }
 +    
++    @Override
 +    public int read(byte[] arg0, int arg1, int arg2) throws TTransportException {
 +      try {
 +        ioCount++;
 +        return wrappedTransport.read(arg0, arg1, arg2);
 +      } catch (TTransportException tte) {
 +        sawError(tte);
 +        throw tte;
 +      } finally {
 +        ioCount++;
 +      }
 +    }
 +    
++    @Override
 +    public int readAll(byte[] arg0, int arg1, int arg2) throws TTransportException {
 +      try {
 +        ioCount++;
 +        return wrappedTransport.readAll(arg0, arg1, arg2);
 +      } catch (TTransportException tte) {
 +        sawError(tte);
 +        throw tte;
 +      } finally {
 +        ioCount++;
 +      }
 +    }
 +    
++    @Override
 +    public void write(byte[] arg0, int arg1, int arg2) throws TTransportException {
 +      try {
 +        ioCount++;
 +        wrappedTransport.write(arg0, arg1, arg2);
 +      } catch (TTransportException tte) {
 +        sawError(tte);
 +        throw tte;
 +      } finally {
 +        ioCount++;
 +      }
 +    }
 +    
++    @Override
 +    public void write(byte[] arg0) throws TTransportException {
 +      try {
 +        ioCount++;
 +        wrappedTransport.write(arg0);
 +      } catch (TTransportException tte) {
 +        sawError(tte);
 +        throw tte;
 +      } finally {
 +        ioCount++;
 +      }
 +    }
 +    
++    @Override
 +    public void close() {
 +      try {
 +        ioCount++;
 +        wrappedTransport.close();
 +      } finally {
 +        ioCount++;
 +      }
 +      
 +    }
 +    
++    @Override
 +    public void flush() throws TTransportException {
 +      try {
 +        ioCount++;
 +        wrappedTransport.flush();
 +      } catch (TTransportException tte) {
 +        sawError(tte);
 +        throw tte;
 +      } finally {
 +        ioCount++;
 +      }
 +    }
 +    
++    @Override
 +    public boolean peek() {
 +      try {
 +        ioCount++;
 +        return wrappedTransport.peek();
 +      } finally {
 +        ioCount++;
 +      }
 +    }
 +    
++    @Override
 +    public byte[] getBuffer() {
 +      try {
 +        ioCount++;
 +        return wrappedTransport.getBuffer();
 +      } finally {
 +        ioCount++;
 +      }
 +    }
 +    
++    @Override
 +    public int getBufferPosition() {
 +      try {
 +        ioCount++;
 +        return wrappedTransport.getBufferPosition();
 +      } finally {
 +        ioCount++;
 +      }
 +    }
 +    
++    @Override
 +    public int getBytesRemainingInBuffer() {
 +      try {
 +        ioCount++;
 +        return wrappedTransport.getBytesRemainingInBuffer();
 +      } finally {
 +        ioCount++;
 +      }
 +    }
 +    
++    @Override
 +    public void consumeBuffer(int len) {
 +      try {
 +        ioCount++;
 +        wrappedTransport.consumeBuffer(len);
 +      } finally {
 +        ioCount++;
 +      }
 +    }
 +    
 +    public ThriftTransportKey getCacheKey() {
 +      return cacheKey;
 +    }
 +    
 +  }
 +  
 +  private ThriftTransportPool() {}
 +  
 +  public TTransport getTransport(String location, int port) throws TTransportException {
 +    return getTransport(location, port, 0);
 +  }
 +  
 +  public TTransport getTransportWithDefaultTimeout(InetSocketAddress addr, AccumuloConfiguration conf) throws TTransportException {
 +    return getTransport(addr.getAddress().getHostAddress(), addr.getPort(), conf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT));
 +  }
 +  
 +  public TTransport getTransport(InetSocketAddress addr, long timeout) throws TTransportException {
 +    return getTransport(addr.getAddress().getHostAddress(), addr.getPort(), timeout);
 +  }
 +  
 +  public TTransport getTransportWithDefaultTimeout(String location, int port, AccumuloConfiguration conf) throws TTransportException {
 +    return getTransport(location, port, conf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT));
 +  }
 +  
 +  Pair<String,TTransport> getAnyTransport(List<ThriftTransportKey> servers, boolean preferCachedConnection) throws TTransportException {
 +    
 +    servers = new ArrayList<ThriftTransportKey>(servers);
 +    
 +    if (preferCachedConnection) {
 +      HashSet<ThriftTransportKey> serversSet = new HashSet<ThriftTransportKey>(servers);
 +      
 +      synchronized (this) {
 +        
 +        // randomly pick a server from the connection cache
 +        serversSet.retainAll(getCache().keySet());
 +        
 +        if (serversSet.size() > 0) {
 +          ArrayList<ThriftTransportKey> cachedServers = new ArrayList<ThriftTransportKey>(serversSet);
 +          Collections.shuffle(cachedServers, random);
 +          
 +          for (ThriftTransportKey ttk : cachedServers) {
 +            for (CachedConnection cachedConnection : getCache().get(ttk)) {
 +              if (!cachedConnection.isReserved()) {
 +                cachedConnection.setReserved(true);
 +                if (log.isTraceEnabled())
 +                  log.trace("Using existing connection to " + ttk.getLocation() + ":" + ttk.getPort());
 +                return new Pair<String,TTransport>(ttk.getLocation() + ":" + ttk.getPort(), cachedConnection.transport);
 +              }
 +            }
 +          }
 +        }
 +      }
 +    }
 +    
 +    int retryCount = 0;
 +    while (servers.size() > 0 && retryCount < 10) {
 +      int index = random.nextInt(servers.size());
 +      ThriftTransportKey ttk = servers.get(index);
 +      
 +      if (!preferCachedConnection) {
 +        synchronized (this) {
 +          List<CachedConnection> cachedConnList = getCache().get(ttk);
 +          if (cachedConnList != null) {
 +            for (CachedConnection cachedConnection : cachedConnList) {
 +              if (!cachedConnection.isReserved()) {
 +                cachedConnection.setReserved(true);
 +                if (log.isTraceEnabled())
 +                  log.trace("Using existing connection to " + ttk.getLocation() + ":" + ttk.getPort() + " timeout " + ttk.getTimeout());
 +                return new Pair<String,TTransport>(ttk.getLocation() + ":" + ttk.getPort(), cachedConnection.transport);
 +              }
 +            }
 +          }
 +        }
 +      }
 +
 +      try {
 +        return new Pair<String,TTransport>(ttk.getLocation() + ":" + ttk.getPort(), createNewTransport(ttk));
 +      } catch (TTransportException tte) {
 +        log.debug("Failed to connect to " + servers.get(index), tte);
 +        servers.remove(index);
 +        retryCount++;
 +      }
 +    }
 +    
 +    throw new TTransportException("Failed to connect to a server");
 +  }
 +  
 +  public TTransport getTransport(String location, int port, long milliseconds) throws TTransportException {
 +    return getTransport(new ThriftTransportKey(location, port, milliseconds));
 +  }
 +  
 +  private TTransport getTransport(ThriftTransportKey cacheKey) throws TTransportException {
 +    synchronized (this) {
 +      // atomically reserve location if it exist in cache
 +      List<CachedConnection> ccl = getCache().get(cacheKey);
 +      
 +      if (ccl == null) {
 +        ccl = new LinkedList<CachedConnection>();
 +        getCache().put(cacheKey, ccl);
 +      }
 +      
 +      for (CachedConnection cachedConnection : ccl) {
 +        if (!cachedConnection.isReserved()) {
 +          cachedConnection.setReserved(true);
 +          if (log.isTraceEnabled())
 +            log.trace("Using existing connection to " + cacheKey.getLocation() + ":" + cacheKey.getPort());
 +          return cachedConnection.transport;
 +        }
 +      }
 +    }
 +    
 +    return createNewTransport(cacheKey);
 +  }
 +  
 +  private TTransport createNewTransport(ThriftTransportKey cacheKey) throws TTransportException {
 +    TTransport transport;
 +    if (cacheKey.getTimeout() == 0) {
 +      transport = AddressUtil.createTSocket(cacheKey.getLocation(), cacheKey.getPort());
 +    } else {
 +      try {
 +        transport = TTimeoutTransport.create(AddressUtil.parseAddress(cacheKey.getLocation(), cacheKey.getPort()), cacheKey.getTimeout());
 +      } catch (IOException ex) {
 +        throw new TTransportException(ex);
 +      }
 +    }
 +    transport = ThriftUtil.transportFactory().getTransport(transport);
 +    transport.open();
 +    
 +    if (log.isTraceEnabled())
 +      log.trace("Creating new connection to connection to " + cacheKey.getLocation() + ":" + cacheKey.getPort());
 +    
 +    CachedTTransport tsc = new CachedTTransport(transport, cacheKey);
 +    
 +    CachedConnection cc = new CachedConnection(tsc);
 +    cc.setReserved(true);
 +    
 +    try {
 +      synchronized (this) {
 +        List<CachedConnection> ccl = getCache().get(cacheKey);
 +
 +        if (ccl == null) {
 +          ccl = new LinkedList<CachedConnection>();
 +          getCache().put(cacheKey, ccl);
 +        }
 +      
 +        ccl.add(cc);
 +      }
 +    } catch (TransportPoolShutdownException e) {
 +      cc.transport.close();
 +      throw e;
 +    }
 +    return cc.transport;
 +  }
 +  
 +  public void returnTransport(TTransport tsc) {
 +    if (tsc == null) {
 +      return;
 +    }
 +    
 +    boolean existInCache = false;
 +    CachedTTransport ctsc = (CachedTTransport) tsc;
 +    
 +    ArrayList<CachedConnection> closeList = new ArrayList<ThriftTransportPool.CachedConnection>();
 +
 +    synchronized (this) {
 +      List<CachedConnection> ccl = getCache().get(ctsc.getCacheKey());
 +      for (Iterator<CachedConnection> iterator = ccl.iterator(); iterator.hasNext();) {
 +        CachedConnection cachedConnection = iterator.next();
 +        if (cachedConnection.transport == tsc) {
 +          if (ctsc.sawError) {
 +            closeList.add(cachedConnection);
 +            iterator.remove();
 +            
 +            if (log.isTraceEnabled())
 +              log.trace("Returned connection had error " + ctsc.getCacheKey());
 +            
 +            Long ecount = errorCount.get(ctsc.getCacheKey());
 +            if (ecount == null)
 +              ecount = 0l;
 +            ecount++;
 +            errorCount.put(ctsc.getCacheKey(), ecount);
 +            
 +            Long etime = errorTime.get(ctsc.getCacheKey());
 +            if (etime == null) {
 +              errorTime.put(ctsc.getCacheKey(), System.currentTimeMillis());
 +            }
 +            
 +            if (ecount >= ERROR_THRESHOLD && !serversWarnedAbout.contains(ctsc.getCacheKey())) {
 +              log.warn("Server " + ctsc.getCacheKey() + " had " + ecount + " failures in a short time period, will not complain anymore ");
 +              serversWarnedAbout.add(ctsc.getCacheKey());
 +            }
 +            
 +            cachedConnection.setReserved(false);
 +            
 +          } else {
 +            
 +            if (log.isTraceEnabled())
 +              log.trace("Returned connection " + ctsc.getCacheKey() + " ioCount : " + cachedConnection.transport.ioCount);
 +            
 +            cachedConnection.lastReturnTime = System.currentTimeMillis();
 +            cachedConnection.setReserved(false);
 +          }
 +          existInCache = true;
 +          break;
 +        }
 +      }
 +      
 +      // remove all unreserved cached connection when a sever has an error, not just the connection that was returned
 +      if (ctsc.sawError) {
 +        for (Iterator<CachedConnection> iterator = ccl.iterator(); iterator.hasNext();) {
 +          CachedConnection cachedConnection = iterator.next();
 +          if (!cachedConnection.isReserved()) {
 +            closeList.add(cachedConnection);
 +            iterator.remove();
 +          }
 +        }
 +      }
 +    }
 +    
 +    // close outside of sync block
 +    for (CachedConnection cachedConnection : closeList) {
 +      try {
 +        cachedConnection.transport.close();
 +      } catch (Exception e) {
 +        log.debug("Failed to close connection w/ errors", e);
 +      }
 +    }
 +    
 +    if (!existInCache) {
 +      log.warn("Returned tablet server connection to cache that did not come from cache");
 +      // close outside of sync block
 +      tsc.close();
 +    }
 +  }
 +  
 +  /**
 +   * Set the time after which idle connections should be closed
-    * 
-    * @param time
 +   */
 +  public synchronized void setIdleTime(long time) {
 +    this.killTime = time;
 +    log.debug("Set thrift transport pool idle time to " + time);
 +  }
 +
 +  private static ThriftTransportPool instance = new ThriftTransportPool();
 +  private static final AtomicBoolean daemonStarted = new AtomicBoolean(false);
 +  
 +  public static ThriftTransportPool getInstance() {
 +    SecurityManager sm = System.getSecurityManager();
 +    if (sm != null) {
 +      sm.checkPermission(TRANSPORT_POOL_PERMISSION);
 +    }
 +    
 +    if (daemonStarted.compareAndSet(false, true)) {
 +      CountDownLatch closerExitLatch = new CountDownLatch(1);
 +      new Daemon(new Closer(instance, closerExitLatch), "Thrift Connection Pool Checker").start();
 +      instance.setCloserExitLatch(closerExitLatch);
 +    }
 +    return instance;
 +  }
 +  
 +  private synchronized void setCloserExitLatch(CountDownLatch closerExitLatch) {
 +    this.closerExitLatch = closerExitLatch;
 +  }
 +
 +  public void shutdown() {
 +    synchronized (this) {
 +      if (cache == null)
 +        return;
 +
 +      // close any connections in the pool... even ones that are in use
 +      for (List<CachedConnection> ccl : getCache().values()) {
 +        Iterator<CachedConnection> iter = ccl.iterator();
 +        while (iter.hasNext()) {
 +          CachedConnection cc = iter.next();
 +          try {
 +            cc.transport.close();
 +          } catch (Exception e) {
 +            log.debug("Error closing transport during shutdown", e);
 +          }
 +        }
 +      }
 +
 +      // this will render the pool unusable and cause the background thread to exit
 +      this.cache = null;
 +    }
 +
 +    try {
 +      closerExitLatch.await();
 +    } catch (InterruptedException e) {
 +      throw new RuntimeException(e);
 +    }
 +  }
 +
 +  private Map<ThriftTransportKey,List<CachedConnection>> getCache() {
 +    if (cache == null)
 +      throw new TransportPoolShutdownException();
 +    return cache;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/92613388/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
index ee4aca5,0000000..d7be37c
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
@@@ -1,511 -1,0 +1,510 @@@
 +/*
 + * 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 java.io.IOException;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +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.Instance;
 +import org.apache.accumulo.core.client.MultiTableBatchWriter;
 +import org.apache.accumulo.core.client.MutationsRejectedException;
 +import org.apache.accumulo.core.client.TableExistsException;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.ZooKeeperInstance;
 +import org.apache.accumulo.core.client.mapreduce.lib.util.OutputConfigurator;
 +import org.apache.accumulo.core.client.mock.MockInstance;
 +import org.apache.accumulo.core.client.security.SecurityErrorCode;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.data.ColumnUpdate;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.security.ColumnVisibility;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.mapred.JobConf;
 +import org.apache.hadoop.mapred.OutputFormat;
 +import org.apache.hadoop.mapred.RecordWriter;
 +import org.apache.hadoop.mapred.Reporter;
 +import org.apache.hadoop.util.Progressable;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * This class allows MapReduce jobs to use Accumulo as the sink for data. This {@link OutputFormat} accepts keys and values of type {@link Text} (for a table
 + * name) and {@link Mutation} from the Map and Reduce functions.
 + * 
 + * The user must specify the following via static configurator methods:
 + * 
 + * <ul>
 + * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
 + * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(JobConf, String, String)} OR {@link AccumuloOutputFormat#setMockInstance(JobConf, String)}
 + * </ul>
 + * 
 + * Other static methods are optional.
 + */
 +public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
 +  
 +  private static final Class<?> CLASS = AccumuloOutputFormat.class;
 +  protected static final Logger log = Logger.getLogger(CLASS);
 +  
 +  /**
 +   * Sets the connector information needed to communicate with Accumulo in this job.
 +   * 
 +   * <p>
 +   * <b>WARNING:</b> The serialized token 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.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param principal
 +   *          a valid Accumulo user name (user must have Table.CREATE permission if {@link #setCreateTables(JobConf, boolean)} is set to true)
 +   * @param token
 +   *          the user's password
-    * @throws AccumuloSecurityException
 +   * @since 1.5.0
 +   */
 +  public static void setConnectorInfo(JobConf job, String principal, AuthenticationToken token) throws AccumuloSecurityException {
 +    OutputConfigurator.setConnectorInfo(CLASS, job, principal, token);
 +  }
 +  
 +  /**
 +   * Determines if the connector has been configured.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return true if the connector has been configured, false otherwise
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
 +   */
 +  protected static Boolean isConnectorInfoSet(JobConf job) {
 +    return OutputConfigurator.isConnectorInfoSet(CLASS, job);
 +  }
 +  
 +  /**
 +   * Gets the principal from the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the user name
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
 +   */
 +  protected static String getPrincipal(JobConf job) {
 +    return OutputConfigurator.getPrincipal(CLASS, job);
 +  }
 +  
 +  /**
 +   * Gets the serialized token class from the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the user name
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
 +   */
 +  protected static String getTokenClass(JobConf job) {
 +    return OutputConfigurator.getTokenClass(CLASS, job);
 +  }
 +  
 +  /**
 +   * 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.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the decoded user password
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
 +   */
 +  protected static byte[] getToken(JobConf job) {
 +    return OutputConfigurator.getToken(CLASS, job);
 +  }
 +  
 +  /**
 +   * Configures a {@link ZooKeeperInstance} for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param instanceName
 +   *          the Accumulo instance name
 +   * @param zooKeepers
 +   *          a comma-separated list of zookeeper servers
 +   * @since 1.5.0
 +   */
 +  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
 +    OutputConfigurator.setZooKeeperInstance(CLASS, job, instanceName, zooKeepers);
 +  }
 +  
 +  /**
 +   * Configures a {@link MockInstance} for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param instanceName
 +   *          the Accumulo instance name
 +   * @since 1.5.0
 +   */
 +  public static void setMockInstance(JobConf job, String instanceName) {
 +    OutputConfigurator.setMockInstance(CLASS, job, instanceName);
 +  }
 +  
 +  /**
 +   * Initializes an Accumulo {@link Instance} based on the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return an Accumulo instance
 +   * @since 1.5.0
 +   * @see #setZooKeeperInstance(JobConf, String, String)
 +   * @see #setMockInstance(JobConf, String)
 +   */
 +  protected static Instance getInstance(JobConf job) {
 +    return OutputConfigurator.getInstance(CLASS, job);
 +  }
 +  
 +  /**
 +   * Sets the log level for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param level
 +   *          the logging level
 +   * @since 1.5.0
 +   */
 +  public static void setLogLevel(JobConf job, Level level) {
 +    OutputConfigurator.setLogLevel(CLASS, job, level);
 +  }
 +  
 +  /**
 +   * Gets the log level from this configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the log level
 +   * @since 1.5.0
 +   * @see #setLogLevel(JobConf, Level)
 +   */
 +  protected static Level getLogLevel(JobConf job) {
 +    return OutputConfigurator.getLogLevel(CLASS, job);
 +  }
 +  
 +  /**
 +   * Sets the default table name to use if one emits a null in place of a table name for a given mutation. Table names can only be alpha-numeric and
 +   * underscores.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param tableName
 +   *          the table to use when the tablename is null in the write call
 +   * @since 1.5.0
 +   */
 +  public static void setDefaultTableName(JobConf job, String tableName) {
 +    OutputConfigurator.setDefaultTableName(CLASS, job, tableName);
 +  }
 +  
 +  /**
 +   * Gets the default table name from the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the default table name
 +   * @since 1.5.0
 +   * @see #setDefaultTableName(JobConf, String)
 +   */
 +  protected static String getDefaultTableName(JobConf job) {
 +    return OutputConfigurator.getDefaultTableName(CLASS, job);
 +  }
 +  
 +  /**
 +   * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new {@link BatchWriterConfig}, with sensible built-in defaults is
 +   * used. Setting the configuration multiple times overwrites any previous configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param bwConfig
 +   *          the configuration for the {@link BatchWriter}
 +   * @since 1.5.0
 +   */
 +  public static void setBatchWriterOptions(JobConf job, BatchWriterConfig bwConfig) {
 +    OutputConfigurator.setBatchWriterOptions(CLASS, job, bwConfig);
 +  }
 +  
 +  /**
 +   * Gets the {@link BatchWriterConfig} settings.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the configuration object
 +   * @since 1.5.0
 +   * @see #setBatchWriterOptions(JobConf, BatchWriterConfig)
 +   */
 +  protected static BatchWriterConfig getBatchWriterOptions(JobConf job) {
 +    return OutputConfigurator.getBatchWriterOptions(CLASS, job);
 +  }
 +  
 +  /**
 +   * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric and underscores.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.5.0
 +   */
 +  public static void setCreateTables(JobConf job, boolean enableFeature) {
 +    OutputConfigurator.setCreateTables(CLASS, job, enableFeature);
 +  }
 +  
 +  /**
 +   * Determines whether tables are permitted to be created as needed.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return true if the feature is disabled, false otherwise
 +   * @since 1.5.0
 +   * @see #setCreateTables(JobConf, boolean)
 +   */
 +  protected static Boolean canCreateTables(JobConf job) {
 +    return OutputConfigurator.canCreateTables(CLASS, job);
 +  }
 +  
 +  /**
 +   * Sets the directive to use simulation mode for this job. In simulation mode, no output is produced. This is useful for testing.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.5.0
 +   */
 +  public static void setSimulationMode(JobConf job, boolean enableFeature) {
 +    OutputConfigurator.setSimulationMode(CLASS, job, enableFeature);
 +  }
 +  
 +  /**
 +   * Determines whether this feature is enabled.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.5.0
 +   * @see #setSimulationMode(JobConf, boolean)
 +   */
 +  protected static Boolean getSimulationMode(JobConf job) {
 +    return OutputConfigurator.getSimulationMode(CLASS, job);
 +  }
 +  
 +  /**
 +   * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
 +   */
 +  protected static class AccumuloRecordWriter implements RecordWriter<Text,Mutation> {
 +    private MultiTableBatchWriter mtbw = null;
 +    private HashMap<Text,BatchWriter> bws = null;
 +    private Text defaultTableName = null;
 +    
 +    private boolean simulate = false;
 +    private boolean createTables = false;
 +    
 +    private long mutCount = 0;
 +    private long valCount = 0;
 +    
 +    private Connector conn;
 +    
 +    protected AccumuloRecordWriter(JobConf job) throws AccumuloException, AccumuloSecurityException, IOException {
 +      Level l = getLogLevel(job);
 +      if (l != null)
 +        log.setLevel(getLogLevel(job));
 +      this.simulate = getSimulationMode(job);
 +      this.createTables = canCreateTables(job);
 +      
 +      if (simulate)
 +        log.info("Simulating output only. No writes to tables will occur");
 +      
 +      this.bws = new HashMap<Text,BatchWriter>();
 +      
 +      String tname = getDefaultTableName(job);
 +      this.defaultTableName = (tname == null) ? null : new Text(tname);
 +      
 +      if (!simulate) {
 +        this.conn = getInstance(job).getConnector(getPrincipal(job), CredentialHelper.extractToken(getTokenClass(job), getToken(job)));
 +        mtbw = conn.createMultiTableBatchWriter(getBatchWriterOptions(job));
 +      }
 +    }
 +    
 +    /**
 +     * Push a mutation into a table. If table is null, the defaultTable will be used. If canCreateTable is set, the table will be created if it does not exist.
 +     * The table name must only contain alphanumerics and underscore.
 +     */
 +    @Override
 +    public void write(Text table, Mutation mutation) throws IOException {
 +      if (table == null || table.toString().isEmpty())
 +        table = this.defaultTableName;
 +      
 +      if (!simulate && table == null)
 +        throw new IOException("No table or default table specified. Try simulation mode next time");
 +      
 +      ++mutCount;
 +      valCount += mutation.size();
 +      printMutation(table, mutation);
 +      
 +      if (simulate)
 +        return;
 +      
 +      if (!bws.containsKey(table))
 +        try {
 +          addTable(table);
 +        } catch (Exception e) {
 +          e.printStackTrace();
 +          throw new IOException(e);
 +        }
 +      
 +      try {
 +        bws.get(table).addMutation(mutation);
 +      } catch (MutationsRejectedException e) {
 +        throw new IOException(e);
 +      }
 +    }
 +    
 +    public void addTable(Text tableName) throws AccumuloException, AccumuloSecurityException {
 +      if (simulate) {
 +        log.info("Simulating adding table: " + tableName);
 +        return;
 +      }
 +      
 +      log.debug("Adding table: " + tableName);
 +      BatchWriter bw = null;
 +      String table = tableName.toString();
 +      
 +      if (createTables && !conn.tableOperations().exists(table)) {
 +        try {
 +          conn.tableOperations().create(table);
 +        } catch (AccumuloSecurityException e) {
 +          log.error("Accumulo security violation creating " + table, e);
 +          throw e;
 +        } catch (TableExistsException e) {
 +          // Shouldn't happen
 +        }
 +      }
 +      
 +      try {
 +        bw = mtbw.getBatchWriter(table);
 +      } catch (TableNotFoundException e) {
 +        log.error("Accumulo table " + table + " doesn't exist and cannot be created.", e);
 +        throw new AccumuloException(e);
 +      } catch (AccumuloException e) {
 +        throw e;
 +      } catch (AccumuloSecurityException e) {
 +        throw e;
 +      }
 +      
 +      if (bw != null)
 +        bws.put(tableName, bw);
 +    }
 +    
 +    private int printMutation(Text table, Mutation m) {
 +      if (log.isTraceEnabled()) {
 +        log.trace(String.format("Table %s row key: %s", table, hexDump(m.getRow())));
 +        for (ColumnUpdate cu : m.getUpdates()) {
 +          log.trace(String.format("Table %s column: %s:%s", table, hexDump(cu.getColumnFamily()), hexDump(cu.getColumnQualifier())));
 +          log.trace(String.format("Table %s security: %s", table, new ColumnVisibility(cu.getColumnVisibility()).toString()));
 +          log.trace(String.format("Table %s value: %s", table, hexDump(cu.getValue())));
 +        }
 +      }
 +      return m.getUpdates().size();
 +    }
 +    
 +    private String hexDump(byte[] ba) {
 +      StringBuilder sb = new StringBuilder();
 +      for (byte b : ba) {
 +        if ((b > 0x20) && (b < 0x7e))
 +          sb.append((char) b);
 +        else
 +          sb.append(String.format("x%02x", b));
 +      }
 +      return sb.toString();
 +    }
 +    
 +    @Override
 +    public void close(Reporter reporter) throws IOException {
 +      log.debug("mutations written: " + mutCount + ", values written: " + valCount);
 +      if (simulate)
 +        return;
 +      
 +      try {
 +        mtbw.close();
 +      } catch (MutationsRejectedException e) {
 +        if (e.getAuthorizationFailuresMap().size() >= 0) {
 +          HashMap<String,Set<SecurityErrorCode>> tables = new HashMap<String,Set<SecurityErrorCode>>();
 +          for (Entry<KeyExtent,Set<SecurityErrorCode>> ke : e.getAuthorizationFailuresMap().entrySet()) {
 +            Set<SecurityErrorCode> secCodes = tables.get(ke.getKey().getTableId().toString());
 +            if (secCodes == null) {
 +              secCodes = new HashSet<SecurityErrorCode>();
 +              tables.put(ke.getKey().getTableId().toString(), secCodes);
 +            }
 +            secCodes.addAll(ke.getValue());
 +          }
 +          
 +          log.error("Not authorized to write to tables : " + tables);
 +        }
 +        
 +        if (e.getConstraintViolationSummaries().size() > 0) {
 +          log.error("Constraint violations : " + e.getConstraintViolationSummaries().size());
 +        }
 +      }
 +    }
 +  }
 +  
 +  @Override
 +  public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
 +    if (!isConnectorInfoSet(job))
 +      throw new IOException("Connector info has not been set.");
 +    try {
 +      // if the instance isn't configured, it will complain here
 +      Connector c = getInstance(job).getConnector(getPrincipal(job), CredentialHelper.extractToken(getTokenClass(job), getToken(job)));
 +      if (!c.securityOperations().authenticateUser(getPrincipal(job), CredentialHelper.extractToken(getTokenClass(job), getToken(job))))
 +        throw new IOException("Unable to authenticate user");
 +    } catch (AccumuloException e) {
 +      throw new IOException(e);
 +    } catch (AccumuloSecurityException e) {
 +      throw new IOException(e);
 +    }
 +  }
 +  
 +  @Override
 +  public RecordWriter<Text,Mutation> getRecordWriter(FileSystem ignored, JobConf job, String name, Progressable progress) throws IOException {
 +    try {
 +      return new AccumuloRecordWriter(job);
 +    } catch (Exception e) {
 +      throw new IOException(e);
 +    }
 +  }
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/92613388/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
index 16efa89,0000000..bc568e8
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
@@@ -1,925 -1,0 +1,924 @@@
 +/*
 + * 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 java.io.IOException;
 +import java.net.InetAddress;
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.HashMap;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.ClientSideIteratorScanner;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.IsolatedScanner;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.client.RowIterator;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableDeletedException;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.TableOfflineException;
 +import org.apache.accumulo.core.client.ZooKeeperInstance;
 +import org.apache.accumulo.core.client.impl.OfflineScanner;
 +import org.apache.accumulo.core.client.impl.Tables;
 +import org.apache.accumulo.core.client.impl.TabletLocator;
 +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.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.master.state.tables.TableState;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.mapred.InputFormat;
 +import org.apache.hadoop.mapred.InputSplit;
 +import org.apache.hadoop.mapred.JobConf;
 +import org.apache.hadoop.mapred.RecordReader;
 +import org.apache.hadoop.mapred.Reporter;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of K,V pairs.
 + * <p>
 + * Subclasses must implement a {@link #getRecordReader(InputSplit, JobConf, Reporter)} to provide a {@link RecordReader} for K,V.
 + * <p>
 + * A static base class, RecordReaderBase, is provided to retrieve Accumulo {@link Key}/{@link Value} pairs, but one must implement its
 + * {@link RecordReaderBase#next(Object, Object)} to transform them to the desired generic types K,V.
 + * <p>
 + * See {@link AccumuloInputFormat} for an example implementation.
 + */
 +public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
 +
 +  private static final Class<?> CLASS = AccumuloInputFormat.class;
 +  protected static final Logger log = Logger.getLogger(CLASS);
 +
 +  /**
 +   * Sets the connector information needed to communicate with Accumulo in this job.
 +   * 
 +   * <p>
 +   * <b>WARNING:</b> The serialized token 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.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param principal
 +   *          a valid Accumulo user name (user must have Table.CREATE permission)
 +   * @param token
 +   *          the user's password
-    * @throws AccumuloSecurityException
 +   * @since 1.5.0
 +   */
 +  public static void setConnectorInfo(JobConf job, String principal, AuthenticationToken token) throws AccumuloSecurityException {
 +    InputConfigurator.setConnectorInfo(CLASS, job, principal, token);
 +  }
 +
 +  /**
 +   * Determines if the connector has been configured.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return true if the connector has been configured, false otherwise
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
 +   */
 +  protected static Boolean isConnectorInfoSet(JobConf job) {
 +    return InputConfigurator.isConnectorInfoSet(CLASS, job);
 +  }
 +
 +  /**
 +   * Gets the user name from the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the user name
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
 +   */
 +  protected static String getPrincipal(JobConf job) {
 +    return InputConfigurator.getPrincipal(CLASS, job);
 +  }
 +
 +  /**
 +   * Gets the serialized token class from the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the user name
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
 +   */
 +  protected static String getTokenClass(JobConf job) {
 +    return InputConfigurator.getTokenClass(CLASS, job);
 +  }
 +
 +  /**
 +   * 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.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the decoded user password
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
 +   */
 +  protected static byte[] getToken(JobConf job) {
 +    return InputConfigurator.getToken(CLASS, job);
 +  }
 +
 +  /**
 +   * Configures a {@link ZooKeeperInstance} for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param instanceName
 +   *          the Accumulo instance name
 +   * @param zooKeepers
 +   *          a comma-separated list of zookeeper servers
 +   * @since 1.5.0
 +   */
 +  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
 +    InputConfigurator.setZooKeeperInstance(CLASS, job, instanceName, zooKeepers);
 +  }
 +
 +  /**
 +   * Configures a {@link MockInstance} for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param instanceName
 +   *          the Accumulo instance name
 +   * @since 1.5.0
 +   */
 +  public static void setMockInstance(JobConf job, String instanceName) {
 +    InputConfigurator.setMockInstance(CLASS, job, instanceName);
 +  }
 +
 +  /**
 +   * Initializes an Accumulo {@link Instance} based on the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return an Accumulo instance
 +   * @since 1.5.0
 +   * @see #setZooKeeperInstance(JobConf, String, String)
 +   * @see #setMockInstance(JobConf, String)
 +   */
 +  protected static Instance getInstance(JobConf job) {
 +    return InputConfigurator.getInstance(CLASS, job);
 +  }
 +
 +  /**
 +   * Sets the log level for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param level
 +   *          the logging level
 +   * @since 1.5.0
 +   */
 +  public static void setLogLevel(JobConf job, Level level) {
 +    InputConfigurator.setLogLevel(CLASS, job, level);
 +  }
 +
 +  /**
 +   * Gets the log level from this configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the log level
 +   * @since 1.5.0
 +   * @see #setLogLevel(JobConf, Level)
 +   */
 +  protected static Level getLogLevel(JobConf job) {
 +    return InputConfigurator.getLogLevel(CLASS, job);
 +  }
 +
 +  /**
 +   * Sets the name of the input table, over which this job will scan.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param tableName
 +   *          the table to use when the tablename is null in the write call
 +   * @since 1.5.0
 +   */
 +  public static void setInputTableName(JobConf job, String tableName) {
 +    InputConfigurator.setInputTableName(CLASS, job, tableName);
 +  }
 +
 +  /**
 +   * Gets the table name from the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the table name
 +   * @since 1.5.0
 +   * @see #setInputTableName(JobConf, String)
 +   */
 +  protected static String getInputTableName(JobConf job) {
 +    return InputConfigurator.getInputTableName(CLASS, job);
 +  }
 +
 +  /**
 +   * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param auths
 +   *          the user's authorizations
 +   * @since 1.5.0
 +   */
 +  public static void setScanAuthorizations(JobConf job, Authorizations auths) {
 +    InputConfigurator.setScanAuthorizations(CLASS, job, auths);
 +  }
 +
 +  /**
 +   * Gets the authorizations to set for the scans from the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the Accumulo scan authorizations
 +   * @since 1.5.0
 +   * @see #setScanAuthorizations(JobConf, Authorizations)
 +   */
 +  protected static Authorizations getScanAuthorizations(JobConf job) {
 +    return InputConfigurator.getScanAuthorizations(CLASS, job);
 +  }
 +
 +  /**
 +   * Sets the input ranges to scan for this job. If not set, the entire table will be scanned.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param ranges
 +   *          the ranges that will be mapped over
 +   * @since 1.5.0
 +   */
 +  public static void setRanges(JobConf job, Collection<Range> ranges) {
 +    InputConfigurator.setRanges(CLASS, job, ranges);
 +  }
 +
 +  /**
 +   * Gets the ranges to scan over from a job.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the ranges
 +   * @throws IOException
 +   *           if the ranges have been encoded improperly
 +   * @since 1.5.0
 +   * @see #setRanges(JobConf, Collection)
 +   */
 +  protected static List<Range> getRanges(JobConf job) throws IOException {
 +    return InputConfigurator.getRanges(CLASS, job);
 +  }
 +
 +  /**
 +   * Restricts the columns that will be mapped over for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param columnFamilyColumnQualifierPairs
 +   *          a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
 +   *          selected. An empty set is the default and is equivalent to scanning the all columns.
 +   * @since 1.5.0
 +   */
 +  public static void fetchColumns(JobConf job, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 +    InputConfigurator.fetchColumns(CLASS, job, columnFamilyColumnQualifierPairs);
 +  }
 +
 +  /**
 +   * Gets the columns to be mapped over from this job.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return a set of columns
 +   * @since 1.5.0
 +   * @see #fetchColumns(JobConf, Collection)
 +   */
 +  protected static Set<Pair<Text,Text>> getFetchedColumns(JobConf job) {
 +    return InputConfigurator.getFetchedColumns(CLASS, job);
 +  }
 +
 +  /**
 +   * Encode an iterator on the input for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param cfg
 +   *          the configuration of the iterator
 +   * @since 1.5.0
 +   */
 +  public static void addIterator(JobConf job, IteratorSetting cfg) {
 +    InputConfigurator.addIterator(CLASS, job, cfg);
 +  }
 +
 +  /**
 +   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return a list of iterators
 +   * @since 1.5.0
 +   * @see #addIterator(JobConf, IteratorSetting)
 +   */
 +  protected static List<IteratorSetting> getIterators(JobConf job) {
 +    return InputConfigurator.getIterators(CLASS, job);
 +  }
 +
 +  /**
 +   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
 +   * Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
 +   * 
 +   * <p>
 +   * By default, this feature is <b>enabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @see #setRanges(JobConf, Collection)
 +   * @since 1.5.0
 +   */
 +  public static void setAutoAdjustRanges(JobConf job, boolean enableFeature) {
 +    InputConfigurator.setAutoAdjustRanges(CLASS, job, enableFeature);
 +  }
 +
 +  /**
 +   * Determines whether a configuration has auto-adjust ranges enabled.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return false if the feature is disabled, true otherwise
 +   * @since 1.5.0
 +   * @see #setAutoAdjustRanges(JobConf, boolean)
 +   */
 +  protected static boolean getAutoAdjustRanges(JobConf job) {
 +    return InputConfigurator.getAutoAdjustRanges(CLASS, job);
 +  }
 +
 +  /**
 +   * Controls the use of the {@link IsolatedScanner} in this job.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.5.0
 +   */
 +  public static void setScanIsolation(JobConf job, boolean enableFeature) {
 +    InputConfigurator.setScanIsolation(CLASS, job, enableFeature);
 +  }
 +
 +  /**
 +   * Determines whether a configuration has isolation enabled.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.5.0
 +   * @see #setScanIsolation(JobConf, boolean)
 +   */
 +  protected static boolean isIsolated(JobConf job) {
 +    return InputConfigurator.isIsolated(CLASS, job);
 +  }
 +
 +  /**
 +   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack to be constructed within the Map
 +   * task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be available on the classpath for the task.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.5.0
 +   */
 +  public static void setLocalIterators(JobConf job, boolean enableFeature) {
 +    InputConfigurator.setLocalIterators(CLASS, job, enableFeature);
 +  }
 +
 +  /**
 +   * Determines whether a configuration uses local iterators.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.5.0
 +   * @see #setLocalIterators(JobConf, boolean)
 +   */
 +  protected static boolean usesLocalIterators(JobConf job) {
 +    return InputConfigurator.usesLocalIterators(CLASS, job);
 +  }
 +
 +  /**
 +   * <p>
 +   * Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
 +   * table's files. If the table is not offline, then the job will fail. If the table comes online during the map reduce job, it is likely that the job will
 +   * fail.
 +   * 
 +   * <p>
 +   * To use this option, the map reduce user will need access to read the Accumulo directory in HDFS.
 +   * 
 +   * <p>
 +   * Reading the offline table will create the scan time iterator stack in the map process. So any iterators that are configured for the table will need to be
 +   * on the mapper's classpath.
 +   * 
 +   * <p>
 +   * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
 +   * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
 +   * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
 +   * 
 +   * <p>
 +   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
 +   * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.5.0
 +   */
 +  public static void setOfflineTableScan(JobConf job, boolean enableFeature) {
 +    InputConfigurator.setOfflineTableScan(CLASS, job, enableFeature);
 +  }
 +
 +  /**
 +   * Determines whether a configuration has the offline table scan feature enabled.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.5.0
 +   * @see #setOfflineTableScan(JobConf, boolean)
 +   */
 +  protected static boolean isOfflineScan(JobConf job) {
 +    return InputConfigurator.isOfflineScan(CLASS, job);
 +  }
 +
 +  /**
 +   * Initializes an Accumulo {@link TabletLocator} based on the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return an Accumulo tablet locator
 +   * @throws TableNotFoundException
 +   *           if the table name set on the configuration doesn't exist
 +   * @since 1.5.0
 +   */
 +  protected static TabletLocator getTabletLocator(JobConf job) throws TableNotFoundException {
 +    return InputConfigurator.getTabletLocator(CLASS, job);
 +  }
 +
 +  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
 +  /**
 +   * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @throws IOException
 +   *           if the context is improperly configured
 +   * @since 1.5.0
 +   */
 +  protected static void validateOptions(JobConf job) throws IOException {
 +    InputConfigurator.validateOptions(CLASS, job);
 +  }
 +
 +  /**
 +   * An abstract base class to be used to create {@link RecordReader} instances that convert from Accumulo {@link Key}/{@link Value} pairs to the user's K/V
 +   * types.
 +   * 
 +   * Subclasses must implement {@link #next(Object, Object)} to update key and value, and also to update the following variables:
 +   * <ul>
 +   * <li>Key {@link #currentKey} (used for progress reporting)</li>
 +   * <li>int {@link #numKeysRead} (used for progress reporting)</li>
 +   * </ul>
 +   */
 +  protected abstract static class RecordReaderBase<K,V> implements RecordReader<K,V> {
 +    protected long numKeysRead;
 +    protected Iterator<Entry<Key,Value>> scannerIterator;
 +    protected org.apache.accumulo.core.client.mapred.RangeInputSplit split;
 +
 +    /**
 +     * Apply the configured iterators from the configuration to the scanner.
 +     * 
 +     * @param scanner
 +     *          the scanner to configure
 +     */
 +    protected void setupIterators(List<IteratorSetting> iterators, Scanner scanner) {
 +      for (IteratorSetting iterator : iterators) {
 +        scanner.addScanIterator(iterator);
 +      }
 +    }
 +
 +    /**
 +     * Initialize a scanner over the given input split using this task attempt configuration.
 +     */
 +    public void initialize(InputSplit inSplit, JobConf job) throws IOException {
 +      Scanner scanner;
 +      split = (org.apache.accumulo.core.client.mapred.RangeInputSplit) inSplit;
 +      log.debug("Initializing input split: " + split.getRange());
 +
 +      Instance instance = split.getInstance();
 +      if (null == instance) {
 +        instance = getInstance(job);
 +      }
 +
 +      String principal = split.getPrincipal();
 +      if (null == principal) {
 +        principal = getPrincipal(job);
 +      }
 +
 +      AuthenticationToken token = split.getToken();
 +      if (null == token) {
 +        String tokenClass = getTokenClass(job);
 +        byte[] tokenBytes = getToken(job);
 +        try {
 +          token = CredentialHelper.extractToken(tokenClass, tokenBytes);
 +        } catch (AccumuloSecurityException e) {
 +          throw new IOException(e);
 +        }
 +      }
 +
 +      Authorizations authorizations = split.getAuths();
 +      if (null == authorizations) {
 +        authorizations = getScanAuthorizations(job);
 +      }
 +
 +      String table = split.getTable();
 +      if (null == table) {
 +        table = getInputTableName(job);
 +      }
 +
 +      Boolean isOffline = split.isOffline();
 +      if (null == isOffline) {
 +        isOffline = isOfflineScan(job);
 +      }
 +
 +      Boolean isIsolated = split.isIsolatedScan();
 +      if (null == isIsolated) {
 +        isIsolated = isIsolated(job);
 +      }
 +
 +      Boolean usesLocalIterators = split.usesLocalIterators();
 +      if (null == usesLocalIterators) {
 +        usesLocalIterators = usesLocalIterators(job);
 +      }
 +
 +      List<IteratorSetting> iterators = split.getIterators();
 +      if (null == iterators) {
 +        iterators = getIterators(job);
 +      }
 +
 +      Set<Pair<Text,Text>> columns = split.getFetchedColumns();
 +      if (null == columns) {
 +        columns = getFetchedColumns(job);
 +      }
 +
 +      try {
 +        log.debug("Creating connector with user: " + principal);
 +        Connector conn = instance.getConnector(principal, token);
 +        log.debug("Creating scanner for table: " + table);
 +        log.debug("Authorizations are: " + authorizations);
 +        if (isOffline) {
 +          String tokenClass = token.getClass().getCanonicalName();
 +          ByteBuffer tokenBuffer = ByteBuffer.wrap(CredentialHelper.toBytes(token));
 +          scanner = new OfflineScanner(instance, new TCredentials(principal, tokenClass, tokenBuffer, instance.getInstanceID()), Tables.getTableId(instance,
 +              table), authorizations);
 +        } else {
 +          scanner = conn.createScanner(table, authorizations);
 +        }
 +        if (isIsolated) {
 +          log.info("Creating isolated scanner");
 +          scanner = new IsolatedScanner(scanner);
 +        }
 +        if (usesLocalIterators) {
 +          log.info("Using local iterators");
 +          scanner = new ClientSideIteratorScanner(scanner);
 +        }
 +        setupIterators(iterators, scanner);
 +      } catch (Exception e) {
 +        throw new IOException(e);
 +      }
 +
 +      // setup a scanner within the bounds of this split
 +      for (Pair<Text,Text> c : columns) {
 +        if (c.getSecond() != null) {
 +          log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
 +          scanner.fetchColumn(c.getFirst(), c.getSecond());
 +        } else {
 +          log.debug("Fetching column family " + c.getFirst());
 +          scanner.fetchColumnFamily(c.getFirst());
 +        }
 +      }
 +
 +      scanner.setRange(split.getRange());
 +
 +      numKeysRead = 0;
 +
 +      // do this last after setting all scanner options
 +      scannerIterator = scanner.iterator();
 +    }
 +
 +    @Override
 +    public void close() {}
 +
 +    @Override
 +    public long getPos() throws IOException {
 +      return numKeysRead;
 +    }
 +
 +    @Override
 +    public float getProgress() throws IOException {
 +      if (numKeysRead > 0 && currentKey == null)
 +        return 1.0f;
 +      return split.getProgress(currentKey);
 +    }
 +
 +    protected Key currentKey = null;
 +
 +  }
 +
 +  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf 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);
 +    Connector conn = instance.getConnector(getPrincipal(job), CredentialHelper.extractToken(getTokenClass(job), getToken(job)));
 +    String tableId = Tables.getTableId(instance, tableName);
 +
 +    if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
 +      Tables.clearCache(instance);
 +      if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
 +        throw new AccumuloException("Table is online " + tableName + "(" + tableId + ") cannot scan table in offline mode ");
 +      }
 +    }
 +
 +    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);
 +      Constants.METADATA_PREV_ROW_COLUMN.fetch(scanner);
 +      scanner.fetchColumnFamily(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY);
 +      scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
 +      scanner.fetchColumnFamily(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY);
 +      scanner.setRange(metadataRange);
 +
 +      RowIterator rowIter = new RowIterator(scanner);
 +
 +      KeyExtent lastExtent = null;
 +
 +      while (rowIter.hasNext()) {
 +        Iterator<Entry<Key,Value>> row = rowIter.next();
 +        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();
 +          }
 +
 +          if (Constants.METADATA_PREV_ROW_COLUMN.hasColumns(key)) {
 +            extent = new KeyExtent(key.getRow(), entry.getValue());
 +          }
 +
 +        }
 +
 +        if (location != null)
 +          return null;
 +
 +        if (!extent.getTableId().toString().equals(tableId)) {
 +          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
 +        }
 +
 +        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
 +          throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
 +        }
 +
 +        Map<KeyExtent,List<Range>> tabletRanges = binnedRanges.get(last);
 +        if (tabletRanges == null) {
 +          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;
 +  }
 +
 +  /**
 +   * Read the metadata table to get tablets and match up ranges to them.
 +   */
 +  @Override
 +  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
 +    Level logLevel = getLogLevel(job);
 +    log.setLevel(logLevel);
 +
 +    validateOptions(job);
 +
 +    String tableName = getInputTableName(job);
 +    boolean autoAdjust = getAutoAdjustRanges(job);
 +    List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(job)) : getRanges(job);
 +    Instance instance = getInstance(job);
 +    boolean offline = isOfflineScan(job);
 +    boolean isolated = isIsolated(job);
 +    boolean localIterators = usesLocalIterators(job);
 +    boolean mockInstance = (null != instance && MockInstance.class.equals(instance.getClass()));
 +    Set<Pair<Text,Text>> fetchedColumns = getFetchedColumns(job);
 +    Authorizations auths = getScanAuthorizations(job);
 +    String principal = getPrincipal(job);
 +    String tokenClass = getTokenClass(job);
 +    byte[] tokenBytes = getToken(job);
 +
 +    AuthenticationToken token;
 +    try {
 +      token = CredentialHelper.extractToken(tokenClass, tokenBytes);
 +    } catch (AccumuloSecurityException e) {
 +      throw new IOException(e);
 +    }
 +
 +    List<IteratorSetting> iterators = getIterators(job);
 +
 +    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;
 +    try {
 +      if (isOfflineScan(job)) {
 +        binnedRanges = binOfflineTable(job, tableName, ranges);
 +        while (binnedRanges == null) {
 +          // Some tablets were still online, try again
 +          UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
 +          binnedRanges = binOfflineTable(job, tableName, ranges);
 +        }
 +      } else {
 +        String tableId = null;
 +        tl = getTabletLocator(job);
 +        // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
 +        tl.invalidateCache();
 +        while (!tl.binRanges(ranges, binnedRanges, new TCredentials(principal, tokenClass, ByteBuffer.wrap(tokenBytes), instance.getInstanceID())).isEmpty()) {
 +          if (!(instance instanceof MockInstance)) {
 +            if (tableId == null)
 +              tableId = Tables.getTableId(instance, tableName);
 +            if (!Tables.exists(instance, tableId))
 +              throw new TableDeletedException(tableId);
 +            if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
 +              throw new TableOfflineException(instance, tableId);
 +          }
 +          binnedRanges.clear();
 +          log.warn("Unable to locate bins for specified ranges. Retrying.");
 +          UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
 +          tl.invalidateCache();
 +        }
 +      }
 +    } catch (Exception e) {
 +      throw new IOException(e);
 +    }
 +
 +    ArrayList<org.apache.accumulo.core.client.mapred.RangeInputSplit> splits = new ArrayList<org.apache.accumulo.core.client.mapred.RangeInputSplit>(
 +        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);
 +      if (location == null) {
 +        InetAddress inetAddress = InetAddress.getByName(ip);
 +        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 org.apache.accumulo.core.client.mapred.RangeInputSplit(ke.clip(r), new String[] {location}));
 +          } else {
 +            // don't divide ranges
 +            ArrayList<String> locations = splitsToAdd.get(r);
 +            if (locations == null)
 +              locations = new ArrayList<String>(1);
 +            locations.add(location);
 +            splitsToAdd.put(r, locations);
 +          }
 +        }
 +      }
 +    }
 +
 +    if (!autoAdjust)
 +      for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
 +        splits.add(new org.apache.accumulo.core.client.mapred.RangeInputSplit(entry.getKey(), entry.getValue().toArray(new String[0])));
 +
 +    for (org.apache.accumulo.core.client.mapred.RangeInputSplit split : splits) {
 +      split.setTable(tableName);
 +      split.setOffline(offline);
 +      split.setIsolatedScan(isolated);
 +      split.setUsesLocalIterators(localIterators);
 +      split.setMockInstance(mockInstance);
 +      split.setFetchedColumns(fetchedColumns);
 +      split.setPrincipal(principal);
 +      split.setToken(token);
 +      split.setInstanceName(instance.getInstanceName());
 +      split.setZooKeepers(instance.getZooKeepers());
 +      split.setAuths(auths);
 +      split.setIterators(iterators);
 +      split.setLogLevel(logLevel);
 +    }
 +
 +    return splits.toArray(new InputSplit[splits.size()]);
 +  }
 +
 +  /**
 +   * @deprecated since 1.5.2; Use {@link org.apache.accumulo.core.client.mapred.RangeInputSplit} instead.
 +   * @see org.apache.accumulo.core.client.mapred.RangeInputSplit
 +   */
 +  @Deprecated
 +  public static class RangeInputSplit extends org.apache.accumulo.core.client.mapred.RangeInputSplit {
 +    public RangeInputSplit() {
 +      super();
 +    }
 +
 +    public RangeInputSplit(Range range, String[] locations) {
 +      super(range, locations);
 +    }
 +  }
 +}