View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.client;
21  
22  import java.io.Closeable;
23  import java.io.DataInput;
24  import java.io.DataOutput;
25  import java.io.IOException;
26  import java.lang.reflect.Proxy;
27  import java.util.ArrayList;
28  import java.util.Arrays;
29  import java.util.Iterator;
30  import java.util.LinkedList;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.NavigableMap;
34  import java.util.TreeMap;
35  import java.util.concurrent.ExecutorService;
36  import java.util.concurrent.SynchronousQueue;
37  import java.util.concurrent.ThreadFactory;
38  import java.util.concurrent.ThreadPoolExecutor;
39  import java.util.concurrent.TimeUnit;
40  import java.util.concurrent.atomic.AtomicInteger;
41  
42  import org.apache.commons.logging.Log;
43  import org.apache.commons.logging.LogFactory;
44  import org.apache.hadoop.conf.Configuration;
45  import org.apache.hadoop.hbase.DoNotRetryIOException;
46  import org.apache.hadoop.hbase.HBaseConfiguration;
47  import org.apache.hadoop.hbase.HConstants;
48  import org.apache.hadoop.hbase.HRegionInfo;
49  import org.apache.hadoop.hbase.HRegionLocation;
50  import org.apache.hadoop.hbase.HServerAddress;
51  import org.apache.hadoop.hbase.HTableDescriptor;
52  import org.apache.hadoop.hbase.KeyValue;
53  import org.apache.hadoop.hbase.NotServingRegionException;
54  import org.apache.hadoop.hbase.ServerName;
55  import org.apache.hadoop.hbase.UnknownScannerException;
56  import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
57  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
58  import org.apache.hadoop.hbase.client.coprocessor.Batch;
59  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
60  import org.apache.hadoop.hbase.ipc.ExecRPCInvoker;
61  import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
62  import org.apache.hadoop.hbase.util.Addressing;
63  import org.apache.hadoop.hbase.util.Bytes;
64  import org.apache.hadoop.hbase.util.Pair;
65  import org.apache.hadoop.hbase.util.Writables;
66  
67  /**
68   * <p>Used to communicate with a single HBase table.
69   *
70   * <p>This class is not thread safe for reads nor write.
71   * 
72   * <p>In case of writes (Put, Delete), the underlying write buffer can
73   * be corrupted if multiple threads contend over a single HTable instance.
74   * 
75   * <p>In case of reads, some fields used by a Scan are shared among all threads.
76   * The HTable implementation can either not contract to be safe in case of a Get
77   *
78   * <p>To access a table in a multi threaded environment, please consider
79   * using the {@link HTablePool} class to create your HTable instances.
80   *
81   * <p>Instances of HTable passed the same {@link Configuration} instance will
82   * share connections to servers out on the cluster and to the zookeeper ensemble
83   * as well as caches of region locations.  This is usually a *good* thing and it
84   * is recommended to reuse the same configuration object for all your tables.
85   * This happens because they will all share the same underlying
86   * {@link HConnection} instance. See {@link HConnectionManager} for more on
87   * how this mechanism works.
88   *
89   * <p>{@link HConnection} will read most of the
90   * configuration it needs from the passed {@link Configuration} on initial
91   * construction.  Thereafter, for settings such as
92   * <code>hbase.client.pause</code>, <code>hbase.client.retries.number</code>,
93   * and <code>hbase.client.rpc.maxattempts</code> updating their values in the
94   * passed {@link Configuration} subsequent to {@link HConnection} construction
95   * will go unnoticed.  To run with changed values, make a new
96   * {@link HTable} passing a new {@link Configuration} instance that has the
97   * new configuration.
98   *
99   * <p>Note that this class implements the {@link Closeable} interface. When a
100  * HTable instance is no longer required, it *should* be closed in order to ensure
101  * that the underlying resources are promptly released. Please note that the close 
102  * method can throw java.io.IOException that must be handled.
103  *
104  * @see HBaseAdmin for create, drop, list, enable and disable of tables.
105  * @see HConnection
106  * @see HConnectionManager
107  */
108 public class HTable implements HTableInterface, Closeable {
109   private static final Log LOG = LogFactory.getLog(HTable.class);
110   private HConnection connection;
111   private final byte [] tableName;
112   protected int scannerTimeout;
113   private volatile Configuration configuration;
114   private final ArrayList<Put> writeBuffer = new ArrayList<Put>();
115   private long writeBufferSize;
116   private boolean clearBufferOnFail;
117   private boolean autoFlush;
118   private long currentWriteBufferSize;
119   protected int scannerCaching;
120   private int maxKeyValueSize;
121   private ExecutorService pool;  // For Multi
122   private long maxScannerResultSize;
123   private boolean closed;
124   private int operationTimeout;
125   private static final int DOPUT_WB_CHECK = 10;    // i.e., doPut checks the writebuffer every X Puts.
126   private final boolean cleanupOnClose; // close the connection in close()
127 
128   /**
129    * Creates an object to access a HBase table.
130    * Internally it creates a new instance of {@link Configuration} and a new
131    * client to zookeeper as well as other resources.  It also comes up with
132    * a fresh view of the cluster and must do discovery from scratch of region
133    * locations; i.e. it will not make use of already-cached region locations if
134    * available. Use only when being quick and dirty.
135    * @throws IOException if a remote or network exception occurs
136    * @see #HTable(Configuration, String)
137    */
138   public HTable(final String tableName)
139   throws IOException {
140     this(HBaseConfiguration.create(), Bytes.toBytes(tableName));
141   }
142 
143   /**
144    * Creates an object to access a HBase table.
145    * Internally it creates a new instance of {@link Configuration} and a new
146    * client to zookeeper as well as other resources.  It also comes up with
147    * a fresh view of the cluster and must do discovery from scratch of region
148    * locations; i.e. it will not make use of already-cached region locations if
149    * available. Use only when being quick and dirty.
150    * @param tableName Name of the table.
151    * @throws IOException if a remote or network exception occurs
152    * @see #HTable(Configuration, String)
153    */
154   public HTable(final byte [] tableName)
155   throws IOException {
156     this(HBaseConfiguration.create(), tableName);
157   }
158 
159   /**
160    * Creates an object to access a HBase table.
161    * Shares zookeeper connection and other resources with other HTable instances
162    * created with the same <code>conf</code> instance.  Uses already-populated
163    * region cache if one is available, populated by any other HTable instances
164    * sharing this <code>conf</code> instance.  Recommended.
165    * @param conf Configuration object to use.
166    * @param tableName Name of the table.
167    * @throws IOException if a remote or network exception occurs
168    */
169   public HTable(Configuration conf, final String tableName)
170   throws IOException {
171     this(conf, Bytes.toBytes(tableName));
172   }
173 
174 
175   /**
176    * Creates an object to access a HBase table.
177    * Shares zookeeper connection and other resources with other HTable instances
178    * created with the same <code>conf</code> instance.  Uses already-populated
179    * region cache if one is available, populated by any other HTable instances
180    * sharing this <code>conf</code> instance.  Recommended.
181    * @param conf Configuration object to use.
182    * @param tableName Name of the table.
183    * @throws IOException if a remote or network exception occurs
184    */
185   public HTable(Configuration conf, final byte [] tableName)
186   throws IOException {
187     this.tableName = tableName;
188     this.cleanupOnClose = true;
189     if (conf == null) {
190       this.scannerTimeout = 0;
191       this.connection = null;
192       return;
193     }
194     this.connection = HConnectionManager.getConnection(conf);
195     this.configuration = conf;
196 
197     int maxThreads = conf.getInt("hbase.htable.threads.max", Integer.MAX_VALUE);
198     if (maxThreads == 0) {
199       maxThreads = 1; // is there a better default?
200     }
201     // Using the "direct handoff" approach, new threads will only be created
202     // if it is necessary and will grow unbounded. This could be bad but in HCM
203     // we only create as many Runnables as there are region servers. It means
204     // it also scales when new region servers are added.
205     this.pool = new ThreadPoolExecutor(1, maxThreads,
206         60, TimeUnit.SECONDS,
207         new SynchronousQueue<Runnable>(),
208         new DaemonThreadFactory());
209     ((ThreadPoolExecutor)this.pool).allowCoreThreadTimeOut(true);
210 
211     this.finishSetup();
212   }
213 
214   /**
215    * Creates an object to access a HBase table.
216    * Shares zookeeper connection and other resources with other HTable instances
217    * created with the same <code>connection</code> instance.
218    * Use this constructor when the ExecutorService and HConnection instance are
219    * externally managed.
220    * @param tableName Name of the table.
221    * @param connection HConnection to be used.
222    * @param pool ExecutorService to be used.
223    * @throws IOException if a remote or network exception occurs
224    */
225   public HTable(final byte[] tableName, final HConnection connection, 
226       final ExecutorService pool) throws IOException {
227     if (pool == null || pool.isShutdown()) {
228       throw new IllegalArgumentException("Pool is null or shut down.");
229     }
230     if (connection == null || connection.isClosed()) {
231       throw new IllegalArgumentException("Connection is null or closed.");
232     }
233     this.tableName = tableName;
234     this.cleanupOnClose = false;
235     this.connection = connection;
236     this.configuration = connection.getConfiguration();
237     this.pool = pool;
238 
239     this.finishSetup();
240   }
241 
242   /**
243    * setup this HTable's parameter based on the passed configuration
244    * @param conf
245    */
246   private void finishSetup() throws IOException {
247     this.connection.locateRegion(tableName, HConstants.EMPTY_START_ROW);
248     this.scannerTimeout = (int) this.configuration.getLong(
249         HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
250         HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
251     this.operationTimeout = HTableDescriptor.isMetaTable(tableName) ? HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT
252         : this.configuration.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
253             HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
254     this.writeBufferSize = this.configuration.getLong(
255         "hbase.client.write.buffer", 2097152);
256     this.clearBufferOnFail = true;
257     this.autoFlush = true;
258     this.currentWriteBufferSize = 0;
259     this.scannerCaching = this.configuration.getInt(
260         "hbase.client.scanner.caching", 1);
261 
262     this.maxScannerResultSize = this.configuration.getLong(
263         HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
264         HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
265     this.maxKeyValueSize = this.configuration.getInt(
266         "hbase.client.keyvalue.maxsize", -1);
267     this.closed = false;
268   }
269 
270   /**
271    * {@inheritDoc}
272    */
273   @Override
274   public Configuration getConfiguration() {
275     return configuration;
276   }
277 
278   /**
279    * Tells whether or not a table is enabled or not.
280    * @param tableName Name of table to check.
281    * @return {@code true} if table is online.
282    * @throws IOException if a remote or network exception occurs
283 	* @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
284    */
285   @Deprecated
286   public static boolean isTableEnabled(String tableName) throws IOException {
287     return isTableEnabled(Bytes.toBytes(tableName));
288   }
289 
290   /**
291    * Tells whether or not a table is enabled or not.
292    * @param tableName Name of table to check.
293    * @return {@code true} if table is online.
294    * @throws IOException if a remote or network exception occurs
295 	* @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
296    */
297   @Deprecated
298   public static boolean isTableEnabled(byte[] tableName) throws IOException {
299     return isTableEnabled(HBaseConfiguration.create(), tableName);
300   }
301 
302   /**
303    * Tells whether or not a table is enabled or not.
304    * @param conf The Configuration object to use.
305    * @param tableName Name of table to check.
306    * @return {@code true} if table is online.
307    * @throws IOException if a remote or network exception occurs
308 	* @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
309    */
310   @Deprecated
311   public static boolean isTableEnabled(Configuration conf, String tableName)
312   throws IOException {
313     return isTableEnabled(conf, Bytes.toBytes(tableName));
314   }
315 
316   /**
317    * Tells whether or not a table is enabled or not.
318    * @param conf The Configuration object to use.
319    * @param tableName Name of table to check.
320    * @return {@code true} if table is online.
321    * @throws IOException if a remote or network exception occurs
322    */
323   public static boolean isTableEnabled(Configuration conf,
324       final byte[] tableName) throws IOException {
325     return HConnectionManager.execute(new HConnectable<Boolean>(conf) {
326       @Override
327       public Boolean connect(HConnection connection) throws IOException {
328         return connection.isTableEnabled(tableName);
329       }
330     });
331   }
332 
333   /**
334    * Find region location hosting passed row using cached info
335    * @param row Row to find.
336    * @return The location of the given row.
337    * @throws IOException if a remote or network exception occurs
338    */
339   public HRegionLocation getRegionLocation(final String row)
340   throws IOException {
341     return connection.getRegionLocation(tableName, Bytes.toBytes(row), false);
342   }
343 
344   /**
345    * Finds the region on which the given row is being served.
346    * @param row Row to find.
347    * @return Location of the row.
348    * @throws IOException if a remote or network exception occurs
349    */
350   public HRegionLocation getRegionLocation(final byte [] row)
351   throws IOException {
352     return connection.getRegionLocation(tableName, row, false);
353   }
354 
355   /**
356    * {@inheritDoc}
357    */
358   @Override
359   public byte [] getTableName() {
360     return this.tableName;
361   }
362 
363   /**
364    * <em>INTERNAL</em> Used by unit tests and tools to do low-level
365    * manipulations.
366    * @return An HConnection instance.
367    * @deprecated This method will be changed from public to package protected.
368    */
369   // TODO(tsuna): Remove this.  Unit tests shouldn't require public helpers.
370   public HConnection getConnection() {
371     return this.connection;
372   }
373 
374   /**
375    * Gets the number of rows that a scanner will fetch at once.
376    * <p>
377    * The default value comes from {@code hbase.client.scanner.caching}.
378    */
379   public int getScannerCaching() {
380     return scannerCaching;
381   }
382 
383   /**
384    * Sets the number of rows that a scanner will fetch at once.
385    * <p>
386    * This will override the value specified by
387    * {@code hbase.client.scanner.caching}.
388    * Increasing this value will reduce the amount of work needed each time
389    * {@code next()} is called on a scanner, at the expense of memory use
390    * (since more rows will need to be maintained in memory by the scanners).
391    * @param scannerCaching the number of rows a scanner will fetch at once.
392    */
393   public void setScannerCaching(int scannerCaching) {
394     this.scannerCaching = scannerCaching;
395   }
396 
397   /**
398    * {@inheritDoc}
399    */
400   @Override
401   public HTableDescriptor getTableDescriptor() throws IOException {
402     return new UnmodifyableHTableDescriptor(
403       this.connection.getHTableDescriptor(this.tableName));
404   }
405 
406   /**
407    * Gets the starting row key for every region in the currently open table.
408    * <p>
409    * This is mainly useful for the MapReduce integration.
410    * @return Array of region starting row keys
411    * @throws IOException if a remote or network exception occurs
412    */
413   public byte [][] getStartKeys() throws IOException {
414     return getStartEndKeys().getFirst();
415   }
416 
417   /**
418    * Gets the ending row key for every region in the currently open table.
419    * <p>
420    * This is mainly useful for the MapReduce integration.
421    * @return Array of region ending row keys
422    * @throws IOException if a remote or network exception occurs
423    */
424   public byte[][] getEndKeys() throws IOException {
425     return getStartEndKeys().getSecond();
426   }
427 
428   /**
429    * Gets the starting and ending row keys for every region in the currently
430    * open table.
431    * <p>
432    * This is mainly useful for the MapReduce integration.
433    * @return Pair of arrays of region starting and ending row keys
434    * @throws IOException if a remote or network exception occurs
435    */
436   @SuppressWarnings("unchecked")
437   public Pair<byte[][],byte[][]> getStartEndKeys() throws IOException {
438     final List<byte[]> startKeyList = new ArrayList<byte[]>();
439     final List<byte[]> endKeyList = new ArrayList<byte[]>();
440     MetaScannerVisitor visitor = new MetaScannerVisitor() {
441       public boolean processRow(Result rowResult) throws IOException {
442         byte [] bytes = rowResult.getValue(HConstants.CATALOG_FAMILY,
443           HConstants.REGIONINFO_QUALIFIER);
444         if (bytes == null) {
445           LOG.warn("Null " + HConstants.REGIONINFO_QUALIFIER + " cell in " +
446             rowResult);
447           return true;
448         }
449         HRegionInfo info = Writables.getHRegionInfo(bytes);
450         if (Bytes.equals(info.getTableName(), getTableName())) {
451           if (!(info.isOffline() || info.isSplit())) {
452             startKeyList.add(info.getStartKey());
453             endKeyList.add(info.getEndKey());
454           }
455         }
456         return true;
457       }
458     };
459     MetaScanner.metaScan(configuration, visitor, this.tableName);
460     return new Pair<byte [][], byte [][]>(
461       startKeyList.toArray(new byte[startKeyList.size()][]),
462       endKeyList.toArray(new byte[endKeyList.size()][]));
463   }
464 
465   /**
466    * Gets all the regions and their address for this table.
467    * @return A map of HRegionInfo with it's server address
468    * @throws IOException if a remote or network exception occurs
469    * @deprecated Use {@link #getRegionLocations()} or {@link #getStartEndKeys()}
470    */
471   public Map<HRegionInfo, HServerAddress> getRegionsInfo() throws IOException {
472     final Map<HRegionInfo, HServerAddress> regionMap =
473       new TreeMap<HRegionInfo, HServerAddress>();
474 
475     MetaScannerVisitor visitor = new MetaScannerVisitor() {
476       public boolean processRow(Result rowResult) throws IOException {
477         HRegionInfo info = Writables.getHRegionInfo(
478             rowResult.getValue(HConstants.CATALOG_FAMILY,
479                 HConstants.REGIONINFO_QUALIFIER));
480 
481         if (!(Bytes.equals(info.getTableName(), getTableName()))) {
482           return false;
483         }
484 
485         HServerAddress server = new HServerAddress();
486         byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
487             HConstants.SERVER_QUALIFIER);
488         if (value != null && value.length > 0) {
489           String hostAndPort = Bytes.toString(value);
490           server = new HServerAddress(Addressing.createInetSocketAddressFromHostAndPortStr(hostAndPort));
491         }
492 
493         if (!(info.isOffline() || info.isSplit())) {
494           regionMap.put(new UnmodifyableHRegionInfo(info), server);
495         }
496         return true;
497       }
498 
499     };
500     MetaScanner.metaScan(configuration, visitor, tableName);
501     return regionMap;
502   }
503 
504   /**
505    * Gets all the regions and their address for this table.
506    * <p>
507    * This is mainly useful for the MapReduce integration.
508    * @return A map of HRegionInfo with it's server address
509    * @throws IOException if a remote or network exception occurs
510    */
511   public NavigableMap<HRegionInfo, ServerName> getRegionLocations() throws IOException {
512     return MetaScanner.allTableRegions(getConfiguration(), getTableName(), false);
513   }
514 
515   /**
516    * Save the passed region information and the table's regions
517    * cache.
518    * <p>
519    * This is mainly useful for the MapReduce integration. You can call
520    * {@link #deserializeRegionInfo deserializeRegionInfo}
521    * to deserialize regions information from a
522    * {@link DataInput}, then call this method to load them to cache.
523    *
524    * <pre>
525    * {@code
526    * HTable t1 = new HTable("foo");
527    * FileInputStream fis = new FileInputStream("regions.dat");
528    * DataInputStream dis = new DataInputStream(fis);
529    *
530    * Map<HRegionInfo, HServerAddress> hm = t1.deserializeRegionInfo(dis);
531    * t1.prewarmRegionCache(hm);
532    * }
533    * </pre>
534    * @param regionMap This piece of regions information will be loaded
535    * to region cache.
536    */
537   public void prewarmRegionCache(Map<HRegionInfo, HServerAddress> regionMap) {
538     this.connection.prewarmRegionCache(this.getTableName(), regionMap);
539   }
540 
541   /**
542    * Serialize the regions information of this table and output
543    * to <code>out</code>.
544    * <p>
545    * This is mainly useful for the MapReduce integration. A client could
546    * perform a large scan for all the regions for the table, serialize the
547    * region info to a file. MR job can ship a copy of the meta for the table in
548    * the DistributedCache.
549    * <pre>
550    * {@code
551    * FileOutputStream fos = new FileOutputStream("regions.dat");
552    * DataOutputStream dos = new DataOutputStream(fos);
553    * table.serializeRegionInfo(dos);
554    * dos.flush();
555    * dos.close();
556    * }
557    * </pre>
558    * @param out {@link DataOutput} to serialize this object into.
559    * @throws IOException if a remote or network exception occurs
560    */
561   public void serializeRegionInfo(DataOutput out) throws IOException {
562     Map<HRegionInfo, HServerAddress> allRegions = this.getRegionsInfo();
563     // first, write number of regions
564     out.writeInt(allRegions.size());
565     for (Map.Entry<HRegionInfo, HServerAddress> es : allRegions.entrySet()) {
566       es.getKey().write(out);
567       es.getValue().write(out);
568     }
569   }
570 
571   /**
572    * Read from <code>in</code> and deserialize the regions information.
573    *
574    * <p>It behaves similarly as {@link #getRegionsInfo getRegionsInfo}, except
575    * that it loads the region map from a {@link DataInput} object.
576    *
577    * <p>It is supposed to be followed immediately by  {@link
578    * #prewarmRegionCache prewarmRegionCache}.
579    *
580    * <p>
581    * Please refer to {@link #prewarmRegionCache prewarmRegionCache} for usage.
582    *
583    * @param in {@link DataInput} object.
584    * @return A map of HRegionInfo with its server address.
585    * @throws IOException if an I/O exception occurs.
586    */
587   public Map<HRegionInfo, HServerAddress> deserializeRegionInfo(DataInput in)
588   throws IOException {
589     final Map<HRegionInfo, HServerAddress> allRegions =
590       new TreeMap<HRegionInfo, HServerAddress>();
591 
592     // the first integer is expected to be the size of records
593     int regionsCount = in.readInt();
594     for (int i = 0; i < regionsCount; ++i) {
595       HRegionInfo hri = new HRegionInfo();
596       hri.readFields(in);
597       HServerAddress hsa = new HServerAddress();
598       hsa.readFields(in);
599       allRegions.put(hri, hsa);
600     }
601     return allRegions;
602   }
603 
604   /**
605    * {@inheritDoc}
606    */
607    @Override
608    public Result getRowOrBefore(final byte[] row, final byte[] family)
609    throws IOException {
610      return connection.getRegionServerWithRetries(
611          new ServerCallable<Result>(connection, tableName, row, operationTimeout) {
612        public Result call() throws IOException {
613          return server.getClosestRowBefore(location.getRegionInfo().getRegionName(),
614            row, family);
615        }
616      });
617    }
618 
619    /**
620     * {@inheritDoc}
621     */
622   @Override
623   public ResultScanner getScanner(final Scan scan) throws IOException {
624     ClientScanner s = new ClientScanner(scan);
625     s.initialize();
626     return s;
627   }
628 
629   /**
630    * {@inheritDoc}
631    */
632   @Override
633   public ResultScanner getScanner(byte [] family) throws IOException {
634     Scan scan = new Scan();
635     scan.addFamily(family);
636     return getScanner(scan);
637   }
638 
639   /**
640    * {@inheritDoc}
641    */
642   @Override
643   public ResultScanner getScanner(byte [] family, byte [] qualifier)
644   throws IOException {
645     Scan scan = new Scan();
646     scan.addColumn(family, qualifier);
647     return getScanner(scan);
648   }
649 
650   /**
651    * {@inheritDoc}
652    */
653   @Override
654   public Result get(final Get get) throws IOException {
655     return connection.getRegionServerWithRetries(
656         new ServerCallable<Result>(connection, tableName, get.getRow(), operationTimeout) {
657           public Result call() throws IOException {
658             return server.get(location.getRegionInfo().getRegionName(), get);
659           }
660         }
661     );
662   }
663 
664   /**
665    * {@inheritDoc}
666    */
667   @Override
668   public Result[] get(List<Get> gets) throws IOException {
669     try {
670       Object [] r1 = batch((List)gets);
671 
672       // translate.
673       Result [] results = new Result[r1.length];
674       int i=0;
675       for (Object o : r1) {
676         // batch ensures if there is a failure we get an exception instead
677         results[i++] = (Result) o;
678       }
679 
680       return results;
681     } catch (InterruptedException e) {
682       throw new IOException(e);
683     }
684   }
685 
686   /**
687    * {@inheritDoc}
688    */
689   @Override
690   public synchronized void batch(final List<Row> actions, final Object[] results)
691       throws InterruptedException, IOException {
692     connection.processBatch(actions, tableName, pool, results);
693   }
694 
695   /**
696    * {@inheritDoc}
697    */
698   @Override
699   public synchronized Object[] batch(final List<Row> actions) throws InterruptedException, IOException {
700     Object[] results = new Object[actions.size()];
701     connection.processBatch(actions, tableName, pool, results);
702     return results;
703   }
704 
705   /**
706    * {@inheritDoc}
707    */
708   @Override
709   public void delete(final Delete delete)
710   throws IOException {
711     connection.getRegionServerWithRetries(
712         new ServerCallable<Boolean>(connection, tableName, delete.getRow(), operationTimeout) {
713           public Boolean call() throws IOException {
714             server.delete(location.getRegionInfo().getRegionName(), delete);
715             return null; // FindBugs NP_BOOLEAN_RETURN_NULL
716           }
717         }
718     );
719   }
720 
721   /**
722    * {@inheritDoc}
723    */
724   @Override
725   public void delete(final List<Delete> deletes)
726   throws IOException {
727     Object[] results = new Object[deletes.size()];
728     try {
729       connection.processBatch((List) deletes, tableName, pool, results);
730     } catch (InterruptedException e) {
731       throw new IOException(e);
732     } finally {
733       // mutate list so that it is empty for complete success, or contains only failed records
734       // results are returned in the same order as the requests in list
735       // walk the list backwards, so we can remove from list without impacting the indexes of earlier members
736       for (int i = results.length - 1; i>=0; i--) {
737         // if result is not null, it succeeded
738         if (results[i] instanceof Result) {
739           deletes.remove(i);
740         }
741       }
742     }
743   }
744 
745   /**
746    * {@inheritDoc}
747    */
748   @Override
749   public void put(final Put put) throws IOException {
750     doPut(Arrays.asList(put));
751   }
752 
753   /**
754    * {@inheritDoc}
755    */
756   @Override
757   public void put(final List<Put> puts) throws IOException {
758     doPut(puts);
759   }
760 
761   private void doPut(final List<Put> puts) throws IOException {
762     int n = 0;
763     for (Put put : puts) {
764       validatePut(put);
765       writeBuffer.add(put);
766       currentWriteBufferSize += put.heapSize();
767      
768       // we need to periodically see if the writebuffer is full instead of waiting until the end of the List
769       n++;
770       if (n % DOPUT_WB_CHECK == 0 && currentWriteBufferSize > writeBufferSize) {
771         flushCommits();
772       }
773     }
774     if (autoFlush || currentWriteBufferSize > writeBufferSize) {
775       flushCommits();
776     }
777   }
778 
779   /**
780    * {@inheritDoc}
781    */
782   @Override
783   public Result increment(final Increment increment) throws IOException {
784     if (!increment.hasFamilies()) {
785       throw new IOException(
786           "Invalid arguments to increment, no columns specified");
787     }
788     return connection.getRegionServerWithRetries(
789         new ServerCallable<Result>(connection, tableName, increment.getRow(), operationTimeout) {
790           public Result call() throws IOException {
791             return server.increment(
792                 location.getRegionInfo().getRegionName(), increment);
793           }
794         }
795     );
796   }
797 
798   /**
799    * {@inheritDoc}
800    */
801   @Override
802   public long incrementColumnValue(final byte [] row, final byte [] family,
803       final byte [] qualifier, final long amount)
804   throws IOException {
805     return incrementColumnValue(row, family, qualifier, amount, true);
806   }
807 
808   /**
809    * {@inheritDoc}
810    */
811   @Override
812   public long incrementColumnValue(final byte [] row, final byte [] family,
813       final byte [] qualifier, final long amount, final boolean writeToWAL)
814   throws IOException {
815     NullPointerException npe = null;
816     if (row == null) {
817       npe = new NullPointerException("row is null");
818     } else if (family == null) {
819       npe = new NullPointerException("column is null");
820     }
821     if (npe != null) {
822       throw new IOException(
823           "Invalid arguments to incrementColumnValue", npe);
824     }
825     return connection.getRegionServerWithRetries(
826         new ServerCallable<Long>(connection, tableName, row, operationTimeout) {
827           public Long call() throws IOException {
828             return server.incrementColumnValue(
829                 location.getRegionInfo().getRegionName(), row, family,
830                 qualifier, amount, writeToWAL);
831           }
832         }
833     );
834   }
835 
836   /**
837    * {@inheritDoc}
838    */
839   @Override
840   public boolean checkAndPut(final byte [] row,
841       final byte [] family, final byte [] qualifier, final byte [] value,
842       final Put put)
843   throws IOException {
844     return connection.getRegionServerWithRetries(
845         new ServerCallable<Boolean>(connection, tableName, row, operationTimeout) {
846           public Boolean call() throws IOException {
847             return server.checkAndPut(location.getRegionInfo().getRegionName(),
848                 row, family, qualifier, value, put) ? Boolean.TRUE : Boolean.FALSE;
849           }
850         }
851     );
852   }
853 
854 
855   /**
856    * {@inheritDoc}
857    */
858   @Override
859   public boolean checkAndDelete(final byte [] row,
860       final byte [] family, final byte [] qualifier, final byte [] value,
861       final Delete delete)
862   throws IOException {
863     return connection.getRegionServerWithRetries(
864         new ServerCallable<Boolean>(connection, tableName, row, operationTimeout) {
865           public Boolean call() throws IOException {
866             return server.checkAndDelete(
867                 location.getRegionInfo().getRegionName(),
868                 row, family, qualifier, value, delete)
869             ? Boolean.TRUE : Boolean.FALSE;
870           }
871         }
872     );
873   }
874 
875   /**
876    * {@inheritDoc}
877    */
878   @Override
879   public boolean exists(final Get get) throws IOException {
880     return connection.getRegionServerWithRetries(
881         new ServerCallable<Boolean>(connection, tableName, get.getRow(), operationTimeout) {
882           public Boolean call() throws IOException {
883             return server.
884                 exists(location.getRegionInfo().getRegionName(), get);
885           }
886         }
887     );
888   }
889 
890   /**
891    * {@inheritDoc}
892    */
893   @Override
894   public void flushCommits() throws IOException {
895     try {
896       Object[] results = new Object[writeBuffer.size()];
897       try {
898         this.connection.processBatch(writeBuffer, tableName, pool, results);
899       } catch (InterruptedException e) {
900         throw new IOException(e);
901       } finally {
902         // mutate list so that it is empty for complete success, or contains
903         // only failed records results are returned in the same order as the
904         // requests in list walk the list backwards, so we can remove from list
905         // without impacting the indexes of earlier members
906         for (int i = results.length - 1; i>=0; i--) {
907           if (results[i] instanceof Result) {
908             // successful Puts are removed from the list here.
909             writeBuffer.remove(i);
910           }
911         }
912       }
913     } finally {
914       if (clearBufferOnFail) {
915         writeBuffer.clear();
916         currentWriteBufferSize = 0;
917       } else {
918         // the write buffer was adjusted by processBatchOfPuts
919         currentWriteBufferSize = 0;
920         for (Put aPut : writeBuffer) {
921           currentWriteBufferSize += aPut.heapSize();
922         }
923       }
924     }
925   }
926 
927   /**
928    * {@inheritDoc}
929    */
930   @Override
931   public void close() throws IOException {
932     if (this.closed) {
933       return;
934     }
935     flushCommits();
936     if (cleanupOnClose) {
937       this.pool.shutdown();
938       if (this.connection != null) {
939         this.connection.close();
940       }
941     }
942     this.closed = true;
943   }
944 
945   // validate for well-formedness
946   private void validatePut(final Put put) throws IllegalArgumentException{
947     if (put.isEmpty()) {
948       throw new IllegalArgumentException("No columns to insert");
949     }
950     if (maxKeyValueSize > 0) {
951       for (List<KeyValue> list : put.getFamilyMap().values()) {
952         for (KeyValue kv : list) {
953           if (kv.getLength() > maxKeyValueSize) {
954             throw new IllegalArgumentException("KeyValue size too large");
955           }
956         }
957       }
958     }
959   }
960 
961   /**
962    * {@inheritDoc}
963    */
964   @Override
965   public RowLock lockRow(final byte [] row)
966   throws IOException {
967     return connection.getRegionServerWithRetries(
968       new ServerCallable<RowLock>(connection, tableName, row, operationTimeout) {
969         public RowLock call() throws IOException {
970           long lockId =
971               server.lockRow(location.getRegionInfo().getRegionName(), row);
972           return new RowLock(row,lockId);
973         }
974       }
975     );
976   }
977 
978   /**
979    * {@inheritDoc}
980    */
981   @Override
982   public void unlockRow(final RowLock rl)
983   throws IOException {
984     connection.getRegionServerWithRetries(
985       new ServerCallable<Boolean>(connection, tableName, rl.getRow(), operationTimeout) {
986         public Boolean call() throws IOException {
987           server.unlockRow(location.getRegionInfo().getRegionName(),
988               rl.getLockId());
989           return null; // FindBugs NP_BOOLEAN_RETURN_NULL
990         }
991       }
992     );
993   }
994 
995   /**
996    * {@inheritDoc}
997    */
998   @Override
999   public boolean isAutoFlush() {
1000     return autoFlush;
1001   }
1002 
1003   /**
1004    * See {@link #setAutoFlush(boolean, boolean)}
1005    *
1006    * @param autoFlush
1007    *          Whether or not to enable 'auto-flush'.
1008    */
1009   public void setAutoFlush(boolean autoFlush) {
1010     setAutoFlush(autoFlush, autoFlush);
1011   }
1012 
1013   /**
1014    * Turns 'auto-flush' on or off.
1015    * <p>
1016    * When enabled (default), {@link Put} operations don't get buffered/delayed
1017    * and are immediately executed. Failed operations are not retried. This is
1018    * slower but safer.
1019    * <p>
1020    * Turning off {@link #autoFlush} means that multiple {@link Put}s will be
1021    * accepted before any RPC is actually sent to do the write operations. If the
1022    * application dies before pending writes get flushed to HBase, data will be
1023    * lost.
1024    * <p>
1025    * When you turn {@link #autoFlush} off, you should also consider the
1026    * {@link #clearBufferOnFail} option. By default, asynchronous {@link Put}
1027    * requests will be retried on failure until successful. However, this can
1028    * pollute the writeBuffer and slow down batching performance. Additionally,
1029    * you may want to issue a number of Put requests and call
1030    * {@link #flushCommits()} as a barrier. In both use cases, consider setting
1031    * clearBufferOnFail to true to erase the buffer after {@link #flushCommits()}
1032    * has been called, regardless of success.
1033    *
1034    * @param autoFlush
1035    *          Whether or not to enable 'auto-flush'.
1036    * @param clearBufferOnFail
1037    *          Whether to keep Put failures in the writeBuffer
1038    * @see #flushCommits
1039    */
1040   public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
1041     this.autoFlush = autoFlush;
1042     this.clearBufferOnFail = autoFlush || clearBufferOnFail;
1043   }
1044 
1045   /**
1046    * Returns the maximum size in bytes of the write buffer for this HTable.
1047    * <p>
1048    * The default value comes from the configuration parameter
1049    * {@code hbase.client.write.buffer}.
1050    * @return The size of the write buffer in bytes.
1051    */
1052   public long getWriteBufferSize() {
1053     return writeBufferSize;
1054   }
1055 
1056   /**
1057    * Sets the size of the buffer in bytes.
1058    * <p>
1059    * If the new size is less than the current amount of data in the
1060    * write buffer, the buffer gets flushed.
1061    * @param writeBufferSize The new write buffer size, in bytes.
1062    * @throws IOException if a remote or network exception occurs.
1063    */
1064   public void setWriteBufferSize(long writeBufferSize) throws IOException {
1065     this.writeBufferSize = writeBufferSize;
1066     if(currentWriteBufferSize > writeBufferSize) {
1067       flushCommits();
1068     }
1069   }
1070 
1071   /**
1072    * Returns the write buffer.
1073    * @return The current write buffer.
1074    */
1075   public ArrayList<Put> getWriteBuffer() {
1076     return writeBuffer;
1077   }
1078 
1079   /**
1080    * Implements the scanner interface for the HBase client.
1081    * If there are multiple regions in a table, this scanner will iterate
1082    * through them all.
1083    */
1084   protected class ClientScanner implements ResultScanner {
1085     private final Log CLIENT_LOG = LogFactory.getLog(this.getClass());
1086     // HEADSUP: The scan internal start row can change as we move through table.
1087     private Scan scan;
1088     private boolean closed = false;
1089     // Current region scanner is against.  Gets cleared if current region goes
1090     // wonky: e.g. if it splits on us.
1091     private HRegionInfo currentRegion = null;
1092     private ScannerCallable callable = null;
1093     private final LinkedList<Result> cache = new LinkedList<Result>();
1094     private final int caching;
1095     private long lastNext;
1096     // Keep lastResult returned successfully in case we have to reset scanner.
1097     private Result lastResult = null;
1098 
1099     protected ClientScanner(final Scan scan) {
1100       if (CLIENT_LOG.isDebugEnabled()) {
1101         CLIENT_LOG.debug("Creating scanner over "
1102             + Bytes.toString(getTableName())
1103             + " starting at key '" + Bytes.toStringBinary(scan.getStartRow()) + "'");
1104       }
1105       this.scan = scan;
1106       this.lastNext = System.currentTimeMillis();
1107 
1108       // Use the caching from the Scan.  If not set, use the default cache setting for this table.
1109       if (this.scan.getCaching() > 0) {
1110         this.caching = this.scan.getCaching();
1111       } else {
1112         this.caching = HTable.this.scannerCaching;
1113       }
1114 
1115       // Removed filter validation.  We have a new format now, only one of all
1116       // the current filters has a validate() method.  We can add it back,
1117       // need to decide on what we're going to do re: filter redesign.
1118       // Need, at the least, to break up family from qualifier as separate
1119       // checks, I think it's important server-side filters are optimal in that
1120       // respect.
1121     }
1122 
1123     public void initialize() throws IOException {
1124       nextScanner(this.caching, false);
1125     }
1126 
1127     protected Scan getScan() {
1128       return scan;
1129     }
1130 
1131     protected long getTimestamp() {
1132       return lastNext;
1133     }
1134 
1135     // returns true if the passed region endKey
1136     private boolean checkScanStopRow(final byte [] endKey) {
1137       if (this.scan.getStopRow().length > 0) {
1138         // there is a stop row, check to see if we are past it.
1139         byte [] stopRow = scan.getStopRow();
1140         int cmp = Bytes.compareTo(stopRow, 0, stopRow.length,
1141           endKey, 0, endKey.length);
1142         if (cmp <= 0) {
1143           // stopRow <= endKey (endKey is equals to or larger than stopRow)
1144           // This is a stop.
1145           return true;
1146         }
1147       }
1148       return false; //unlikely.
1149     }
1150 
1151     /*
1152      * Gets a scanner for the next region.  If this.currentRegion != null, then
1153      * we will move to the endrow of this.currentRegion.  Else we will get
1154      * scanner at the scan.getStartRow().  We will go no further, just tidy
1155      * up outstanding scanners, if <code>currentRegion != null</code> and
1156      * <code>done</code> is true.
1157      * @param nbRows
1158      * @param done Server-side says we're done scanning.
1159      */
1160     private boolean nextScanner(int nbRows, final boolean done)
1161     throws IOException {
1162       // Close the previous scanner if it's open
1163       if (this.callable != null) {
1164         this.callable.setClose();
1165         getConnection().getRegionServerWithRetries(callable);
1166         this.callable = null;
1167       }
1168 
1169       // Where to start the next scanner
1170       byte [] localStartKey;
1171 
1172       // if we're at end of table, close and return false to stop iterating
1173       if (this.currentRegion != null) {
1174         byte [] endKey = this.currentRegion.getEndKey();
1175         if (endKey == null ||
1176             Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) ||
1177             checkScanStopRow(endKey) ||
1178             done) {
1179           close();
1180           if (CLIENT_LOG.isDebugEnabled()) {
1181             CLIENT_LOG.debug("Finished with scanning at " + this.currentRegion);
1182           }
1183           return false;
1184         }
1185         localStartKey = endKey;
1186         if (CLIENT_LOG.isDebugEnabled()) {
1187           CLIENT_LOG.debug("Finished with region " + this.currentRegion);
1188         }
1189       } else {
1190         localStartKey = this.scan.getStartRow();
1191       }
1192 
1193       if (CLIENT_LOG.isDebugEnabled()) {
1194         CLIENT_LOG.debug("Advancing internal scanner to startKey at '" +
1195           Bytes.toStringBinary(localStartKey) + "'");
1196       }
1197       try {
1198         callable = getScannerCallable(localStartKey, nbRows);
1199         // Open a scanner on the region server starting at the
1200         // beginning of the region
1201         getConnection().getRegionServerWithRetries(callable);
1202         this.currentRegion = callable.getHRegionInfo();
1203       } catch (IOException e) {
1204         close();
1205         throw e;
1206       }
1207       return true;
1208     }
1209 
1210     protected ScannerCallable getScannerCallable(byte [] localStartKey,
1211         int nbRows) {
1212       scan.setStartRow(localStartKey);
1213       ScannerCallable s = new ScannerCallable(getConnection(),
1214         getTableName(), scan);
1215       s.setCaching(nbRows);
1216       return s;
1217     }
1218 
1219     public Result next() throws IOException {
1220       // If the scanner is closed but there is some rows left in the cache,
1221       // it will first empty it before returning null
1222       if (cache.size() == 0 && this.closed) {
1223         return null;
1224       }
1225       if (cache.size() == 0) {
1226         Result [] values = null;
1227         long remainingResultSize = maxScannerResultSize;
1228         int countdown = this.caching;
1229         // We need to reset it if it's a new callable that was created
1230         // with a countdown in nextScanner
1231         callable.setCaching(this.caching);
1232         // This flag is set when we want to skip the result returned.  We do
1233         // this when we reset scanner because it split under us.
1234         boolean skipFirst = false;
1235         do {
1236           try {
1237             if (skipFirst) {
1238               // Skip only the first row (which was the last row of the last
1239               // already-processed batch).
1240               callable.setCaching(1);
1241               values = getConnection().getRegionServerWithRetries(callable);
1242               callable.setCaching(this.caching);
1243               skipFirst = false;
1244             }
1245             // Server returns a null values if scanning is to stop.  Else,
1246             // returns an empty array if scanning is to go on and we've just
1247             // exhausted current region.
1248             values = getConnection().getRegionServerWithRetries(callable);
1249           } catch (DoNotRetryIOException e) {
1250             if (e instanceof UnknownScannerException) {
1251               long timeout = lastNext + scannerTimeout;
1252               // If we are over the timeout, throw this exception to the client
1253               // Else, it's because the region moved and we used the old id
1254               // against the new region server; reset the scanner.
1255               if (timeout < System.currentTimeMillis()) {
1256                 long elapsed = System.currentTimeMillis() - lastNext;
1257                 ScannerTimeoutException ex = new ScannerTimeoutException(
1258                     elapsed + "ms passed since the last invocation, " +
1259                         "timeout is currently set to " + scannerTimeout);
1260                 ex.initCause(e);
1261                 throw ex;
1262               }
1263             } else {
1264               Throwable cause = e.getCause();
1265               if (cause == null || (!(cause instanceof NotServingRegionException)
1266                   && !(cause instanceof RegionServerStoppedException))) {
1267                 throw e;
1268               }
1269             }
1270             // Else, its signal from depths of ScannerCallable that we got an
1271             // NSRE on a next and that we need to reset the scanner.
1272             if (this.lastResult != null) {
1273               this.scan.setStartRow(this.lastResult.getRow());
1274               // Skip first row returned.  We already let it out on previous
1275               // invocation.
1276               skipFirst = true;
1277             }
1278             // Clear region
1279             this.currentRegion = null;
1280             continue;
1281           }
1282           lastNext = System.currentTimeMillis();
1283           if (values != null && values.length > 0) {
1284             for (Result rs : values) {
1285               cache.add(rs);
1286               for (KeyValue kv : rs.raw()) {
1287                   remainingResultSize -= kv.heapSize();
1288               }
1289               countdown--;
1290               this.lastResult = rs;
1291             }
1292           }
1293           // Values == null means server-side filter has determined we must STOP
1294         } while (remainingResultSize > 0 && countdown > 0 && nextScanner(countdown, values == null));
1295       }
1296 
1297       if (cache.size() > 0) {
1298         return cache.poll();
1299       }
1300       return null;
1301     }
1302 
1303     /**
1304      * Get <param>nbRows</param> rows.
1305      * How many RPCs are made is determined by the {@link Scan#setCaching(int)}
1306      * setting (or hbase.client.scanner.caching in hbase-site.xml).
1307      * @param nbRows number of rows to return
1308      * @return Between zero and <param>nbRows</param> RowResults.  Scan is done
1309      * if returned array is of zero-length (We never return null).
1310      * @throws IOException
1311      */
1312     public Result [] next(int nbRows) throws IOException {
1313       // Collect values to be returned here
1314       ArrayList<Result> resultSets = new ArrayList<Result>(nbRows);
1315       for(int i = 0; i < nbRows; i++) {
1316         Result next = next();
1317         if (next != null) {
1318           resultSets.add(next);
1319         } else {
1320           break;
1321         }
1322       }
1323       return resultSets.toArray(new Result[resultSets.size()]);
1324     }
1325 
1326     public void close() {
1327       if (callable != null) {
1328         callable.setClose();
1329         try {
1330           getConnection().getRegionServerWithRetries(callable);
1331         } catch (IOException e) {
1332           // We used to catch this error, interpret, and rethrow. However, we
1333           // have since decided that it's not nice for a scanner's close to
1334           // throw exceptions. Chances are it was just an UnknownScanner
1335           // exception due to lease time out.
1336         }
1337         callable = null;
1338       }
1339       closed = true;
1340     }
1341 
1342     public Iterator<Result> iterator() {
1343       return new Iterator<Result>() {
1344         // The next RowResult, possibly pre-read
1345         Result next = null;
1346 
1347         // return true if there is another item pending, false if there isn't.
1348         // this method is where the actual advancing takes place, but you need
1349         // to call next() to consume it. hasNext() will only advance if there
1350         // isn't a pending next().
1351         public boolean hasNext() {
1352           if (next == null) {
1353             try {
1354               next = ClientScanner.this.next();
1355               return next != null;
1356             } catch (IOException e) {
1357               throw new RuntimeException(e);
1358             }
1359           }
1360           return true;
1361         }
1362 
1363         // get the pending next item and advance the iterator. returns null if
1364         // there is no next item.
1365         public Result next() {
1366           // since hasNext() does the real advancing, we call this to determine
1367           // if there is a next before proceeding.
1368           if (!hasNext()) {
1369             return null;
1370           }
1371 
1372           // if we get to here, then hasNext() has given us an item to return.
1373           // we want to return the item and then null out the next pointer, so
1374           // we use a temporary variable.
1375           Result temp = next;
1376           next = null;
1377           return temp;
1378         }
1379 
1380         public void remove() {
1381           throw new UnsupportedOperationException();
1382         }
1383       };
1384     }
1385   }
1386 
1387   /**
1388    * The pool is used for mutli requests for this HTable
1389    * @return the pool used for mutli
1390    */
1391   ExecutorService getPool() {
1392     return this.pool;
1393   }
1394 
1395   static class DaemonThreadFactory implements ThreadFactory {
1396     static final AtomicInteger poolNumber = new AtomicInteger(1);
1397         final ThreadGroup group;
1398         final AtomicInteger threadNumber = new AtomicInteger(1);
1399         final String namePrefix;
1400 
1401         DaemonThreadFactory() {
1402             SecurityManager s = System.getSecurityManager();
1403             group = (s != null)? s.getThreadGroup() :
1404                                  Thread.currentThread().getThreadGroup();
1405             namePrefix = "pool-" +
1406                           poolNumber.getAndIncrement() +
1407                          "-thread-";
1408         }
1409 
1410         public Thread newThread(Runnable r) {
1411             Thread t = new Thread(group, r,
1412                                   namePrefix + threadNumber.getAndIncrement(),
1413                                   0);
1414             if (!t.isDaemon()) {
1415               t.setDaemon(true);
1416             }
1417             if (t.getPriority() != Thread.NORM_PRIORITY) {
1418               t.setPriority(Thread.NORM_PRIORITY);
1419             }
1420             return t;
1421         }
1422   }
1423 
1424   /**
1425    * Enable or disable region cache prefetch for the table. It will be
1426    * applied for the given table's all HTable instances who share the same
1427    * connection. By default, the cache prefetch is enabled.
1428    * @param tableName name of table to configure.
1429    * @param enable Set to true to enable region cache prefetch. Or set to
1430    * false to disable it.
1431    * @throws IOException
1432    */
1433   public static void setRegionCachePrefetch(final byte[] tableName,
1434       final boolean enable) throws IOException {
1435     HConnectionManager.execute(new HConnectable<Void>(HBaseConfiguration
1436         .create()) {
1437       @Override
1438       public Void connect(HConnection connection) throws IOException {
1439         connection.setRegionCachePrefetch(tableName, enable);
1440         return null;
1441       }
1442     });
1443   }
1444 
1445   /**
1446    * Enable or disable region cache prefetch for the table. It will be
1447    * applied for the given table's all HTable instances who share the same
1448    * connection. By default, the cache prefetch is enabled.
1449    * @param conf The Configuration object to use.
1450    * @param tableName name of table to configure.
1451    * @param enable Set to true to enable region cache prefetch. Or set to
1452    * false to disable it.
1453    * @throws IOException
1454    */
1455   public static void setRegionCachePrefetch(final Configuration conf,
1456       final byte[] tableName, final boolean enable) throws IOException {
1457     HConnectionManager.execute(new HConnectable<Void>(conf) {
1458       @Override
1459       public Void connect(HConnection connection) throws IOException {
1460         connection.setRegionCachePrefetch(tableName, enable);
1461         return null;
1462       }
1463     });
1464   }
1465 
1466   /**
1467    * Check whether region cache prefetch is enabled or not for the table.
1468    * @param conf The Configuration object to use.
1469    * @param tableName name of table to check
1470    * @return true if table's region cache prefecth is enabled. Otherwise
1471    * it is disabled.
1472    * @throws IOException
1473    */
1474   public static boolean getRegionCachePrefetch(final Configuration conf,
1475       final byte[] tableName) throws IOException {
1476     return HConnectionManager.execute(new HConnectable<Boolean>(conf) {
1477       @Override
1478       public Boolean connect(HConnection connection) throws IOException {
1479         return connection.getRegionCachePrefetch(tableName);
1480       }
1481     });
1482   }
1483 
1484   /**
1485    * Check whether region cache prefetch is enabled or not for the table.
1486    * @param tableName name of table to check
1487    * @return true if table's region cache prefecth is enabled. Otherwise
1488    * it is disabled.
1489    * @throws IOException
1490    */
1491   public static boolean getRegionCachePrefetch(final byte[] tableName) throws IOException {
1492     return HConnectionManager.execute(new HConnectable<Boolean>(
1493         HBaseConfiguration.create()) {
1494       @Override
1495       public Boolean connect(HConnection connection) throws IOException {
1496         return connection.getRegionCachePrefetch(tableName);
1497       }
1498     });
1499  }
1500 
1501   /**
1502    * Explicitly clears the region cache to fetch the latest value from META.
1503    * This is a power user function: avoid unless you know the ramifications.
1504    */
1505   public void clearRegionCache() {
1506     this.connection.clearRegionCache();
1507   }
1508 
1509   /**
1510    * {@inheritDoc}
1511    */
1512   @Override
1513   public <T extends CoprocessorProtocol> T coprocessorProxy(
1514       Class<T> protocol, byte[] row) {
1515     return (T)Proxy.newProxyInstance(this.getClass().getClassLoader(),
1516         new Class[]{protocol},
1517         new ExecRPCInvoker(configuration,
1518             connection,
1519             protocol,
1520             tableName,
1521             row));
1522   }
1523 
1524   /**
1525    * {@inheritDoc}
1526    */
1527   @Override
1528   public <T extends CoprocessorProtocol, R> Map<byte[],R> coprocessorExec(
1529       Class<T> protocol, byte[] startKey, byte[] endKey,
1530       Batch.Call<T,R> callable)
1531       throws IOException, Throwable {
1532 
1533     final Map<byte[],R> results = new TreeMap<byte[],R>(
1534         Bytes.BYTES_COMPARATOR);
1535     coprocessorExec(protocol, startKey, endKey, callable,
1536         new Batch.Callback<R>(){
1537       public void update(byte[] region, byte[] row, R value) {
1538         results.put(region, value);
1539       }
1540     });
1541     return results;
1542   }
1543 
1544   /**
1545    * {@inheritDoc}
1546    */
1547   @Override
1548   public <T extends CoprocessorProtocol, R> void coprocessorExec(
1549       Class<T> protocol, byte[] startKey, byte[] endKey,
1550       Batch.Call<T,R> callable, Batch.Callback<R> callback)
1551       throws IOException, Throwable {
1552 
1553     // get regions covered by the row range
1554     List<byte[]> keys = getStartKeysInRange(startKey, endKey);
1555     connection.processExecs(protocol, keys, tableName, pool, callable,
1556         callback);
1557   }
1558 
1559   private List<byte[]> getStartKeysInRange(byte[] start, byte[] end)
1560   throws IOException {
1561     Pair<byte[][],byte[][]> startEndKeys = getStartEndKeys();
1562     byte[][] startKeys = startEndKeys.getFirst();
1563     byte[][] endKeys = startEndKeys.getSecond();
1564 
1565     if (start == null) {
1566       start = HConstants.EMPTY_START_ROW;
1567     }
1568     if (end == null) {
1569       end = HConstants.EMPTY_END_ROW;
1570     }
1571 
1572     List<byte[]> rangeKeys = new ArrayList<byte[]>();
1573     for (int i=0; i<startKeys.length; i++) {
1574       if (Bytes.compareTo(start, startKeys[i]) >= 0 ) {
1575         if (Bytes.equals(endKeys[i], HConstants.EMPTY_END_ROW) ||
1576             Bytes.compareTo(start, endKeys[i]) < 0) {
1577           rangeKeys.add(start);
1578         }
1579       } else if (Bytes.equals(end, HConstants.EMPTY_END_ROW) ||
1580           Bytes.compareTo(startKeys[i], end) <= 0) {
1581         rangeKeys.add(startKeys[i]);
1582       } else {
1583         break; // past stop
1584       }
1585     }
1586 
1587     return rangeKeys;
1588   }
1589 
1590   public void setOperationTimeout(int operationTimeout) {
1591     this.operationTimeout = operationTimeout;
1592   }
1593 
1594   public int getOperationTimeout() {
1595     return operationTimeout;
1596   }
1597 
1598 }