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:19 UTC
[48/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);
+ }
+ }
+}