View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.rest.client;
21  
22  import java.io.IOException;
23  import java.io.InterruptedIOException;
24  import java.util.ArrayList;
25  import java.util.Collection;
26  import java.util.Iterator;
27  import java.util.List;
28  import java.util.Map;
29  import java.util.Set;
30  import java.util.TreeMap;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.hbase.Cell;
36  import org.apache.hadoop.hbase.CellUtil;
37  import org.apache.hadoop.hbase.HBaseConfiguration;
38  import org.apache.hadoop.hbase.HConstants;
39  import org.apache.hadoop.hbase.HTableDescriptor;
40  import org.apache.hadoop.hbase.KeyValue;
41  import org.apache.hadoop.hbase.TableName;
42  import org.apache.hadoop.hbase.classification.InterfaceAudience;
43  import org.apache.hadoop.hbase.classification.InterfaceStability;
44  import org.apache.hadoop.hbase.client.Append;
45  import org.apache.hadoop.hbase.client.Delete;
46  import org.apache.hadoop.hbase.client.Durability;
47  import org.apache.hadoop.hbase.client.Get;
48  import org.apache.hadoop.hbase.client.Increment;
49  import org.apache.hadoop.hbase.client.Put;
50  import org.apache.hadoop.hbase.client.Result;
51  import org.apache.hadoop.hbase.client.ResultScanner;
52  import org.apache.hadoop.hbase.client.Row;
53  import org.apache.hadoop.hbase.client.RowMutations;
54  import org.apache.hadoop.hbase.client.Scan;
55  import org.apache.hadoop.hbase.client.Table;
56  import org.apache.hadoop.hbase.client.coprocessor.Batch;
57  import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
58  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
59  import org.apache.hadoop.hbase.io.TimeRange;
60  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
61  import org.apache.hadoop.hbase.rest.Constants;
62  import org.apache.hadoop.hbase.rest.model.CellModel;
63  import org.apache.hadoop.hbase.rest.model.CellSetModel;
64  import org.apache.hadoop.hbase.rest.model.RowModel;
65  import org.apache.hadoop.hbase.rest.model.ScannerModel;
66  import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
67  import org.apache.hadoop.hbase.util.Bytes;
68  import org.apache.hadoop.util.StringUtils;
69  
70  import com.google.protobuf.Descriptors;
71  import com.google.protobuf.Message;
72  import com.google.protobuf.Service;
73  import com.google.protobuf.ServiceException;
74  
75  /**
76   * HTable interface to remote tables accessed via REST gateway
77   */
78  @InterfaceAudience.Public
79  @InterfaceStability.Stable
80  public class RemoteHTable implements Table {
81  
82    private static final Log LOG = LogFactory.getLog(RemoteHTable.class);
83  
84    final Client client;
85    final Configuration conf;
86    final byte[] name;
87    final int maxRetries;
88    final long sleepTime;
89  
90    @SuppressWarnings("rawtypes")
91    protected String buildRowSpec(final byte[] row, final Map familyMap,
92        final long startTime, final long endTime, final int maxVersions) {
93      StringBuffer sb = new StringBuffer();
94      sb.append('/');
95      sb.append(Bytes.toStringBinary(name));
96      sb.append('/');
97      sb.append(Bytes.toStringBinary(row));
98      Set families = familyMap.entrySet();
99      if (families != null) {
100       Iterator i = familyMap.entrySet().iterator();
101       sb.append('/');
102       while (i.hasNext()) {
103         Map.Entry e = (Map.Entry)i.next();
104         Collection quals = (Collection)e.getValue();
105         if (quals == null || quals.isEmpty()) {
106           // this is an unqualified family. append the family name and NO ':'
107           sb.append(Bytes.toStringBinary((byte[])e.getKey()));
108         } else {
109           Iterator ii = quals.iterator();
110           while (ii.hasNext()) {
111             sb.append(Bytes.toStringBinary((byte[])e.getKey()));
112             sb.append(':');
113             Object o = ii.next();
114             // Puts use byte[] but Deletes use KeyValue
115             if (o instanceof byte[]) {
116               sb.append(Bytes.toStringBinary((byte[])o));
117             } else if (o instanceof KeyValue) {
118               sb.append(Bytes.toStringBinary(((KeyValue)o).getQualifier()));
119             } else {
120               throw new RuntimeException("object type not handled");
121             }
122             if (ii.hasNext()) {
123               sb.append(',');
124             }
125           }
126         }
127         if (i.hasNext()) {
128           sb.append(',');
129         }
130       }
131     }
132     if (startTime >= 0 && endTime != Long.MAX_VALUE) {
133       sb.append('/');
134       sb.append(startTime);
135       if (startTime != endTime) {
136         sb.append(',');
137         sb.append(endTime);
138       }
139     } else if (endTime != Long.MAX_VALUE) {
140       sb.append('/');
141       sb.append(endTime);
142     }
143     if (maxVersions > 1) {
144       sb.append("?v=");
145       sb.append(maxVersions);
146     }
147     return sb.toString();
148   }
149 
150   protected String buildMultiRowSpec(final byte[][] rows, int maxVersions) {
151     StringBuilder sb = new StringBuilder();
152     sb.append('/');
153     sb.append(Bytes.toStringBinary(name));
154     sb.append("/multiget/");
155     if (rows == null || rows.length == 0) {
156       return sb.toString();
157     }
158     sb.append("?");
159     for(int i=0; i<rows.length; i++) {
160       byte[] rk = rows[i];
161       if (i != 0) {
162         sb.append('&');
163       }
164       sb.append("row=");
165       sb.append(Bytes.toStringBinary(rk));
166     }
167     sb.append("&v=");
168     sb.append(maxVersions);
169 
170     return sb.toString();
171   }
172 
173   protected Result[] buildResultFromModel(final CellSetModel model) {
174     List<Result> results = new ArrayList<Result>();
175     for (RowModel row: model.getRows()) {
176       List<Cell> kvs = new ArrayList<Cell>();
177       for (CellModel cell: row.getCells()) {
178         byte[][] split = KeyValue.parseColumn(cell.getColumn());
179         byte[] column = split[0];
180         byte[] qualifier = null;
181         if (split.length == 1) {
182           qualifier = HConstants.EMPTY_BYTE_ARRAY;
183         } else if (split.length == 2) {
184           qualifier = split[1];
185         } else {
186           throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
187         }
188         kvs.add(new KeyValue(row.getKey(), column, qualifier,
189           cell.getTimestamp(), cell.getValue()));
190       }
191       results.add(Result.create(kvs));
192     }
193     return results.toArray(new Result[results.size()]);
194   }
195 
196   protected CellSetModel buildModelFromPut(Put put) {
197     RowModel row = new RowModel(put.getRow());
198     long ts = put.getTimeStamp();
199     for (List<Cell> cells: put.getFamilyCellMap().values()) {
200       for (Cell cell: cells) {
201         row.addCell(new CellModel(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell),
202           ts != HConstants.LATEST_TIMESTAMP ? ts : cell.getTimestamp(),
203           CellUtil.cloneValue(cell)));
204       }
205     }
206     CellSetModel model = new CellSetModel();
207     model.addRow(row);
208     return model;
209   }
210 
211   /**
212    * Constructor
213    * @param client
214    * @param name
215    */
216   public RemoteHTable(Client client, String name) {
217     this(client, HBaseConfiguration.create(), Bytes.toBytes(name));
218   }
219 
220   /**
221    * Constructor
222    * @param client
223    * @param conf
224    * @param name
225    */
226   public RemoteHTable(Client client, Configuration conf, String name) {
227     this(client, conf, Bytes.toBytes(name));
228   }
229 
230   /**
231    * Constructor
232    * @param client
233    * @param conf
234    * @param name
235    */
236   public RemoteHTable(Client client, Configuration conf, byte[] name) {
237     this.client = client;
238     this.conf = conf;
239     this.name = name;
240     this.maxRetries = conf.getInt("hbase.rest.client.max.retries", 10);
241     this.sleepTime = conf.getLong("hbase.rest.client.sleep", 1000);
242   }
243 
244   public byte[] getTableName() {
245     return name.clone();
246   }
247 
248   @Override
249   public TableName getName() {
250     return TableName.valueOf(name);
251   }
252 
253   @Override
254   public Configuration getConfiguration() {
255     return conf;
256   }
257 
258   @Override
259   public HTableDescriptor getTableDescriptor() throws IOException {
260     StringBuilder sb = new StringBuilder();
261     sb.append('/');
262     sb.append(Bytes.toStringBinary(name));
263     sb.append('/');
264     sb.append("schema");
265     for (int i = 0; i < maxRetries; i++) {
266       Response response = client.get(sb.toString(), Constants.MIMETYPE_PROTOBUF);
267       int code = response.getCode();
268       switch (code) {
269       case 200:
270         TableSchemaModel schema = new TableSchemaModel();
271         schema.getObjectFromMessage(response.getBody());
272         return schema.getTableDescriptor();
273       case 509:
274         try {
275           Thread.sleep(sleepTime);
276         } catch (InterruptedException e) {
277           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
278         }
279         break;
280       default:
281         throw new IOException("schema request returned " + code);
282       }
283     }
284     throw new IOException("schema request timed out");
285   }
286 
287   @Override
288   public void close() throws IOException {
289     client.shutdown();
290   }
291 
292   @Override
293   public Result get(Get get) throws IOException {
294     TimeRange range = get.getTimeRange();
295     String spec = buildRowSpec(get.getRow(), get.getFamilyMap(),
296       range.getMin(), range.getMax(), get.getMaxVersions());
297     if (get.getFilter() != null) {
298       LOG.warn("filters not supported on gets");
299     }
300     Result[] results = getResults(spec);
301     if (results.length > 0) {
302       if (results.length > 1) {
303         LOG.warn("too many results for get (" + results.length + ")");
304       }
305       return results[0];
306     } else {
307       return new Result();
308     }
309   }
310 
311   @Override
312   public Result[] get(List<Get> gets) throws IOException {
313     byte[][] rows = new byte[gets.size()][];
314     int maxVersions = 1;
315     int count = 0;
316 
317     for(Get g:gets) {
318 
319       if ( count == 0 ) {
320         maxVersions = g.getMaxVersions();
321       } else if (g.getMaxVersions() != maxVersions) {
322         LOG.warn("MaxVersions on Gets do not match, using the first in the list ("+maxVersions+")");
323       }
324 
325       if (g.getFilter() != null) {
326         LOG.warn("filters not supported on gets");
327       }
328 
329       rows[count] = g.getRow();
330       count ++;
331     }
332 
333     String spec = buildMultiRowSpec(rows, maxVersions);
334 
335     return getResults(spec);
336   }
337 
338   private Result[] getResults(String spec) throws IOException {
339     for (int i = 0; i < maxRetries; i++) {
340       Response response = client.get(spec, Constants.MIMETYPE_PROTOBUF);
341       int code = response.getCode();
342       switch (code) {
343         case 200:
344           CellSetModel model = new CellSetModel();
345           model.getObjectFromMessage(response.getBody());
346           Result[] results = buildResultFromModel(model);
347           if ( results.length > 0) {
348             return results;
349           }
350           // fall through
351         case 404:
352           return new Result[0];
353 
354         case 509:
355           try {
356             Thread.sleep(sleepTime);
357           } catch (InterruptedException e) {
358             throw (InterruptedIOException)new InterruptedIOException().initCause(e);
359           }
360           break;
361         default:
362           throw new IOException("get request returned " + code);
363       }
364     }
365     throw new IOException("get request timed out");
366   }
367 
368   @Override
369   public boolean exists(Get get) throws IOException {
370     LOG.warn("exists() is really get(), just use get()");
371     Result result = get(get);
372     return (result != null && !(result.isEmpty()));
373   }
374 
375   /**
376    * exists(List) is really a list of get() calls. Just use get().
377    * @param gets list of Get to test for the existence
378    */
379   @Override
380   public boolean[] existsAll(List<Get> gets) throws IOException {
381     LOG.warn("exists(List<Get>) is really list of get() calls, just use get()");
382     boolean[] results = new boolean[gets.size()];
383     for (int i = 0; i < results.length; i++) {
384       results[i] = exists(gets.get(i));
385     }
386     return results;
387   }
388 
389   @Deprecated
390   public Boolean[] exists(List<Get> gets) throws IOException {
391     boolean[] results = existsAll(gets);
392     Boolean[] objectResults = new Boolean[results.length];
393     for (int i = 0; i < results.length; ++i) {
394       objectResults[i] = results[i];
395     }
396     return objectResults;
397   }
398 
399   @Override
400   public void put(Put put) throws IOException {
401     CellSetModel model = buildModelFromPut(put);
402     StringBuilder sb = new StringBuilder();
403     sb.append('/');
404     sb.append(Bytes.toStringBinary(name));
405     sb.append('/');
406     sb.append(Bytes.toStringBinary(put.getRow()));
407     for (int i = 0; i < maxRetries; i++) {
408       Response response = client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF,
409         model.createProtobufOutput());
410       int code = response.getCode();
411       switch (code) {
412       case 200:
413         return;
414       case 509:
415         try {
416           Thread.sleep(sleepTime);
417         } catch (InterruptedException e) {
418           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
419         }
420         break;
421       default:
422         throw new IOException("put request failed with " + code);
423       }
424     }
425     throw new IOException("put request timed out");
426   }
427 
428   @Override
429   public void put(List<Put> puts) throws IOException {
430     // this is a trick: The gateway accepts multiple rows in a cell set and
431     // ignores the row specification in the URI
432 
433     // separate puts by row
434     TreeMap<byte[],List<Cell>> map =
435       new TreeMap<byte[],List<Cell>>(Bytes.BYTES_COMPARATOR);
436     for (Put put: puts) {
437       byte[] row = put.getRow();
438       List<Cell> cells = map.get(row);
439       if (cells == null) {
440         cells = new ArrayList<Cell>();
441         map.put(row, cells);
442       }
443       for (List<Cell> l: put.getFamilyCellMap().values()) {
444         cells.addAll(l);
445       }
446     }
447 
448     // build the cell set
449     CellSetModel model = new CellSetModel();
450     for (Map.Entry<byte[], List<Cell>> e: map.entrySet()) {
451       RowModel row = new RowModel(e.getKey());
452       for (Cell cell: e.getValue()) {
453         row.addCell(new CellModel(cell));
454       }
455       model.addRow(row);
456     }
457 
458     // build path for multiput
459     StringBuilder sb = new StringBuilder();
460     sb.append('/');
461     sb.append(Bytes.toStringBinary(name));
462     sb.append("/$multiput"); // can be any nonexistent row
463     for (int i = 0; i < maxRetries; i++) {
464       Response response = client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF,
465         model.createProtobufOutput());
466       int code = response.getCode();
467       switch (code) {
468       case 200:
469         return;
470       case 509:
471         try {
472           Thread.sleep(sleepTime);
473         } catch (InterruptedException e) {
474           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
475         }
476         break;
477       default:
478         throw new IOException("multiput request failed with " + code);
479       }
480     }
481     throw new IOException("multiput request timed out");
482   }
483 
484   @Override
485   public void delete(Delete delete) throws IOException {
486     String spec = buildRowSpec(delete.getRow(), delete.getFamilyCellMap(),
487       delete.getTimeStamp(), delete.getTimeStamp(), 1);
488     for (int i = 0; i < maxRetries; i++) {
489       Response response = client.delete(spec);
490       int code = response.getCode();
491       switch (code) {
492       case 200:
493         return;
494       case 509:
495         try {
496           Thread.sleep(sleepTime);
497         } catch (InterruptedException e) {
498           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
499         }
500         break;
501       default:
502         throw new IOException("delete request failed with " + code);
503       }
504     }
505     throw new IOException("delete request timed out");
506   }
507 
508   @Override
509   public void delete(List<Delete> deletes) throws IOException {
510     for (Delete delete: deletes) {
511       delete(delete);
512     }
513   }
514 
515   public void flushCommits() throws IOException {
516     // no-op
517   }
518 
519   class Scanner implements ResultScanner {
520 
521     String uri;
522 
523     public Scanner(Scan scan) throws IOException {
524       ScannerModel model;
525       try {
526         model = ScannerModel.fromScan(scan);
527       } catch (Exception e) {
528         throw new IOException(e);
529       }
530       StringBuffer sb = new StringBuffer();
531       sb.append('/');
532       sb.append(Bytes.toStringBinary(name));
533       sb.append('/');
534       sb.append("scanner");
535       for (int i = 0; i < maxRetries; i++) {
536         Response response = client.post(sb.toString(),
537           Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
538         int code = response.getCode();
539         switch (code) {
540         case 201:
541           uri = response.getLocation();
542           return;
543         case 509:
544           try {
545             Thread.sleep(sleepTime);
546           } catch (InterruptedException e) {
547             throw (InterruptedIOException)new InterruptedIOException().initCause(e);
548           }
549           break;
550         default:
551           throw new IOException("scan request failed with " + code);
552         }
553       }
554       throw new IOException("scan request timed out");
555     }
556 
557     @Override
558     public Result[] next(int nbRows) throws IOException {
559       StringBuilder sb = new StringBuilder(uri);
560       sb.append("?n=");
561       sb.append(nbRows);
562       for (int i = 0; i < maxRetries; i++) {
563         Response response = client.get(sb.toString(),
564           Constants.MIMETYPE_PROTOBUF);
565         int code = response.getCode();
566         switch (code) {
567         case 200:
568           CellSetModel model = new CellSetModel();
569           model.getObjectFromMessage(response.getBody());
570           return buildResultFromModel(model);
571         case 204:
572         case 206:
573           return null;
574         case 509:
575           try {
576             Thread.sleep(sleepTime);
577           } catch (InterruptedException e) {
578             throw (InterruptedIOException)new InterruptedIOException().initCause(e);
579           }
580           break;
581         default:
582           throw new IOException("scanner.next request failed with " + code);
583         }
584       }
585       throw new IOException("scanner.next request timed out");
586     }
587 
588     @Override
589     public Result next() throws IOException {
590       Result[] results = next(1);
591       if (results == null || results.length < 1) {
592         return null;
593       }
594       return results[0];
595     }
596 
597     class Iter implements Iterator<Result> {
598 
599       Result cache;
600 
601       public Iter() {
602         try {
603           cache = Scanner.this.next();
604         } catch (IOException e) {
605           LOG.warn(StringUtils.stringifyException(e));
606         }
607       }
608 
609       @Override
610       public boolean hasNext() {
611         return cache != null;
612       }
613 
614       @Override
615       public Result next() {
616         Result result = cache;
617         try {
618           cache = Scanner.this.next();
619         } catch (IOException e) {
620           LOG.warn(StringUtils.stringifyException(e));
621           cache = null;
622         }
623         return result;
624       }
625 
626       @Override
627       public void remove() {
628         throw new RuntimeException("remove() not supported");
629       }
630 
631     }
632 
633     @Override
634     public Iterator<Result> iterator() {
635       return new Iter();
636     }
637 
638     @Override
639     public void close() {
640       try {
641         client.delete(uri);
642       } catch (IOException e) {
643         LOG.warn(StringUtils.stringifyException(e));
644       }
645     }
646   }
647 
648   @Override
649   public ResultScanner getScanner(Scan scan) throws IOException {
650     return new Scanner(scan);
651   }
652 
653   @Override
654   public ResultScanner getScanner(byte[] family) throws IOException {
655     Scan scan = new Scan();
656     scan.addFamily(family);
657     return new Scanner(scan);
658   }
659 
660   @Override
661   public ResultScanner getScanner(byte[] family, byte[] qualifier)
662       throws IOException {
663     Scan scan = new Scan();
664     scan.addColumn(family, qualifier);
665     return new Scanner(scan);
666   }
667 
668   public boolean isAutoFlush() {
669     return true;
670   }
671 
672   public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
673     throw new IOException("getRowOrBefore not supported");
674   }
675 
676   @Override
677   public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
678       byte[] value, Put put) throws IOException {
679     // column to check-the-value
680     put.add(new KeyValue(row, family, qualifier, value));
681 
682     CellSetModel model = buildModelFromPut(put);
683     StringBuilder sb = new StringBuilder();
684     sb.append('/');
685     sb.append(Bytes.toStringBinary(name));
686     sb.append('/');
687     sb.append(Bytes.toStringBinary(put.getRow()));
688     sb.append("?check=put");
689 
690     for (int i = 0; i < maxRetries; i++) {
691       Response response = client.put(sb.toString(),
692         Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
693       int code = response.getCode();
694       switch (code) {
695       case 200:
696         return true;
697       case 304: // NOT-MODIFIED
698         return false;
699       case 509:
700         try {
701           Thread.sleep(sleepTime);
702         } catch (final InterruptedException e) {
703           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
704         }
705         break;
706       default:
707         throw new IOException("checkAndPut request failed with " + code);
708       }
709     }
710     throw new IOException("checkAndPut request timed out");
711   }
712 
713   @Override
714   public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
715       CompareOp compareOp, byte[] value, Put put) throws IOException {
716     throw new IOException("checkAndPut for non-equal comparison not implemented");
717   }
718 
719   @Override
720   public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
721       byte[] value, Delete delete) throws IOException {
722     Put put = new Put(row);
723     put.setFamilyCellMap(delete.getFamilyCellMap());
724     // column to check-the-value
725     put.add(new KeyValue(row, family, qualifier, value));
726     CellSetModel model = buildModelFromPut(put);
727     StringBuilder sb = new StringBuilder();
728     sb.append('/');
729     sb.append(Bytes.toStringBinary(name));
730     sb.append('/');
731     sb.append(Bytes.toStringBinary(row));
732     sb.append("?check=delete");
733 
734     for (int i = 0; i < maxRetries; i++) {
735       Response response = client.put(sb.toString(),
736         Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
737       int code = response.getCode();
738       switch (code) {
739       case 200:
740         return true;
741       case 304: // NOT-MODIFIED
742         return false;
743       case 509:
744         try {
745           Thread.sleep(sleepTime);
746         } catch (final InterruptedException e) {
747           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
748         }
749         break;
750       default:
751         throw new IOException("checkAndDelete request failed with " + code);
752       }
753     }
754     throw new IOException("checkAndDelete request timed out");
755   }
756 
757   @Override
758   public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
759       CompareOp compareOp, byte[] value, Delete delete) throws IOException {
760     throw new IOException("checkAndDelete for non-equal comparison not implemented");
761   }
762 
763   @Override
764   public Result increment(Increment increment) throws IOException {
765     throw new IOException("Increment not supported");
766   }
767 
768   @Override
769   public Result append(Append append) throws IOException {
770     throw new IOException("Append not supported");
771   }
772 
773   @Override
774   public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
775       long amount) throws IOException {
776     throw new IOException("incrementColumnValue not supported");
777   }
778 
779   @Override
780   public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
781       long amount, Durability durability) throws IOException {
782     throw new IOException("incrementColumnValue not supported");
783   }
784 
785   @Override
786   public void batch(List<? extends Row> actions, Object[] results) throws IOException {
787     throw new IOException("batch not supported");
788   }
789 
790   @Override
791   public Object[] batch(List<? extends Row> actions) throws IOException {
792     throw new IOException("batch not supported");
793   }
794 
795   @Override
796   public <R> void batchCallback(List<? extends Row> actions, Object[] results,
797       Batch.Callback<R> callback) throws IOException, InterruptedException {
798     throw new IOException("batchCallback not supported");
799   }
800 
801   @Override
802   public <R> Object[] batchCallback(List<? extends Row> actions, Batch.Callback<R> callback)
803    throws IOException, InterruptedException {
804     throw new IOException("batchCallback not supported");
805   }
806 
807   @Override
808   public CoprocessorRpcChannel coprocessorService(byte[] row) {
809     throw new UnsupportedOperationException("coprocessorService not implemented");
810   }
811 
812   @Override
813   public <T extends Service, R> Map<byte[], R> coprocessorService(Class<T> service,
814       byte[] startKey, byte[] endKey, Batch.Call<T, R> callable)
815       throws ServiceException, Throwable {
816     throw new UnsupportedOperationException("coprocessorService not implemented");
817   }
818 
819   @Override
820   public <T extends Service, R> void coprocessorService(Class<T> service,
821       byte[] startKey, byte[] endKey, Batch.Call<T, R> callable, Batch.Callback<R> callback)
822       throws ServiceException, Throwable {
823     throw new UnsupportedOperationException("coprocessorService not implemented");
824   }
825 
826   @Override
827   public void mutateRow(RowMutations rm) throws IOException {
828     throw new IOException("atomicMutation not supported");
829   }
830 
831   @Override
832   public long getWriteBufferSize() {
833     throw new UnsupportedOperationException("getWriteBufferSize not implemented");
834   }
835 
836   @Override
837   public void setWriteBufferSize(long writeBufferSize) throws IOException {
838     throw new IOException("setWriteBufferSize not supported");
839   }
840 
841   @Override
842   public <R extends Message> Map<byte[], R> batchCoprocessorService(
843       Descriptors.MethodDescriptor method, Message request,
844       byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable {
845     throw new UnsupportedOperationException("batchCoprocessorService not implemented");
846   }
847 
848   @Override
849   public <R extends Message> void batchCoprocessorService(
850       Descriptors.MethodDescriptor method, Message request,
851       byte[] startKey, byte[] endKey, R responsePrototype, Callback<R> callback)
852       throws ServiceException, Throwable {
853     throw new UnsupportedOperationException("batchCoprocessorService not implemented");
854   }
855 
856   @Override public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
857       CompareOp compareOp, byte[] value, RowMutations rm) throws IOException {
858     throw new UnsupportedOperationException("checkAndMutate not implemented");
859   }
860 
861 }