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