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