001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.rest.client;
019
020import java.io.IOException;
021import java.io.InterruptedIOException;
022import java.io.UnsupportedEncodingException;
023import java.net.URLEncoder;
024import java.nio.charset.StandardCharsets;
025import java.util.ArrayList;
026import java.util.Collection;
027import java.util.Iterator;
028import java.util.List;
029import java.util.Map;
030import java.util.Set;
031import java.util.TreeMap;
032import java.util.concurrent.TimeUnit;
033import org.apache.commons.lang3.NotImplementedException;
034import org.apache.hadoop.conf.Configuration;
035import org.apache.hadoop.hbase.Cell;
036import org.apache.hadoop.hbase.CellUtil;
037import org.apache.hadoop.hbase.CompareOperator;
038import org.apache.hadoop.hbase.HBaseConfiguration;
039import org.apache.hadoop.hbase.HConstants;
040import org.apache.hadoop.hbase.KeyValue;
041import org.apache.hadoop.hbase.TableName;
042import org.apache.hadoop.hbase.client.Append;
043import org.apache.hadoop.hbase.client.CheckAndMutate;
044import org.apache.hadoop.hbase.client.CheckAndMutateResult;
045import org.apache.hadoop.hbase.client.Delete;
046import org.apache.hadoop.hbase.client.Durability;
047import org.apache.hadoop.hbase.client.Get;
048import org.apache.hadoop.hbase.client.Increment;
049import org.apache.hadoop.hbase.client.Put;
050import org.apache.hadoop.hbase.client.RegionLocator;
051import org.apache.hadoop.hbase.client.Result;
052import org.apache.hadoop.hbase.client.ResultScanner;
053import org.apache.hadoop.hbase.client.Row;
054import org.apache.hadoop.hbase.client.RowMutations;
055import org.apache.hadoop.hbase.client.Scan;
056import org.apache.hadoop.hbase.client.Table;
057import org.apache.hadoop.hbase.client.TableDescriptor;
058import org.apache.hadoop.hbase.client.coprocessor.Batch;
059import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
060import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
061import org.apache.hadoop.hbase.filter.Filter;
062import org.apache.hadoop.hbase.io.TimeRange;
063import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
064import org.apache.hadoop.hbase.rest.Constants;
065import org.apache.hadoop.hbase.rest.model.CellModel;
066import org.apache.hadoop.hbase.rest.model.CellSetModel;
067import org.apache.hadoop.hbase.rest.model.RowModel;
068import org.apache.hadoop.hbase.rest.model.ScannerModel;
069import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
070import org.apache.hadoop.hbase.util.Bytes;
071import org.apache.hadoop.util.StringUtils;
072import org.apache.yetus.audience.InterfaceAudience;
073import org.slf4j.Logger;
074import org.slf4j.LoggerFactory;
075
076import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
077import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
078import org.apache.hbase.thirdparty.com.google.protobuf.Message;
079import org.apache.hbase.thirdparty.com.google.protobuf.Service;
080import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
081
082/**
083 * HTable interface to remote tables accessed via REST gateway
084 */
085@InterfaceAudience.Private
086public class RemoteHTable implements Table {
087
088  private static final Logger LOG = LoggerFactory.getLogger(RemoteHTable.class);
089
090  final Client client;
091  final Configuration conf;
092  final byte[] name;
093  final int maxRetries;
094  final long sleepTime;
095
096  @SuppressWarnings("rawtypes")
097  protected String buildRowSpec(final byte[] row, final Map familyMap, final long startTime,
098    final long endTime, final int maxVersions) {
099    StringBuffer sb = new StringBuffer();
100    sb.append('/');
101    sb.append(Bytes.toString(name));
102    sb.append('/');
103    sb.append(toURLEncodedBytes(row));
104    Set families = familyMap.entrySet();
105    if (families != null) {
106      Iterator i = familyMap.entrySet().iterator();
107      sb.append('/');
108      while (i.hasNext()) {
109        Map.Entry e = (Map.Entry) i.next();
110        Collection quals = (Collection) e.getValue();
111        if (quals == null || quals.isEmpty()) {
112          // this is an unqualified family. append the family name and NO ':'
113          sb.append(toURLEncodedBytes((byte[]) e.getKey()));
114        } else {
115          Iterator ii = quals.iterator();
116          while (ii.hasNext()) {
117            sb.append(toURLEncodedBytes((byte[]) e.getKey()));
118            Object o = ii.next();
119            // Puts use byte[] but Deletes use KeyValue
120            if (o instanceof byte[]) {
121              sb.append(':');
122              sb.append(toURLEncodedBytes((byte[]) o));
123            } else if (o instanceof KeyValue) {
124              if (((KeyValue) o).getQualifierLength() != 0) {
125                sb.append(':');
126                sb.append(toURLEncodedBytes(CellUtil.cloneQualifier((KeyValue) o)));
127              }
128            } else {
129              throw new RuntimeException("object type not handled");
130            }
131            if (ii.hasNext()) {
132              sb.append(',');
133            }
134          }
135        }
136        if (i.hasNext()) {
137          sb.append(',');
138        }
139      }
140    }
141    if (startTime >= 0 && endTime != Long.MAX_VALUE) {
142      sb.append('/');
143      sb.append(startTime);
144      if (startTime != endTime) {
145        sb.append(',');
146        sb.append(endTime);
147      }
148    } else if (endTime != Long.MAX_VALUE) {
149      sb.append('/');
150      sb.append(endTime);
151    }
152    if (maxVersions > 1) {
153      sb.append("?v=");
154      sb.append(maxVersions);
155    }
156    return sb.toString();
157  }
158
159  protected String buildMultiRowSpec(final byte[][] rows, int maxVersions) {
160    StringBuilder sb = new StringBuilder();
161    sb.append('/');
162    sb.append(Bytes.toString(name));
163    sb.append("/multiget/");
164    if (rows == null || rows.length == 0) {
165      return sb.toString();
166    }
167    sb.append("?");
168    for (int i = 0; i < rows.length; i++) {
169      byte[] rk = rows[i];
170      if (i != 0) {
171        sb.append('&');
172      }
173      sb.append("row=");
174      sb.append(toURLEncodedBytes(rk));
175    }
176    sb.append("&v=");
177    sb.append(maxVersions);
178
179    return sb.toString();
180  }
181
182  protected Result[] buildResultFromModel(final CellSetModel model) {
183    List<Result> results = new ArrayList<>();
184    for (RowModel row : model.getRows()) {
185      List<Cell> kvs = new ArrayList<>(row.getCells().size());
186      for (CellModel cell : row.getCells()) {
187        byte[][] split = CellUtil.parseColumn(cell.getColumn());
188        byte[] column = split[0];
189        byte[] qualifier = null;
190        if (split.length == 1) {
191          qualifier = HConstants.EMPTY_BYTE_ARRAY;
192        } else if (split.length == 2) {
193          qualifier = split[1];
194        } else {
195          throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
196        }
197        kvs
198          .add(new KeyValue(row.getKey(), column, qualifier, cell.getTimestamp(), cell.getValue()));
199      }
200      results.add(Result.create(kvs));
201    }
202    return results.toArray(new Result[results.size()]);
203  }
204
205  protected CellSetModel buildModelFromPut(Put put) {
206    RowModel row = new RowModel(put.getRow());
207    long ts = put.getTimestamp();
208    for (List<Cell> cells : put.getFamilyCellMap().values()) {
209      for (Cell cell : cells) {
210        row.addCell(new CellModel(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell),
211          ts != HConstants.LATEST_TIMESTAMP ? ts : cell.getTimestamp(), CellUtil.cloneValue(cell)));
212      }
213    }
214    CellSetModel model = new CellSetModel();
215    model.addRow(row);
216    return model;
217  }
218
219  /**
220   * Constructor
221   */
222  public RemoteHTable(Client client, String name) {
223    this(client, HBaseConfiguration.create(), Bytes.toBytes(name));
224  }
225
226  /**
227   * Constructor
228   */
229  public RemoteHTable(Client client, Configuration conf, String name) {
230    this(client, conf, Bytes.toBytes(name));
231  }
232
233  /**
234   * Constructor
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 void close() throws IOException {
260    client.shutdown();
261  }
262
263  @Override
264  public Result get(Get get) throws IOException {
265    TimeRange range = get.getTimeRange();
266    String spec = buildRowSpec(get.getRow(), get.getFamilyMap(), range.getMin(), range.getMax(),
267      get.getMaxVersions());
268    if (get.getFilter() != null) {
269      LOG.warn("filters not supported on gets");
270    }
271    Result[] results = getResults(spec);
272    if (results.length > 0) {
273      if (results.length > 1) {
274        LOG.warn("too many results for get (" + results.length + ")");
275      }
276      return results[0];
277    } else {
278      return new Result();
279    }
280  }
281
282  @Override
283  public Result[] get(List<Get> gets) throws IOException {
284    byte[][] rows = new byte[gets.size()][];
285    int maxVersions = 1;
286    int count = 0;
287
288    for (Get g : gets) {
289
290      if (count == 0) {
291        maxVersions = g.getMaxVersions();
292      } else if (g.getMaxVersions() != maxVersions) {
293        LOG.warn(
294          "MaxVersions on Gets do not match, using the first in the list (" + maxVersions + ")");
295      }
296
297      if (g.getFilter() != null) {
298        LOG.warn("filters not supported on gets");
299      }
300
301      rows[count] = g.getRow();
302      count++;
303    }
304
305    String spec = buildMultiRowSpec(rows, maxVersions);
306
307    return getResults(spec);
308  }
309
310  private Result[] getResults(String spec) throws IOException {
311    for (int i = 0; i < maxRetries; i++) {
312      Response response = client.get(spec, Constants.MIMETYPE_PROTOBUF);
313      int code = response.getCode();
314      switch (code) {
315        case 200:
316          CellSetModel model = new CellSetModel();
317          model.getObjectFromMessage(response.getBody());
318          Result[] results = buildResultFromModel(model);
319          if (results.length > 0) {
320            return results;
321          }
322          // fall through
323        case 404:
324          return new Result[0];
325
326        case 509:
327          try {
328            Thread.sleep(sleepTime);
329          } catch (InterruptedException e) {
330            throw (InterruptedIOException) new InterruptedIOException().initCause(e);
331          }
332          break;
333        default:
334          throw new IOException("get request returned " + code);
335      }
336    }
337    throw new IOException("get request timed out");
338  }
339
340  @Override
341  public boolean exists(Get get) throws IOException {
342    LOG.warn("exists() is really get(), just use get()");
343    Result result = get(get);
344    return (result != null && !(result.isEmpty()));
345  }
346
347  @Override
348  public boolean[] exists(List<Get> gets) throws IOException {
349    LOG.warn("exists(List<Get>) is really list of get() calls, just use get()");
350    boolean[] results = new boolean[gets.size()];
351    for (int i = 0; i < results.length; i++) {
352      results[i] = exists(gets.get(i));
353    }
354    return results;
355  }
356
357  @Override
358  public void put(Put put) throws IOException {
359    CellSetModel model = buildModelFromPut(put);
360    StringBuilder sb = new StringBuilder();
361    sb.append('/');
362    sb.append(Bytes.toString(name));
363    sb.append('/');
364    sb.append(toURLEncodedBytes(put.getRow()));
365    for (int i = 0; i < maxRetries; i++) {
366      Response response =
367        client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
368      int code = response.getCode();
369      switch (code) {
370        case 200:
371          return;
372        case 509:
373          try {
374            Thread.sleep(sleepTime);
375          } catch (InterruptedException e) {
376            throw (InterruptedIOException) new InterruptedIOException().initCause(e);
377          }
378          break;
379        default:
380          throw new IOException("put request failed with " + code);
381      }
382    }
383    throw new IOException("put request timed out");
384  }
385
386  @Override
387  public void put(List<Put> puts) throws IOException {
388    // this is a trick: The gateway accepts multiple rows in a cell set and
389    // ignores the row specification in the URI
390
391    // separate puts by row
392    TreeMap<byte[], List<Cell>> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
393    for (Put put : puts) {
394      byte[] row = put.getRow();
395      List<Cell> cells = map.get(row);
396      if (cells == null) {
397        cells = new ArrayList<>();
398        map.put(row, cells);
399      }
400      for (List<Cell> l : put.getFamilyCellMap().values()) {
401        cells.addAll(l);
402      }
403    }
404
405    // build the cell set
406    CellSetModel model = new CellSetModel();
407    for (Map.Entry<byte[], List<Cell>> e : map.entrySet()) {
408      RowModel row = new RowModel(e.getKey());
409      for (Cell cell : e.getValue()) {
410        row.addCell(new CellModel(cell));
411      }
412      model.addRow(row);
413    }
414
415    // build path for multiput
416    StringBuilder sb = new StringBuilder();
417    sb.append('/');
418    sb.append(Bytes.toString(name));
419    sb.append("/$multiput"); // can be any nonexistent row
420    for (int i = 0; i < maxRetries; i++) {
421      Response response =
422        client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
423      int code = response.getCode();
424      switch (code) {
425        case 200:
426          return;
427        case 509:
428          try {
429            Thread.sleep(sleepTime);
430          } catch (InterruptedException e) {
431            throw (InterruptedIOException) new InterruptedIOException().initCause(e);
432          }
433          break;
434        default:
435          throw new IOException("multiput request failed with " + code);
436      }
437    }
438    throw new IOException("multiput request timed out");
439  }
440
441  @Override
442  public void delete(Delete delete) throws IOException {
443    String spec = buildRowSpec(delete.getRow(), delete.getFamilyCellMap(), delete.getTimestamp(),
444      delete.getTimestamp(), 1);
445    for (int i = 0; i < maxRetries; i++) {
446      Response response = client.delete(spec);
447      int code = response.getCode();
448      switch (code) {
449        case 200:
450          return;
451        case 509:
452          try {
453            Thread.sleep(sleepTime);
454          } catch (InterruptedException e) {
455            throw (InterruptedIOException) new InterruptedIOException().initCause(e);
456          }
457          break;
458        default:
459          throw new IOException("delete request failed with " + code);
460      }
461    }
462    throw new IOException("delete request timed out");
463  }
464
465  @Override
466  public void delete(List<Delete> deletes) throws IOException {
467    for (Delete delete : deletes) {
468      delete(delete);
469    }
470  }
471
472  public void flushCommits() throws IOException {
473    // no-op
474  }
475
476  @Override
477  public TableDescriptor getDescriptor() throws IOException {
478    StringBuilder sb = new StringBuilder();
479    sb.append('/');
480    sb.append(Bytes.toString(name));
481    sb.append('/');
482    sb.append("schema");
483    for (int i = 0; i < maxRetries; i++) {
484      Response response = client.get(sb.toString(), Constants.MIMETYPE_PROTOBUF);
485      int code = response.getCode();
486      switch (code) {
487        case 200:
488          TableSchemaModel schema = new TableSchemaModel();
489          schema.getObjectFromMessage(response.getBody());
490          return schema.getTableDescriptor();
491        case 509:
492          try {
493            Thread.sleep(sleepTime);
494          } catch (InterruptedException e) {
495            throw (InterruptedIOException) new InterruptedIOException().initCause(e);
496          }
497          break;
498        default:
499          throw new IOException("schema request returned " + code);
500      }
501    }
502    throw new IOException("schema request timed out");
503  }
504
505  class Scanner implements ResultScanner {
506
507    String uri;
508
509    public Scanner(Scan scan) throws IOException {
510      ScannerModel model;
511      try {
512        model = ScannerModel.fromScan(scan);
513      } catch (Exception e) {
514        throw new IOException(e);
515      }
516      StringBuffer sb = new StringBuffer();
517      sb.append('/');
518      sb.append(Bytes.toString(name));
519      sb.append('/');
520      sb.append("scanner");
521      for (int i = 0; i < maxRetries; i++) {
522        Response response =
523          client.post(sb.toString(), Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
524        int code = response.getCode();
525        switch (code) {
526          case 201:
527            uri = response.getLocation();
528            return;
529          case 509:
530            try {
531              Thread.sleep(sleepTime);
532            } catch (InterruptedException e) {
533              throw (InterruptedIOException) new InterruptedIOException().initCause(e);
534            }
535            break;
536          default:
537            throw new IOException("scan request failed with " + code);
538        }
539      }
540      throw new IOException("scan request timed out");
541    }
542
543    @Override
544    public Result[] next(int nbRows) throws IOException {
545      StringBuilder sb = new StringBuilder(uri);
546      sb.append("?n=");
547      sb.append(nbRows);
548      for (int i = 0; i < maxRetries; i++) {
549        Response response = client.get(sb.toString(), Constants.MIMETYPE_PROTOBUF);
550        int code = response.getCode();
551        switch (code) {
552          case 200:
553            CellSetModel model = new CellSetModel();
554            model.getObjectFromMessage(response.getBody());
555            return buildResultFromModel(model);
556          case 204:
557          case 206:
558            return null;
559          case 509:
560            try {
561              Thread.sleep(sleepTime);
562            } catch (InterruptedException e) {
563              throw (InterruptedIOException) new InterruptedIOException().initCause(e);
564            }
565            break;
566          default:
567            throw new IOException("scanner.next request failed with " + code);
568        }
569      }
570      throw new IOException("scanner.next request timed out");
571    }
572
573    @Override
574    public Result next() throws IOException {
575      Result[] results = next(1);
576      if (results == null || results.length < 1) {
577        return null;
578      }
579      return results[0];
580    }
581
582    class Iter implements Iterator<Result> {
583
584      Result cache;
585
586      public Iter() {
587        try {
588          cache = Scanner.this.next();
589        } catch (IOException e) {
590          LOG.warn(StringUtils.stringifyException(e));
591        }
592      }
593
594      @Override
595      public boolean hasNext() {
596        return cache != null;
597      }
598
599      @Override
600      public Result next() {
601        Result result = cache;
602        try {
603          cache = Scanner.this.next();
604        } catch (IOException e) {
605          LOG.warn(StringUtils.stringifyException(e));
606          cache = null;
607        }
608        return result;
609      }
610
611      @Override
612      public void remove() {
613        throw new RuntimeException("remove() not supported");
614      }
615
616    }
617
618    @Override
619    public Iterator<Result> iterator() {
620      return new Iter();
621    }
622
623    @Override
624    public void close() {
625      try {
626        client.delete(uri);
627      } catch (IOException e) {
628        LOG.warn(StringUtils.stringifyException(e));
629      }
630    }
631
632    @Override
633    public boolean renewLease() {
634      throw new RuntimeException("renewLease() not supported");
635    }
636
637    @Override
638    public ScanMetrics getScanMetrics() {
639      throw new RuntimeException("getScanMetrics() not supported");
640    }
641  }
642
643  @Override
644  public ResultScanner getScanner(Scan scan) throws IOException {
645    return new Scanner(scan);
646  }
647
648  @Override
649  public ResultScanner getScanner(byte[] family) throws IOException {
650    Scan scan = new Scan();
651    scan.addFamily(family);
652    return new Scanner(scan);
653  }
654
655  @Override
656  public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException {
657    Scan scan = new Scan();
658    scan.addColumn(family, qualifier);
659    return new Scanner(scan);
660  }
661
662  public boolean isAutoFlush() {
663    return true;
664  }
665
666  private boolean doCheckAndPut(byte[] row, byte[] family, byte[] qualifier, byte[] value, Put put)
667    throws IOException {
668    // column to check-the-value
669    put.add(new KeyValue(row, family, qualifier, value));
670
671    CellSetModel model = buildModelFromPut(put);
672    StringBuilder sb = new StringBuilder();
673    sb.append('/');
674    sb.append(Bytes.toString(name));
675    sb.append('/');
676    sb.append(toURLEncodedBytes(put.getRow()));
677    sb.append("?check=put");
678
679    for (int i = 0; i < maxRetries; i++) {
680      Response response =
681        client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
682      int code = response.getCode();
683      switch (code) {
684        case 200:
685          return true;
686        case 304: // NOT-MODIFIED
687          return false;
688        case 509:
689          try {
690            Thread.sleep(sleepTime);
691          } catch (final InterruptedException e) {
692            throw (InterruptedIOException) new InterruptedIOException().initCause(e);
693          }
694          break;
695        default:
696          throw new IOException("checkAndPut request failed with " + code);
697      }
698    }
699    throw new IOException("checkAndPut request timed out");
700  }
701
702  private boolean doCheckAndDelete(byte[] row, byte[] family, byte[] qualifier, byte[] value,
703    Delete delete) throws IOException {
704    Put put = new Put(row, HConstants.LATEST_TIMESTAMP, delete.getFamilyCellMap());
705    // column to check-the-value
706    put.add(new KeyValue(row, family, qualifier, value));
707    CellSetModel model = buildModelFromPut(put);
708    StringBuilder sb = new StringBuilder();
709    sb.append('/');
710    sb.append(Bytes.toString(name));
711    sb.append('/');
712    sb.append(toURLEncodedBytes(row));
713    sb.append("?check=delete");
714
715    for (int i = 0; i < maxRetries; i++) {
716      Response response =
717        client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
718      int code = response.getCode();
719      switch (code) {
720        case 200:
721          return true;
722        case 304: // NOT-MODIFIED
723          return false;
724        case 509:
725          try {
726            Thread.sleep(sleepTime);
727          } catch (final InterruptedException e) {
728            throw (InterruptedIOException) new InterruptedIOException().initCause(e);
729          }
730          break;
731        default:
732          throw new IOException("checkAndDelete request failed with " + code);
733      }
734    }
735    throw new IOException("checkAndDelete request timed out");
736  }
737
738  @Override
739  public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
740    return new CheckAndMutateBuilderImpl(row, family);
741  }
742
743  @Override
744  public CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter) {
745    throw new NotImplementedException("Implement later");
746  }
747
748  @Override
749  public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) {
750    throw new NotImplementedException("Implement later");
751  }
752
753  @Override
754  public List<CheckAndMutateResult> checkAndMutate(List<CheckAndMutate> checkAndMutates) {
755    throw new NotImplementedException("Implement later");
756  }
757
758  @Override
759  public Result increment(Increment increment) throws IOException {
760    throw new IOException("Increment not supported");
761  }
762
763  @Override
764  public Result append(Append append) throws IOException {
765    throw new IOException("Append not supported");
766  }
767
768  @Override
769  public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount)
770    throws IOException {
771    throw new IOException("incrementColumnValue not supported");
772  }
773
774  @Override
775  public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount,
776    Durability durability) throws IOException {
777    throw new IOException("incrementColumnValue not supported");
778  }
779
780  @Override
781  public void batch(List<? extends Row> actions, Object[] results) throws IOException {
782    throw new IOException("batch not supported");
783  }
784
785  @Override
786  public <R> void batchCallback(List<? extends Row> actions, Object[] results,
787    Batch.Callback<R> callback) throws IOException, InterruptedException {
788    throw new IOException("batchCallback not supported");
789  }
790
791  @Override
792  public CoprocessorRpcChannel coprocessorService(byte[] row) {
793    throw new UnsupportedOperationException("coprocessorService not implemented");
794  }
795
796  @Override
797  public <T extends Service, R> Map<byte[], R> coprocessorService(Class<T> service, byte[] startKey,
798    byte[] endKey, Batch.Call<T, R> callable) throws ServiceException, Throwable {
799    throw new UnsupportedOperationException("coprocessorService not implemented");
800  }
801
802  @Override
803  public <T extends Service, R> void coprocessorService(Class<T> service, byte[] startKey,
804    byte[] endKey, Batch.Call<T, R> callable, Batch.Callback<R> callback)
805    throws ServiceException, Throwable {
806    throw new UnsupportedOperationException("coprocessorService not implemented");
807  }
808
809  @Override
810  public Result mutateRow(RowMutations rm) throws IOException {
811    throw new IOException("atomicMutation not supported");
812  }
813
814  @Override
815  public <R extends Message> Map<byte[], R> batchCoprocessorService(
816    Descriptors.MethodDescriptor method, Message request, byte[] startKey, byte[] endKey,
817    R responsePrototype) throws ServiceException, Throwable {
818    throw new UnsupportedOperationException("batchCoprocessorService not implemented");
819  }
820
821  @Override
822  public <R extends Message> void batchCoprocessorService(Descriptors.MethodDescriptor method,
823    Message request, byte[] startKey, byte[] endKey, R responsePrototype, Callback<R> callback)
824    throws ServiceException, Throwable {
825    throw new UnsupportedOperationException("batchCoprocessorService not implemented");
826  }
827
828  @Override
829  public long getReadRpcTimeout(TimeUnit unit) {
830    throw new UnsupportedOperationException();
831  }
832
833  @Override
834  public long getRpcTimeout(TimeUnit unit) {
835    throw new UnsupportedOperationException();
836  }
837
838  @Override
839  public long getWriteRpcTimeout(TimeUnit unit) {
840    throw new UnsupportedOperationException();
841  }
842
843  @Override
844  public long getOperationTimeout(TimeUnit unit) {
845    throw new UnsupportedOperationException();
846  }
847
848  /*
849   * Only a small subset of characters are valid in URLs. Row keys, column families, and qualifiers
850   * cannot be appended to URLs without first URL escaping. Table names are ok because they can only
851   * contain alphanumeric, ".","_", and "-" which are valid characters in URLs.
852   */
853  private static String toURLEncodedBytes(byte[] row) {
854    try {
855      return URLEncoder.encode(new String(row, StandardCharsets.UTF_8), "UTF-8");
856    } catch (UnsupportedEncodingException e) {
857      throw new IllegalStateException("URLEncoder doesn't support UTF-8", e);
858    }
859  }
860
861  private class CheckAndMutateBuilderImpl implements CheckAndMutateBuilder {
862
863    private final byte[] row;
864    private final byte[] family;
865    private byte[] qualifier;
866    private byte[] value;
867
868    CheckAndMutateBuilderImpl(byte[] row, byte[] family) {
869      this.row = Preconditions.checkNotNull(row, "row is null");
870      this.family = Preconditions.checkNotNull(family, "family is null");
871    }
872
873    @Override
874    public CheckAndMutateBuilder qualifier(byte[] qualifier) {
875      this.qualifier = Preconditions.checkNotNull(qualifier, "qualifier is null. Consider using"
876        + " an empty byte array, or just do not call this method if you want a null qualifier");
877      return this;
878    }
879
880    @Override
881    public CheckAndMutateBuilder timeRange(TimeRange timeRange) {
882      throw new UnsupportedOperationException("timeRange not implemented");
883    }
884
885    @Override
886    public CheckAndMutateBuilder ifNotExists() {
887      throw new UnsupportedOperationException(
888        "CheckAndMutate for non-equal comparison " + "not implemented");
889    }
890
891    @Override
892    public CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value) {
893      if (compareOp == CompareOperator.EQUAL) {
894        this.value = Preconditions.checkNotNull(value, "value is null");
895        return this;
896      } else {
897        throw new UnsupportedOperationException(
898          "CheckAndMutate for non-equal comparison " + "not implemented");
899      }
900    }
901
902    @Override
903    public CheckAndMutateBuilder ifEquals(byte[] value) {
904      this.value = Preconditions.checkNotNull(value, "value is null");
905      return this;
906    }
907
908    @Override
909    public boolean thenPut(Put put) throws IOException {
910      return doCheckAndPut(row, family, qualifier, value, put);
911    }
912
913    @Override
914    public boolean thenDelete(Delete delete) throws IOException {
915      return doCheckAndDelete(row, family, qualifier, value, delete);
916    }
917
918    @Override
919    public boolean thenMutate(RowMutations mutation) throws IOException {
920      throw new UnsupportedOperationException("thenMutate not implemented");
921    }
922  }
923
924  @Override
925  public RegionLocator getRegionLocator() throws IOException {
926    throw new UnsupportedOperationException();
927  }
928}