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 org.apache.commons.logging.Log;
23  import org.apache.commons.logging.LogFactory;
24  import org.apache.hadoop.conf.Configuration;
25  import org.apache.hadoop.hbase.DoNotRetryIOException;
26  import org.apache.hadoop.hbase.HBaseConfiguration;
27  import org.apache.hadoop.hbase.HConstants;
28  import org.apache.hadoop.hbase.HRegionInfo;
29  import org.apache.hadoop.hbase.HRegionLocation;
30  import org.apache.hadoop.hbase.HServerAddress;
31  import org.apache.hadoop.hbase.HTableDescriptor;
32  import org.apache.hadoop.hbase.KeyValue;
33  import org.apache.hadoop.hbase.NotServingRegionException;
34  import org.apache.hadoop.hbase.UnknownScannerException;
35  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
36  import org.apache.hadoop.hbase.util.Bytes;
37  import org.apache.hadoop.hbase.util.Pair;
38  import org.apache.hadoop.hbase.util.Writables;
39  
40  import java.io.IOException;
41  import java.util.ArrayList;
42  import java.util.Arrays;
43  import java.util.Iterator;
44  import java.util.LinkedList;
45  import java.util.List;
46  import java.util.Map;
47  import java.util.TreeMap;
48  import java.util.concurrent.ExecutorService;
49  import java.util.concurrent.LinkedBlockingQueue;
50  import java.util.concurrent.ThreadFactory;
51  import java.util.concurrent.ThreadPoolExecutor;
52  import java.util.concurrent.TimeUnit;
53  import java.util.concurrent.atomic.AtomicInteger;
54  
55  import java.io.DataInput;
56  import java.io.DataOutput;
57  
58  /**
59   * Used to communicate with a single HBase table.
60   * 
61   * This class is not thread safe for writes.
62   * Gets, puts, and deletes take out a row lock for the duration
63   * of their operation.  Scans (currently) do not respect
64   * row locking.
65   * 
66   * See {@link HBaseAdmin} to create, drop, list, enable and disable tables.
67   */
68  public class HTable implements HTableInterface {
69    private final HConnection connection;
70    private final byte [] tableName;
71    protected final int scannerTimeout;
72    private volatile Configuration configuration;
73    private final ArrayList<Put> writeBuffer = new ArrayList<Put>();
74    private long writeBufferSize;
75    private boolean autoFlush;
76    private long currentWriteBufferSize;
77    protected int scannerCaching;
78    private int maxKeyValueSize;
79  
80    private long maxScannerResultSize;
81  
82    /**
83     * Creates an object to access a HBase table.
84     *
85     * @param tableName Name of the table.
86     * @throws IOException if a remote or network exception occurs
87     */
88    public HTable(final String tableName)
89    throws IOException {
90      this(HBaseConfiguration.create(), Bytes.toBytes(tableName));
91    }
92  
93    /**
94     * Creates an object to access a HBase table.
95     *
96     * @param tableName Name of the table.
97     * @throws IOException if a remote or network exception occurs
98     */
99    public HTable(final byte [] tableName)
100   throws IOException {
101     this(HBaseConfiguration.create(), tableName);
102   }
103 
104   /**
105    * Creates an object to access a HBase table.
106    *
107    * @param conf Configuration object to use.
108    * @param tableName Name of the table.
109    * @throws IOException if a remote or network exception occurs
110    */
111   public HTable(Configuration conf, final String tableName)
112   throws IOException {
113     this(conf, Bytes.toBytes(tableName));
114   }
115 
116 
117   /**
118    * Creates an object to access a HBase table.
119    *
120    * @param conf Configuration object to use.
121    * @param tableName Name of the table.
122    * @throws IOException if a remote or network exception occurs
123    */
124   public HTable(Configuration conf, final byte [] tableName)
125   throws IOException {
126     this.tableName = tableName;
127     if (conf == null) {
128       this.scannerTimeout = 0;
129       this.connection = null;
130       return;
131     }
132     this.connection = HConnectionManager.getConnection(conf);
133     this.scannerTimeout =
134       (int) conf.getLong(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
135     this.configuration = conf;
136     this.connection.locateRegion(tableName, HConstants.EMPTY_START_ROW);
137     this.writeBufferSize = conf.getLong("hbase.client.write.buffer", 2097152);
138     this.autoFlush = true;
139     this.currentWriteBufferSize = 0;
140     this.scannerCaching = conf.getInt("hbase.client.scanner.caching", 1);
141 
142     this.maxScannerResultSize = conf.getLong(
143       HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
144       HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
145     this.maxKeyValueSize = conf.getInt("hbase.client.keyvalue.maxsize", -1);
146 
147     int nrHRS = getCurrentNrHRS();
148     if (nrHRS == 0) {
149       // No servers running -- set default of 10 threads.
150       nrHRS = 10;
151     }
152     int nrThreads = conf.getInt("hbase.htable.threads.max", nrHRS);
153 
154     // Unfortunately Executors.newCachedThreadPool does not allow us to
155     // set the maximum size of the pool, so we have to do it ourselves.
156     this.pool = new ThreadPoolExecutor(0, nrThreads,
157         60, TimeUnit.SECONDS,
158         new LinkedBlockingQueue<Runnable>(),
159         new DaemonThreadFactory());
160   }
161 
162   public Configuration getConfiguration() {
163     return configuration;
164   }
165 
166   /**
167    * TODO Might want to change this to public, would be nice if the number
168    * of threads would automatically change when servers were added and removed
169    * @return the number of region servers that are currently running
170    * @throws IOException if a remote or network exception occurs
171    */
172   int getCurrentNrHRS() throws IOException {
173     return HConnectionManager
174       .getClientZooKeeperWatcher(this.configuration)
175       .getZooKeeperWrapper()
176       .getRSDirectoryCount();
177   }
178 
179   // For multiput
180   private ExecutorService pool;
181 
182   /**
183    * Tells whether or not a table is enabled or not.
184    * @param tableName Name of table to check.
185    * @return {@code true} if table is online.
186    * @throws IOException if a remote or network exception occurs
187    */
188   public static boolean isTableEnabled(String tableName) throws IOException {
189     return isTableEnabled(Bytes.toBytes(tableName));
190   }
191 
192   /**
193    * Tells whether or not a table is enabled or not.
194    * @param tableName Name of table to check.
195    * @return {@code true} if table is online.
196    * @throws IOException if a remote or network exception occurs
197    */
198   public static boolean isTableEnabled(byte[] tableName) throws IOException {
199     return isTableEnabled(HBaseConfiguration.create(), tableName);
200   }
201 
202   /**
203    * Tells whether or not a table is enabled or not.
204    * @param conf The Configuration object to use.
205    * @param tableName Name of table to check.
206    * @return {@code true} if table is online.
207    * @throws IOException if a remote or network exception occurs
208    */
209   public static boolean isTableEnabled(Configuration conf, String tableName)
210   throws IOException {
211     return isTableEnabled(conf, Bytes.toBytes(tableName));
212   }
213 
214   /**
215    * Tells whether or not a table is enabled or not.
216    * @param conf The Configuration object to use.
217    * @param tableName Name of table to check.
218    * @return {@code true} if table is online.
219    * @throws IOException if a remote or network exception occurs
220    */
221   public static boolean isTableEnabled(Configuration conf, byte[] tableName)
222   throws IOException {
223     return HConnectionManager.getConnection(conf).isTableEnabled(tableName);
224   }
225 
226   /**
227    * Find region location hosting passed row using cached info
228    * @param row Row to find.
229    * @return The location of the given row.
230    * @throws IOException if a remote or network exception occurs
231    */
232   public HRegionLocation getRegionLocation(final String row)
233   throws IOException {
234     return connection.getRegionLocation(tableName, Bytes.toBytes(row), false);
235   }
236 
237   /**
238    * Finds the region on which the given row is being served.
239    * @param row Row to find.
240    * @return Location of the row.
241    * @throws IOException if a remote or network exception occurs
242    */
243   public HRegionLocation getRegionLocation(final byte [] row)
244   throws IOException {
245     return connection.getRegionLocation(tableName, row, false);
246   }
247 
248   public byte [] getTableName() {
249     return this.tableName;
250   }
251 
252   /**
253    * <em>INTERNAL</em> Used by unit tests and tools to do low-level
254    * manipulations.
255    * @return An HConnection instance.
256    */
257   // TODO(tsuna): Remove this.  Unit tests shouldn't require public helpers.
258   public HConnection getConnection() {
259     return this.connection;
260   }
261 
262   /**
263    * Gets the number of rows that a scanner will fetch at once.
264    * <p>
265    * The default value comes from {@code hbase.client.scanner.caching}.
266    */
267   public int getScannerCaching() {
268     return scannerCaching;
269   }
270 
271   /**
272    * Sets the number of rows that a scanner will fetch at once.
273    * <p>
274    * This will override the value specified by
275    * {@code hbase.client.scanner.caching}.
276    * Increasing this value will reduce the amount of work needed each time
277    * {@code next()} is called on a scanner, at the expense of memory use
278    * (since more rows will need to be maintained in memory by the scanners).
279    * @param scannerCaching the number of rows a scanner will fetch at once.
280    */
281   public void setScannerCaching(int scannerCaching) {
282     this.scannerCaching = scannerCaching;
283   }
284 
285   public HTableDescriptor getTableDescriptor() throws IOException {
286     return new UnmodifyableHTableDescriptor(
287       this.connection.getHTableDescriptor(this.tableName));
288   }
289 
290   /**
291    * Gets the starting row key for every region in the currently open table.
292    * <p>
293    * This is mainly useful for the MapReduce integration.
294    * @return Array of region starting row keys
295    * @throws IOException if a remote or network exception occurs
296    */
297   public byte [][] getStartKeys() throws IOException {
298     return getStartEndKeys().getFirst();
299   }
300 
301   /**
302    * Gets the ending row key for every region in the currently open table.
303    * <p>
304    * This is mainly useful for the MapReduce integration.
305    * @return Array of region ending row keys
306    * @throws IOException if a remote or network exception occurs
307    */
308   public byte[][] getEndKeys() throws IOException {
309     return getStartEndKeys().getSecond();
310   }
311 
312   /**
313    * Gets the starting and ending row keys for every region in the currently
314    * open table.
315    * <p>
316    * This is mainly useful for the MapReduce integration.
317    * @return Pair of arrays of region starting and ending row keys
318    * @throws IOException if a remote or network exception occurs
319    */
320   @SuppressWarnings("unchecked")
321   public Pair<byte[][],byte[][]> getStartEndKeys() throws IOException {
322     final List<byte[]> startKeyList = new ArrayList<byte[]>();
323     final List<byte[]> endKeyList = new ArrayList<byte[]>();
324     MetaScannerVisitor visitor = new MetaScannerVisitor() {
325       public boolean processRow(Result rowResult) throws IOException {
326         HRegionInfo info = Writables.getHRegionInfo(
327             rowResult.getValue(HConstants.CATALOG_FAMILY,
328                 HConstants.REGIONINFO_QUALIFIER));
329         if (Bytes.equals(info.getTableDesc().getName(), getTableName())) {
330           if (!(info.isOffline() || info.isSplit())) {
331             startKeyList.add(info.getStartKey());
332             endKeyList.add(info.getEndKey());
333           }
334         }
335         return true;
336       }
337     };
338     MetaScanner.metaScan(configuration, visitor, this.tableName);
339     return new Pair(startKeyList.toArray(new byte[startKeyList.size()][]),
340                 endKeyList.toArray(new byte[endKeyList.size()][]));
341   }
342 
343   /**
344    * Gets all the regions and their address for this table.
345    * <p>
346    * This is mainly useful for the MapReduce integration.
347    * @return A map of HRegionInfo with it's server address
348    * @throws IOException if a remote or network exception occurs
349    */
350   public Map<HRegionInfo, HServerAddress> getRegionsInfo() throws IOException {
351     final Map<HRegionInfo, HServerAddress> regionMap =
352       new TreeMap<HRegionInfo, HServerAddress>();
353 
354     MetaScannerVisitor visitor = new MetaScannerVisitor() {
355       public boolean processRow(Result rowResult) throws IOException {
356         HRegionInfo info = Writables.getHRegionInfo(
357             rowResult.getValue(HConstants.CATALOG_FAMILY,
358                 HConstants.REGIONINFO_QUALIFIER));
359 
360         if (!(Bytes.equals(info.getTableDesc().getName(), getTableName()))) {
361           return false;
362         }
363 
364         HServerAddress server = new HServerAddress();
365         byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
366             HConstants.SERVER_QUALIFIER);
367         if (value != null && value.length > 0) {
368           String address = Bytes.toString(value);
369           server = new HServerAddress(address);
370         }
371 
372         if (!(info.isOffline() || info.isSplit())) {
373           regionMap.put(new UnmodifyableHRegionInfo(info), server);
374         }
375         return true;
376       }
377 
378     };
379     MetaScanner.metaScan(configuration, visitor, tableName);
380     return regionMap;
381   }
382 
383   /**
384    * Save the passed region information and the table's regions
385    * cache.
386    * <p>
387    * This is mainly useful for the MapReduce integration. You can call
388    * {@link #deserializeRegionInfo deserializeRegionInfo}
389    * to deserialize regions information from a
390    * {@link DataInput}, then call this method to load them to cache.
391    *
392    * <pre>
393    * {@code
394    * HTable t1 = new HTable("foo");
395    * FileInputStream fis = new FileInputStream("regions.dat");
396    * DataInputStream dis = new DataInputStream(fis);
397    *
398    * Map<HRegionInfo, HServerAddress> hm = t1.deserializeRegionInfo(dis);
399    * t1.prewarmRegionCache(hm);
400    * }
401    * </pre>
402    * @param regionMap This piece of regions information will be loaded
403    * to region cache.
404    */
405   public void prewarmRegionCache(Map<HRegionInfo, HServerAddress> regionMap) {
406     this.connection.prewarmRegionCache(this.getTableName(), regionMap);
407   }
408 
409   /**
410    * Serialize the regions information of this table and output
411    * to <code>out</code>.
412    * <p>
413    * This is mainly useful for the MapReduce integration. A client could
414    * perform a large scan for all the regions for the table, serialize the
415    * region info to a file. MR job can ship a copy of the meta for the table in
416    * the DistributedCache.
417    * <pre>
418    * {@code
419    * FileOutputStream fos = new FileOutputStream("regions.dat");
420    * DataOutputStream dos = new DataOutputStream(fos);
421    * table.serializeRegionInfo(dos);
422    * dos.flush();
423    * dos.close();
424    * }
425    * </pre>
426    * @param out {@link DataOutput} to serialize this object into.
427    * @throws IOException if a remote or network exception occurs
428    */
429   public void serializeRegionInfo(DataOutput out) throws IOException {
430     Map<HRegionInfo, HServerAddress> allRegions = this.getRegionsInfo();
431     // first, write number of regions
432     out.writeInt(allRegions.size());
433     for (Map.Entry<HRegionInfo, HServerAddress> es : allRegions.entrySet()) {
434       es.getKey().write(out);
435       es.getValue().write(out);
436     }
437   }
438 
439   /**
440    * Read from <code>in</code> and deserialize the regions information.
441    *
442    * <p>It behaves similarly as {@link #getRegionsInfo getRegionsInfo}, except
443    * that it loads the region map from a {@link DataInput} object.
444    *
445    * <p>It is supposed to be followed immediately by  {@link
446    * #prewarmRegionCache prewarmRegionCache}.
447    *
448    * <p>
449    * Please refer to {@link #prewarmRegionCache prewarmRegionCache} for usage.
450    *
451    * @param in {@link DataInput} object.
452    * @return A map of HRegionInfo with its server address.
453    * @throws IOException if an I/O exception occurs.
454    */
455   public Map<HRegionInfo, HServerAddress> deserializeRegionInfo(DataInput in)
456   throws IOException {
457     final Map<HRegionInfo, HServerAddress> allRegions =
458       new TreeMap<HRegionInfo, HServerAddress>();
459 
460     // the first integer is expected to be the size of records
461     int regionsCount = in.readInt();
462     for (int i = 0; i < regionsCount; ++i) {
463       HRegionInfo hri = new HRegionInfo();
464       hri.readFields(in);
465       HServerAddress hsa = new HServerAddress();
466       hsa.readFields(in);
467       allRegions.put(hri, hsa);
468     }
469     return allRegions;
470   }
471 
472    public Result getRowOrBefore(final byte[] row, final byte[] family)
473    throws IOException {
474      return connection.getRegionServerWithRetries(
475          new ServerCallable<Result>(connection, tableName, row) {
476        public Result call() throws IOException {
477          return server.getClosestRowBefore(location.getRegionInfo().getRegionName(),
478            row, family);
479        }
480      });
481    }
482 
483   public ResultScanner getScanner(final Scan scan) throws IOException {
484     ClientScanner s = new ClientScanner(scan);
485     s.initialize();
486     return s;
487   }
488 
489   public ResultScanner getScanner(byte [] family) throws IOException {
490     Scan scan = new Scan();
491     scan.addFamily(family);
492     return getScanner(scan);
493   }
494 
495   public ResultScanner getScanner(byte [] family, byte [] qualifier)
496   throws IOException {
497     Scan scan = new Scan();
498     scan.addColumn(family, qualifier);
499     return getScanner(scan);
500   }
501 
502   public Result get(final Get get) throws IOException {
503     return connection.getRegionServerWithRetries(
504         new ServerCallable<Result>(connection, tableName, get.getRow()) {
505           public Result call() throws IOException {
506             return server.get(location.getRegionInfo().getRegionName(), get);
507           }
508         }
509     );
510   }
511 
512   public void delete(final Delete delete)
513   throws IOException {
514     connection.getRegionServerWithRetries(
515         new ServerCallable<Boolean>(connection, tableName, delete.getRow()) {
516           public Boolean call() throws IOException {
517             server.delete(location.getRegionInfo().getRegionName(), delete);
518             return null; // FindBugs NP_BOOLEAN_RETURN_NULL
519           }
520         }
521     );
522   }
523 
524   public void delete(final List<Delete> deletes)
525   throws IOException {
526     int last = 0;
527     try {
528       last = connection.processBatchOfDeletes(deletes, this.tableName);
529     } finally {
530       deletes.subList(0, last).clear();
531     }
532   }
533 
534   public void put(final Put put) throws IOException {
535     doPut(Arrays.asList(put));
536   }
537 
538   public void put(final List<Put> puts) throws IOException {
539     doPut(puts);
540   }
541 
542   private void doPut(final List<Put> puts) throws IOException {
543     for (Put put : puts) {
544       validatePut(put);
545       writeBuffer.add(put);
546       currentWriteBufferSize += put.heapSize();
547     }
548     if (autoFlush || currentWriteBufferSize > writeBufferSize) {
549       flushCommits();
550     }
551   }
552 
553   public long incrementColumnValue(final byte [] row, final byte [] family,
554       final byte [] qualifier, final long amount)
555   throws IOException {
556     return incrementColumnValue(row, family, qualifier, amount, true);
557   }
558 
559   @SuppressWarnings({"ThrowableInstanceNeverThrown"})
560   public long incrementColumnValue(final byte [] row, final byte [] family,
561       final byte [] qualifier, final long amount, final boolean writeToWAL)
562   throws IOException {
563     NullPointerException npe = null;
564     if (row == null) {
565       npe = new NullPointerException("row is null");
566     } else if (family == null) {
567       npe = new NullPointerException("column is null");
568     }
569     if (npe != null) {
570       throw new IOException(
571           "Invalid arguments to incrementColumnValue", npe);
572     }
573     return connection.getRegionServerWithRetries(
574         new ServerCallable<Long>(connection, tableName, row) {
575           public Long call() throws IOException {
576             return server.incrementColumnValue(
577                 location.getRegionInfo().getRegionName(), row, family,
578                 qualifier, amount, writeToWAL);
579           }
580         }
581     );
582   }
583 
584   /**
585    * Atomically checks if a row/family/qualifier value match the expectedValue.
586    * If it does, it adds the put.  If value == null, checks for non-existence
587    * of the value.
588    *
589    * @param row to check
590    * @param family column family
591    * @param qualifier column qualifier
592    * @param value the expected value
593    * @param put put to execute if value matches.
594    * @throws IOException
595    * @return true if the new put was execute, false otherwise
596    */
597   public boolean checkAndPut(final byte [] row,
598       final byte [] family, final byte [] qualifier, final byte [] value,
599       final Put put)
600   throws IOException {
601     return connection.getRegionServerWithRetries(
602         new ServerCallable<Boolean>(connection, tableName, row) {
603           public Boolean call() throws IOException {
604             return server.checkAndPut(location.getRegionInfo().getRegionName(),
605                 row, family, qualifier, value, put) ? Boolean.TRUE : Boolean.FALSE;
606           }
607         }
608     );
609   }
610 
611   /**
612    * Atomically checks if a row/family/qualifier value match the expectedValue.
613    * If it does, it adds the delete.  If value == null, checks for non-existence
614    * of the value.
615    *
616    * @param row to check
617    * @param family column family
618    * @param qualifier column qualifier
619    * @param value the expected value
620    * @param delete delete to execute if value matches.
621    * @throws IOException
622    * @return true if the new delete was executed, false otherwise
623    */
624   public boolean checkAndDelete(final byte [] row,
625       final byte [] family, final byte [] qualifier, final byte [] value,
626       final Delete delete)
627   throws IOException {
628     return connection.getRegionServerWithRetries(
629         new ServerCallable<Boolean>(connection, tableName, row) {
630           public Boolean call() throws IOException {
631             return server.checkAndDelete(
632                 location.getRegionInfo().getRegionName(),
633                 row, family, qualifier, value, delete) 
634             ? Boolean.TRUE : Boolean.FALSE;
635           }
636         }
637     );
638   }
639 
640   /**
641    * Test for the existence of columns in the table, as specified in the Get.<p>
642    *
643    * This will return true if the Get matches one or more keys, false if not.<p>
644    *
645    * This is a server-side call so it prevents any data from being transfered
646    * to the client.
647    * @param get param to check for
648    * @return true if the specified Get matches one or more keys, false if not
649    * @throws IOException
650    */
651   public boolean exists(final Get get) throws IOException {
652     return connection.getRegionServerWithRetries(
653         new ServerCallable<Boolean>(connection, tableName, get.getRow()) {
654           public Boolean call() throws IOException {
655             return server.
656                 exists(location.getRegionInfo().getRegionName(), get);
657           }
658         }
659     );
660   }
661 
662   public void flushCommits() throws IOException {
663     try {
664       connection.processBatchOfPuts(writeBuffer,
665           tableName, pool);
666     } finally {
667       // the write buffer was adjusted by processBatchOfPuts
668       currentWriteBufferSize = 0;
669       for (Put aPut : writeBuffer) {
670         currentWriteBufferSize += aPut.heapSize();
671       }
672     }
673   }
674 
675   public void close() throws IOException{
676     flushCommits();
677   }
678 
679   // validate for well-formedness
680   private void validatePut(final Put put) throws IllegalArgumentException{
681     if (put.isEmpty()) {
682       throw new IllegalArgumentException("No columns to insert");
683     }
684     if (maxKeyValueSize > 0) {
685       for (List<KeyValue> list : put.getFamilyMap().values()) {
686         for (KeyValue kv : list) {
687           if (kv.getLength() > maxKeyValueSize) {
688             throw new IllegalArgumentException("KeyValue size too large");
689           }
690         }
691       }
692     }
693   }
694 
695   public RowLock lockRow(final byte [] row)
696   throws IOException {
697     return connection.getRegionServerWithRetries(
698       new ServerCallable<RowLock>(connection, tableName, row) {
699         public RowLock call() throws IOException {
700           long lockId =
701               server.lockRow(location.getRegionInfo().getRegionName(), row);
702           return new RowLock(row,lockId);
703         }
704       }
705     );
706   }
707 
708   public void unlockRow(final RowLock rl)
709   throws IOException {
710     connection.getRegionServerWithRetries(
711       new ServerCallable<Boolean>(connection, tableName, rl.getRow()) {
712         public Boolean call() throws IOException {
713           server.unlockRow(location.getRegionInfo().getRegionName(),
714               rl.getLockId());
715           return null; // FindBugs NP_BOOLEAN_RETURN_NULL
716         }
717       }
718     );
719   }
720 
721   public boolean isAutoFlush() {
722     return autoFlush;
723   }
724 
725   /**
726    * Turns 'auto-flush' on or off.
727    * <p>
728    * When enabled (default), {@link Put} operations don't get buffered/delayed
729    * and are immediately executed.  This is slower but safer.
730    * <p>
731    * Turning this off means that multiple {@link Put}s will be accepted before
732    * any RPC is actually sent to do the write operations.  If the application
733    * dies before pending writes get flushed to HBase, data will be lost.
734    * Other side effects may include the fact that the application thinks a
735    * {@link Put} was executed successfully whereas it was in fact only
736    * buffered and the operation may fail when attempting to flush all pending
737    * writes.  In that case though, the code will retry the failed {@link Put}
738    * upon its next attempt to flush the buffer.
739    *
740    * @param autoFlush Whether or not to enable 'auto-flush'.
741    * @see #flushCommits
742    */
743   public void setAutoFlush(boolean autoFlush) {
744     this.autoFlush = autoFlush;
745   }
746 
747   /**
748    * Returns the maximum size in bytes of the write buffer for this HTable.
749    * <p>
750    * The default value comes from the configuration parameter
751    * {@code hbase.client.write.buffer}.
752    * @return The size of the write buffer in bytes.
753    */
754   public long getWriteBufferSize() {
755     return writeBufferSize;
756   }
757 
758   /**
759    * Sets the size of the buffer in bytes.
760    * <p>
761    * If the new size is less than the current amount of data in the
762    * write buffer, the buffer gets flushed.
763    * @param writeBufferSize The new write buffer size, in bytes.
764    * @throws IOException if a remote or network exception occurs.
765    */
766   public void setWriteBufferSize(long writeBufferSize) throws IOException {
767     this.writeBufferSize = writeBufferSize;
768     if(currentWriteBufferSize > writeBufferSize) {
769       flushCommits();
770     }
771   }
772 
773   /**
774    * Returns the write buffer.
775    * @return The current write buffer.
776    */
777   public ArrayList<Put> getWriteBuffer() {
778     return writeBuffer;
779   }
780 
781   /**
782    * Implements the scanner interface for the HBase client.
783    * If there are multiple regions in a table, this scanner will iterate
784    * through them all.
785    */
786   protected class ClientScanner implements ResultScanner {
787     private final Log CLIENT_LOG = LogFactory.getLog(this.getClass());
788     // HEADSUP: The scan internal start row can change as we move through table.
789     private Scan scan;
790     private boolean closed = false;
791     // Current region scanner is against.  Gets cleared if current region goes
792     // wonky: e.g. if it splits on us.
793     private HRegionInfo currentRegion = null;
794     private ScannerCallable callable = null;
795     private final LinkedList<Result> cache = new LinkedList<Result>();
796     private final int caching;
797     private long lastNext;
798     // Keep lastResult returned successfully in case we have to reset scanner.
799     private Result lastResult = null;
800 
801     protected ClientScanner(final Scan scan) {
802       if (CLIENT_LOG.isDebugEnabled()) {
803         CLIENT_LOG.debug("Creating scanner over "
804             + Bytes.toString(getTableName())
805             + " starting at key '" + Bytes.toStringBinary(scan.getStartRow()) + "'");
806       }
807       this.scan = scan;
808       this.lastNext = System.currentTimeMillis();
809 
810       // Use the caching from the Scan.  If not set, use the default cache setting for this table.
811       if (this.scan.getCaching() > 0) {
812         this.caching = this.scan.getCaching();
813       } else {
814         this.caching = HTable.this.scannerCaching;
815       }
816 
817       // Removed filter validation.  We have a new format now, only one of all
818       // the current filters has a validate() method.  We can add it back,
819       // need to decide on what we're going to do re: filter redesign.
820       // Need, at the least, to break up family from qualifier as separate
821       // checks, I think it's important server-side filters are optimal in that
822       // respect.
823     }
824 
825     public void initialize() throws IOException {
826       nextScanner(this.caching, false);
827     }
828 
829     protected Scan getScan() {
830       return scan;
831     }
832 
833     protected long getTimestamp() {
834       return lastNext;
835     }
836 
837     // returns true if the passed region endKey
838     private boolean checkScanStopRow(final byte [] endKey) {
839       if (this.scan.getStopRow().length > 0) {
840         // there is a stop row, check to see if we are past it.
841         byte [] stopRow = scan.getStopRow();
842         int cmp = Bytes.compareTo(stopRow, 0, stopRow.length,
843           endKey, 0, endKey.length);
844         if (cmp <= 0) {
845           // stopRow <= endKey (endKey is equals to or larger than stopRow)
846           // This is a stop.
847           return true;
848         }
849       }
850       return false; //unlikely.
851     }
852 
853     /*
854      * Gets a scanner for the next region.  If this.currentRegion != null, then
855      * we will move to the endrow of this.currentRegion.  Else we will get
856      * scanner at the scan.getStartRow().  We will go no further, just tidy
857      * up outstanding scanners, if <code>currentRegion != null</code> and
858      * <code>done</code> is true.
859      * @param nbRows
860      * @param done Server-side says we're done scanning.
861      */
862     private boolean nextScanner(int nbRows, final boolean done)
863     throws IOException {
864       // Close the previous scanner if it's open
865       if (this.callable != null) {
866         this.callable.setClose();
867         getConnection().getRegionServerWithRetries(callable);
868         this.callable = null;
869       }
870 
871       // Where to start the next scanner
872       byte [] localStartKey;
873 
874       // if we're at end of table, close and return false to stop iterating
875       if (this.currentRegion != null) {
876         byte [] endKey = this.currentRegion.getEndKey();
877         if (endKey == null ||
878             Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) ||
879             checkScanStopRow(endKey) ||
880             done) {
881           close();
882           if (CLIENT_LOG.isDebugEnabled()) {
883             CLIENT_LOG.debug("Finished with scanning at " + this.currentRegion);
884           }
885           return false;
886         }
887         localStartKey = endKey;
888         if (CLIENT_LOG.isDebugEnabled()) {
889           CLIENT_LOG.debug("Finished with region " + this.currentRegion);
890         }
891       } else {
892         localStartKey = this.scan.getStartRow();
893       }
894 
895       if (CLIENT_LOG.isDebugEnabled()) {
896         CLIENT_LOG.debug("Advancing internal scanner to startKey at '" +
897           Bytes.toStringBinary(localStartKey) + "'");
898       }
899       try {
900         callable = getScannerCallable(localStartKey, nbRows);
901         // Open a scanner on the region server starting at the
902         // beginning of the region
903         getConnection().getRegionServerWithRetries(callable);
904         this.currentRegion = callable.getHRegionInfo();
905       } catch (IOException e) {
906         close();
907         throw e;
908       }
909       return true;
910     }
911 
912     protected ScannerCallable getScannerCallable(byte [] localStartKey,
913         int nbRows) {
914       scan.setStartRow(localStartKey);
915       ScannerCallable s = new ScannerCallable(getConnection(),
916         getTableName(), scan);
917       s.setCaching(nbRows);
918       return s;
919     }
920 
921     public Result next() throws IOException {
922       // If the scanner is closed but there is some rows left in the cache,
923       // it will first empty it before returning null
924       if (cache.size() == 0 && this.closed) {
925         return null;
926       }
927       if (cache.size() == 0) {
928         Result [] values = null;
929         long remainingResultSize = maxScannerResultSize;
930         int countdown = this.caching;
931         // We need to reset it if it's a new callable that was created
932         // with a countdown in nextScanner
933         callable.setCaching(this.caching);
934         // This flag is set when we want to skip the result returned.  We do
935         // this when we reset scanner because it split under us.
936         boolean skipFirst = false;
937         do {
938           try {
939             // Server returns a null values if scanning is to stop.  Else,
940             // returns an empty array if scanning is to go on and we've just
941             // exhausted current region.
942             values = getConnection().getRegionServerWithRetries(callable);
943             if (skipFirst) {
944               skipFirst = false;
945               // Reget.
946               values = getConnection().getRegionServerWithRetries(callable);
947             }
948           } catch (DoNotRetryIOException e) {
949             if (e instanceof UnknownScannerException) {
950               long timeout = lastNext + scannerTimeout;
951               // If we are over the timeout, throw this exception to the client
952               // Else, it's because the region moved and we used the old id
953               // against the new region server; reset the scanner.
954               if (timeout < System.currentTimeMillis()) {
955                 long elapsed = System.currentTimeMillis() - lastNext;
956                 ScannerTimeoutException ex = new ScannerTimeoutException(
957                     elapsed + "ms passed since the last invocation, " +
958                         "timeout is currently set to " + scannerTimeout);
959                 ex.initCause(e);
960                 throw ex;
961               }
962             } else {
963               Throwable cause = e.getCause();
964               if (cause == null || !(cause instanceof NotServingRegionException)) {
965                 throw e;
966               }
967             }
968             // Else, its signal from depths of ScannerCallable that we got an
969             // NSRE on a next and that we need to reset the scanner.
970             if (this.lastResult != null) {
971               this.scan.setStartRow(this.lastResult.getRow());
972               // Skip first row returned.  We already let it out on previous
973               // invocation.
974               skipFirst = true;
975             }
976             // Clear region
977             this.currentRegion = null;
978             continue;
979           }
980           lastNext = System.currentTimeMillis();
981           if (values != null && values.length > 0) {
982             for (Result rs : values) {
983               cache.add(rs);
984               for (KeyValue kv : rs.raw()) {
985                   remainingResultSize -= kv.heapSize();
986               }
987               countdown--;
988               this.lastResult = rs;
989             }
990           }
991           // Values == null means server-side filter has determined we must STOP
992         } while (remainingResultSize > 0 && countdown > 0 && nextScanner(countdown, values == null));
993       }
994 
995       if (cache.size() > 0) {
996         return cache.poll();
997       }
998       return null;
999     }
1000 
1001     /**
1002      * Get <param>nbRows</param> rows.
1003      * How many RPCs are made is determined by the {@link Scan#setCaching(int)}
1004      * setting (or hbase.client.scanner.caching in hbase-site.xml).
1005      * @param nbRows number of rows to return
1006      * @return Between zero and <param>nbRows</param> RowResults.  Scan is done
1007      * if returned array is of zero-length (We never return null).
1008      * @throws IOException
1009      */
1010     public Result [] next(int nbRows) throws IOException {
1011       // Collect values to be returned here
1012       ArrayList<Result> resultSets = new ArrayList<Result>(nbRows);
1013       for(int i = 0; i < nbRows; i++) {
1014         Result next = next();
1015         if (next != null) {
1016           resultSets.add(next);
1017         } else {
1018           break;
1019         }
1020       }
1021       return resultSets.toArray(new Result[resultSets.size()]);
1022     }
1023 
1024     public void close() {
1025       if (callable != null) {
1026         callable.setClose();
1027         try {
1028           getConnection().getRegionServerWithRetries(callable);
1029         } catch (IOException e) {
1030           // We used to catch this error, interpret, and rethrow. However, we
1031           // have since decided that it's not nice for a scanner's close to
1032           // throw exceptions. Chances are it was just an UnknownScanner
1033           // exception due to lease time out.
1034         }
1035         callable = null;
1036       }
1037       closed = true;
1038     }
1039 
1040     public Iterator<Result> iterator() {
1041       return new Iterator<Result>() {
1042         // The next RowResult, possibly pre-read
1043         Result next = null;
1044 
1045         // return true if there is another item pending, false if there isn't.
1046         // this method is where the actual advancing takes place, but you need
1047         // to call next() to consume it. hasNext() will only advance if there
1048         // isn't a pending next().
1049         public boolean hasNext() {
1050           if (next == null) {
1051             try {
1052               next = ClientScanner.this.next();
1053               return next != null;
1054             } catch (IOException e) {
1055               throw new RuntimeException(e);
1056             }
1057           }
1058           return true;
1059         }
1060 
1061         // get the pending next item and advance the iterator. returns null if
1062         // there is no next item.
1063         public Result next() {
1064           // since hasNext() does the real advancing, we call this to determine
1065           // if there is a next before proceeding.
1066           if (!hasNext()) {
1067             return null;
1068           }
1069 
1070           // if we get to here, then hasNext() has given us an item to return.
1071           // we want to return the item and then null out the next pointer, so
1072           // we use a temporary variable.
1073           Result temp = next;
1074           next = null;
1075           return temp;
1076         }
1077 
1078         public void remove() {
1079           throw new UnsupportedOperationException();
1080         }
1081       };
1082     }
1083   }
1084 
1085   static class DaemonThreadFactory implements ThreadFactory {
1086     static final AtomicInteger poolNumber = new AtomicInteger(1);
1087         final ThreadGroup group;
1088         final AtomicInteger threadNumber = new AtomicInteger(1);
1089         final String namePrefix;
1090 
1091         DaemonThreadFactory() {
1092             SecurityManager s = System.getSecurityManager();
1093             group = (s != null)? s.getThreadGroup() :
1094                                  Thread.currentThread().getThreadGroup();
1095             namePrefix = "pool-" +
1096                           poolNumber.getAndIncrement() +
1097                          "-thread-";
1098         }
1099 
1100         public Thread newThread(Runnable r) {
1101             Thread t = new Thread(group, r,
1102                                   namePrefix + threadNumber.getAndIncrement(),
1103                                   0);
1104             if (!t.isDaemon())
1105                 t.setDaemon(true);
1106             if (t.getPriority() != Thread.NORM_PRIORITY)
1107                 t.setPriority(Thread.NORM_PRIORITY);
1108             return t;
1109         }
1110   }
1111 
1112   /**
1113    * Enable or disable region cache prefetch for the table. It will be
1114    * applied for the given table's all HTable instances who share the same
1115    * connection. By default, the cache prefetch is enabled.
1116    * @param tableName name of table to configure.
1117    * @param enable Set to true to enable region cache prefetch. Or set to
1118    * false to disable it.
1119    */
1120   public static void setRegionCachePrefetch(final byte[] tableName,
1121       boolean enable) {
1122     HConnectionManager.getConnection(HBaseConfiguration.create()).
1123     setRegionCachePrefetch(tableName, enable);
1124   }
1125 
1126   /**
1127    * Enable or disable region cache prefetch for the table. It will be
1128    * applied for the given table's all HTable instances who share the same
1129    * connection. By default, the cache prefetch is enabled.
1130    * @param conf The Configuration object to use.
1131    * @param tableName name of table to configure.
1132    * @param enable Set to true to enable region cache prefetch. Or set to
1133    * false to disable it.
1134    */
1135   public static void setRegionCachePrefetch(final Configuration conf,
1136       final byte[] tableName, boolean enable) {
1137     HConnectionManager.getConnection(conf).setRegionCachePrefetch(
1138         tableName, enable);
1139   }
1140 
1141   /**
1142    * Check whether region cache prefetch is enabled or not for the table.
1143    * @param conf The Configuration object to use.
1144    * @param tableName name of table to check
1145    * @return true if table's region cache prefecth is enabled. Otherwise
1146    * it is disabled.
1147    */
1148   public static boolean getRegionCachePrefetch(final Configuration conf,
1149       final byte[] tableName) {
1150     return HConnectionManager.getConnection(conf).getRegionCachePrefetch(
1151         tableName);
1152   }
1153 
1154   /**
1155    * Check whether region cache prefetch is enabled or not for the table.
1156    * @param tableName name of table to check
1157    * @return true if table's region cache prefecth is enabled. Otherwise
1158    * it is disabled.
1159    */
1160   public static boolean getRegionCachePrefetch(final byte[] tableName) {
1161     return HConnectionManager.getConnection(HBaseConfiguration.create()).
1162     getRegionCachePrefetch(tableName);
1163   }
1164 }