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.thrift2;
019
020import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD;
021import static org.apache.hadoop.hbase.HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD;
022import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED;
023import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED_DEFAULT;
024import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.appendFromThrift;
025import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.columnFamilyDescriptorFromThrift;
026import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.compareOpFromThrift;
027import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deleteFromThrift;
028import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deletesFromThrift;
029import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getFromThrift;
030import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getsFromThrift;
031import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.incrementFromThrift;
032import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.namespaceDescriptorFromHBase;
033import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.namespaceDescriptorFromThrift;
034import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.namespaceDescriptorsFromHBase;
035import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.putFromThrift;
036import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.putsFromThrift;
037import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.resultFromHBase;
038import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.resultsFromHBase;
039import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.rowMutationsFromThrift;
040import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.scanFromThrift;
041import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.splitKeyFromThrift;
042import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableDescriptorFromHBase;
043import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableDescriptorFromThrift;
044import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableDescriptorsFromHBase;
045import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableNameFromThrift;
046import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableNamesFromHBase;
047import static org.apache.thrift.TBaseHelper.byteBufferToByteArray;
048
049import java.io.IOException;
050import java.nio.ByteBuffer;
051import java.util.ArrayList;
052import java.util.Collections;
053import java.util.List;
054import java.util.Set;
055import java.util.concurrent.TimeUnit;
056import java.util.concurrent.atomic.AtomicInteger;
057import java.util.regex.Pattern;
058import org.apache.commons.lang3.NotImplementedException;
059import org.apache.hadoop.conf.Configuration;
060import org.apache.hadoop.hbase.DoNotRetryIOException;
061import org.apache.hadoop.hbase.HRegionLocation;
062import org.apache.hadoop.hbase.NamespaceDescriptor;
063import org.apache.hadoop.hbase.ServerName;
064import org.apache.hadoop.hbase.TableName;
065import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
066import org.apache.hadoop.hbase.client.LogQueryFilter;
067import org.apache.hadoop.hbase.client.OnlineLogRecord;
068import org.apache.hadoop.hbase.client.RegionLocator;
069import org.apache.hadoop.hbase.client.ResultScanner;
070import org.apache.hadoop.hbase.client.Table;
071import org.apache.hadoop.hbase.client.TableDescriptor;
072import org.apache.hadoop.hbase.security.UserProvider;
073import org.apache.hadoop.hbase.security.access.AccessControlClient;
074import org.apache.hadoop.hbase.security.access.Permission;
075import org.apache.hadoop.hbase.thrift.HBaseServiceHandler;
076import org.apache.hadoop.hbase.thrift2.generated.TAccessControlEntity;
077import org.apache.hadoop.hbase.thrift2.generated.TAppend;
078import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
079import org.apache.hadoop.hbase.thrift2.generated.TCompareOperator;
080import org.apache.hadoop.hbase.thrift2.generated.TDelete;
081import org.apache.hadoop.hbase.thrift2.generated.TGet;
082import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
083import org.apache.hadoop.hbase.thrift2.generated.THRegionLocation;
084import org.apache.hadoop.hbase.thrift2.generated.TIOError;
085import org.apache.hadoop.hbase.thrift2.generated.TIllegalArgument;
086import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
087import org.apache.hadoop.hbase.thrift2.generated.TLogQueryFilter;
088import org.apache.hadoop.hbase.thrift2.generated.TNamespaceDescriptor;
089import org.apache.hadoop.hbase.thrift2.generated.TOnlineLogRecord;
090import org.apache.hadoop.hbase.thrift2.generated.TPermissionScope;
091import org.apache.hadoop.hbase.thrift2.generated.TPut;
092import org.apache.hadoop.hbase.thrift2.generated.TResult;
093import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
094import org.apache.hadoop.hbase.thrift2.generated.TScan;
095import org.apache.hadoop.hbase.thrift2.generated.TServerName;
096import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
097import org.apache.hadoop.hbase.thrift2.generated.TTableName;
098import org.apache.hadoop.hbase.thrift2.generated.TThriftServerType;
099import org.apache.hadoop.hbase.util.Bytes;
100import org.apache.thrift.TException;
101import org.apache.yetus.audience.InterfaceAudience;
102import org.slf4j.Logger;
103import org.slf4j.LoggerFactory;
104
105import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
106import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
107import org.apache.hbase.thirdparty.com.google.common.cache.RemovalListener;
108
109/**
110 * This class is a glue object that connects Thrift RPC calls to the HBase client API primarily
111 * defined in the Table interface.
112 */
113@InterfaceAudience.Private
114@SuppressWarnings("deprecation")
115public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements THBaseService.Iface {
116
117  // TODO: Size of pool configuraple
118  private static final Logger LOG = LoggerFactory.getLogger(ThriftHBaseServiceHandler.class);
119
120  // nextScannerId and scannerMap are used to manage scanner state
121  private final AtomicInteger nextScannerId = new AtomicInteger(0);
122  private final Cache<Integer, ResultScanner> scannerMap;
123
124  private static final IOException ioe =
125    new DoNotRetryIOException("Thrift Server is in Read-only mode.");
126  private boolean isReadOnly;
127
128  private static class TIOErrorWithCause extends TIOError {
129    private Throwable cause;
130
131    public TIOErrorWithCause(Throwable cause) {
132      super();
133      this.cause = cause;
134    }
135
136    @Override
137    public synchronized Throwable getCause() {
138      return cause;
139    }
140
141    @Override
142    public boolean equals(Object other) {
143      if (super.equals(other) && other instanceof TIOErrorWithCause) {
144        Throwable otherCause = ((TIOErrorWithCause) other).getCause();
145        if (this.getCause() != null) {
146          return otherCause != null && this.getCause().equals(otherCause);
147        } else {
148          return otherCause == null;
149        }
150      }
151      return false;
152    }
153
154    @Override
155    public int hashCode() {
156      int result = super.hashCode();
157      result = 31 * result + (cause != null ? cause.hashCode() : 0);
158      return result;
159    }
160  }
161
162  public ThriftHBaseServiceHandler(final Configuration conf, final UserProvider userProvider)
163    throws IOException {
164    super(conf, userProvider);
165    long cacheTimeout = conf.getLong(HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
166      DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
167    isReadOnly = conf.getBoolean(THRIFT_READONLY_ENABLED, THRIFT_READONLY_ENABLED_DEFAULT);
168    scannerMap = CacheBuilder.newBuilder().expireAfterAccess(cacheTimeout, TimeUnit.MILLISECONDS)
169      .removalListener((RemovalListener<Integer,
170        ResultScanner>) removalNotification -> removalNotification.getValue().close())
171      .build();
172  }
173
174  @Override
175  protected Table getTable(ByteBuffer tableName) {
176    try {
177      return connectionCache.getTable(Bytes.toString(byteBufferToByteArray(tableName)));
178    } catch (IOException ie) {
179      throw new RuntimeException(ie);
180    }
181  }
182
183  private RegionLocator getLocator(ByteBuffer tableName) {
184    try {
185      return connectionCache.getRegionLocator(byteBufferToByteArray(tableName));
186    } catch (IOException ie) {
187      throw new RuntimeException(ie);
188    }
189  }
190
191  private void closeTable(Table table) throws TIOError {
192    try {
193      table.close();
194    } catch (IOException e) {
195      throw getTIOError(e);
196    }
197  }
198
199  private TIOError getTIOError(IOException e) {
200    TIOError err = new TIOErrorWithCause(e);
201    err.setCanRetry(!(e instanceof DoNotRetryIOException));
202    err.setMessage(e.getMessage());
203    return err;
204  }
205
206  /**
207   * Assigns a unique ID to the scanner and adds the mapping to an internal HashMap.
208   * @param scanner to add
209   * @return Id for this Scanner
210   */
211  private int addScanner(ResultScanner scanner) {
212    int id = nextScannerId.getAndIncrement();
213    scannerMap.put(id, scanner);
214    return id;
215  }
216
217  /**
218   * Returns the Scanner associated with the specified Id.
219   * @param id of the Scanner to get
220   * @return a Scanner, or null if the Id is invalid
221   */
222  private ResultScanner getScanner(int id) {
223    return scannerMap.getIfPresent(id);
224  }
225
226  /**
227   * Removes the scanner associated with the specified ID from the internal HashMap.
228   * @param id of the Scanner to remove
229   */
230  protected void removeScanner(int id) {
231    scannerMap.invalidate(id);
232  }
233
234  @Override
235  public boolean exists(ByteBuffer table, TGet get) throws TIOError, TException {
236    Table htable = getTable(table);
237    try {
238      return htable.exists(getFromThrift(get));
239    } catch (IOException e) {
240      throw getTIOError(e);
241    } finally {
242      closeTable(htable);
243    }
244  }
245
246  @Override
247  public List<Boolean> existsAll(ByteBuffer table, List<TGet> gets) throws TIOError, TException {
248    Table htable = getTable(table);
249    try {
250      boolean[] exists = htable.exists(getsFromThrift(gets));
251      List<Boolean> result = new ArrayList<>(exists.length);
252      for (boolean exist : exists) {
253        result.add(exist);
254      }
255      return result;
256    } catch (IOException e) {
257      throw getTIOError(e);
258    } finally {
259      closeTable(htable);
260    }
261  }
262
263  @Override
264  public TResult get(ByteBuffer table, TGet get) throws TIOError, TException {
265    Table htable = getTable(table);
266    try {
267      return resultFromHBase(htable.get(getFromThrift(get)));
268    } catch (IOException e) {
269      throw getTIOError(e);
270    } finally {
271      closeTable(htable);
272    }
273  }
274
275  @Override
276  public List<TResult> getMultiple(ByteBuffer table, List<TGet> gets) throws TIOError, TException {
277    Table htable = getTable(table);
278    try {
279      return resultsFromHBase(htable.get(getsFromThrift(gets)));
280    } catch (IOException e) {
281      throw getTIOError(e);
282    } finally {
283      closeTable(htable);
284    }
285  }
286
287  @Override
288  public void put(ByteBuffer table, TPut put) throws TIOError, TException {
289    checkReadOnlyMode();
290    Table htable = getTable(table);
291    try {
292      htable.put(putFromThrift(put));
293    } catch (IOException e) {
294      throw getTIOError(e);
295    } finally {
296      closeTable(htable);
297    }
298  }
299
300  @Override
301  public boolean checkAndPut(ByteBuffer table, ByteBuffer row, ByteBuffer family,
302    ByteBuffer qualifier, ByteBuffer value, TPut put) throws TIOError, TException {
303    checkReadOnlyMode();
304    Table htable = getTable(table);
305    try {
306      Table.CheckAndMutateBuilder builder =
307        htable.checkAndMutate(byteBufferToByteArray(row), byteBufferToByteArray(family))
308          .qualifier(byteBufferToByteArray(qualifier));
309      if (value == null) {
310        return builder.ifNotExists().thenPut(putFromThrift(put));
311      } else {
312        return builder.ifEquals(byteBufferToByteArray(value)).thenPut(putFromThrift(put));
313      }
314    } catch (IOException e) {
315      throw getTIOError(e);
316    } finally {
317      closeTable(htable);
318    }
319  }
320
321  @Override
322  public void putMultiple(ByteBuffer table, List<TPut> puts) throws TIOError, TException {
323    checkReadOnlyMode();
324    Table htable = getTable(table);
325    try {
326      htable.put(putsFromThrift(puts));
327    } catch (IOException e) {
328      throw getTIOError(e);
329    } finally {
330      closeTable(htable);
331    }
332  }
333
334  @Override
335  public void deleteSingle(ByteBuffer table, TDelete deleteSingle) throws TIOError, TException {
336    checkReadOnlyMode();
337    Table htable = getTable(table);
338    try {
339      htable.delete(deleteFromThrift(deleteSingle));
340    } catch (IOException e) {
341      throw getTIOError(e);
342    } finally {
343      closeTable(htable);
344    }
345  }
346
347  @Override
348  public List<TDelete> deleteMultiple(ByteBuffer table, List<TDelete> deletes)
349    throws TIOError, TException {
350    checkReadOnlyMode();
351    Table htable = getTable(table);
352    try {
353      htable.delete(deletesFromThrift(deletes));
354    } catch (IOException e) {
355      throw getTIOError(e);
356    } finally {
357      closeTable(htable);
358    }
359    return Collections.emptyList();
360  }
361
362  @Override
363  public boolean checkAndMutate(ByteBuffer table, ByteBuffer row, ByteBuffer family,
364    ByteBuffer qualifier, TCompareOperator compareOp, ByteBuffer value, TRowMutations rowMutations)
365    throws TIOError, TException {
366    checkReadOnlyMode();
367    try (final Table htable = getTable(table)) {
368      return htable.checkAndMutate(byteBufferToByteArray(row), byteBufferToByteArray(family))
369        .qualifier(byteBufferToByteArray(qualifier))
370        .ifMatches(compareOpFromThrift(compareOp), byteBufferToByteArray(value))
371        .thenMutate(rowMutationsFromThrift(rowMutations));
372    } catch (IOException e) {
373      throw getTIOError(e);
374    }
375  }
376
377  @Override
378  public boolean checkAndDelete(ByteBuffer table, ByteBuffer row, ByteBuffer family,
379    ByteBuffer qualifier, ByteBuffer value, TDelete deleteSingle) throws TIOError, TException {
380    checkReadOnlyMode();
381    Table htable = getTable(table);
382    try {
383      Table.CheckAndMutateBuilder mutateBuilder =
384        htable.checkAndMutate(byteBufferToByteArray(row), byteBufferToByteArray(family))
385          .qualifier(byteBufferToByteArray(qualifier));
386      if (value == null) {
387        return mutateBuilder.ifNotExists().thenDelete(deleteFromThrift(deleteSingle));
388      } else {
389        return mutateBuilder.ifEquals(byteBufferToByteArray(value))
390          .thenDelete(deleteFromThrift(deleteSingle));
391      }
392    } catch (IOException e) {
393      throw getTIOError(e);
394    } finally {
395      closeTable(htable);
396    }
397  }
398
399  @Override
400  public TResult increment(ByteBuffer table, TIncrement increment) throws TIOError, TException {
401    checkReadOnlyMode();
402    Table htable = getTable(table);
403    try {
404      return resultFromHBase(htable.increment(incrementFromThrift(increment)));
405    } catch (IOException e) {
406      throw getTIOError(e);
407    } finally {
408      closeTable(htable);
409    }
410  }
411
412  @Override
413  public TResult append(ByteBuffer table, TAppend append) throws TIOError, TException {
414    checkReadOnlyMode();
415    Table htable = getTable(table);
416    try {
417      return resultFromHBase(htable.append(appendFromThrift(append)));
418    } catch (IOException e) {
419      throw getTIOError(e);
420    } finally {
421      closeTable(htable);
422    }
423  }
424
425  @Override
426  public int openScanner(ByteBuffer table, TScan scan) throws TIOError, TException {
427    Table htable = getTable(table);
428    ResultScanner resultScanner = null;
429    try {
430      resultScanner = htable.getScanner(scanFromThrift(scan));
431    } catch (IOException e) {
432      throw getTIOError(e);
433    } finally {
434      closeTable(htable);
435    }
436    return addScanner(resultScanner);
437  }
438
439  @Override
440  public List<TResult> getScannerRows(int scannerId, int numRows)
441    throws TIOError, TIllegalArgument, TException {
442    ResultScanner scanner = getScanner(scannerId);
443    if (scanner == null) {
444      TIllegalArgument ex = new TIllegalArgument();
445      ex.setMessage("Invalid scanner Id");
446      throw ex;
447    }
448    try {
449      connectionCache.updateConnectionAccessTime();
450      return resultsFromHBase(scanner.next(numRows));
451    } catch (IOException e) {
452      throw getTIOError(e);
453    }
454  }
455
456  @Override
457  public List<TResult> getScannerResults(ByteBuffer table, TScan scan, int numRows)
458    throws TIOError, TException {
459    Table htable = getTable(table);
460    List<TResult> results = null;
461    ResultScanner scanner = null;
462    try {
463      scanner = htable.getScanner(scanFromThrift(scan));
464      results = resultsFromHBase(scanner.next(numRows));
465    } catch (IOException e) {
466      throw getTIOError(e);
467    } finally {
468      if (scanner != null) {
469        scanner.close();
470      }
471      closeTable(htable);
472    }
473    return results;
474  }
475
476  @Override
477  public void closeScanner(int scannerId) throws TIOError, TIllegalArgument, TException {
478    LOG.debug("scannerClose: id=" + scannerId);
479    ResultScanner scanner = getScanner(scannerId);
480    if (scanner == null) {
481      LOG.warn("scanner ID: " + scannerId + "is invalid");
482      // While the scanner could be already expired,
483      // we should not throw exception here. Just log and return.
484      return;
485    }
486    scanner.close();
487    removeScanner(scannerId);
488  }
489
490  @Override
491  public void mutateRow(ByteBuffer table, TRowMutations rowMutations) throws TIOError, TException {
492    checkReadOnlyMode();
493    Table htable = getTable(table);
494    try {
495      htable.mutateRow(rowMutationsFromThrift(rowMutations));
496    } catch (IOException e) {
497      throw getTIOError(e);
498    } finally {
499      closeTable(htable);
500    }
501  }
502
503  @Override
504  public List<THRegionLocation> getAllRegionLocations(ByteBuffer table)
505    throws TIOError, TException {
506    RegionLocator locator = null;
507    try {
508      locator = getLocator(table);
509      return ThriftUtilities.regionLocationsFromHBase(locator.getAllRegionLocations());
510
511    } catch (IOException e) {
512      throw getTIOError(e);
513    } finally {
514      if (locator != null) {
515        try {
516          locator.close();
517        } catch (IOException e) {
518          LOG.warn("Couldn't close the locator.", e);
519        }
520      }
521    }
522  }
523
524  @Override
525  public THRegionLocation getRegionLocation(ByteBuffer table, ByteBuffer row, boolean reload)
526    throws TIOError, TException {
527
528    RegionLocator locator = null;
529    try {
530      locator = getLocator(table);
531      byte[] rowBytes = byteBufferToByteArray(row);
532      HRegionLocation hrl = locator.getRegionLocation(rowBytes, reload);
533      return ThriftUtilities.regionLocationFromHBase(hrl);
534
535    } catch (IOException e) {
536      throw getTIOError(e);
537    } finally {
538      if (locator != null) {
539        try {
540          locator.close();
541        } catch (IOException e) {
542          LOG.warn("Couldn't close the locator.", e);
543        }
544      }
545    }
546  }
547
548  private void checkReadOnlyMode() throws TIOError {
549    if (isReadOnly()) {
550      throw getTIOError(ioe);
551    }
552  }
553
554  private boolean isReadOnly() {
555    return isReadOnly;
556  }
557
558  @Override
559  public TTableDescriptor getTableDescriptor(TTableName table) throws TIOError, TException {
560    try {
561      TableName tableName = ThriftUtilities.tableNameFromThrift(table);
562      TableDescriptor tableDescriptor = connectionCache.getAdmin().getDescriptor(tableName);
563      return tableDescriptorFromHBase(tableDescriptor);
564    } catch (IOException e) {
565      throw getTIOError(e);
566    }
567  }
568
569  @Override
570  public List<TTableDescriptor> getTableDescriptors(List<TTableName> tables)
571    throws TIOError, TException {
572    try {
573      List<TableName> tableNames = ThriftUtilities.tableNamesFromThrift(tables);
574      List<TableDescriptor> tableDescriptors =
575        connectionCache.getAdmin().listTableDescriptors(tableNames);
576      return tableDescriptorsFromHBase(tableDescriptors);
577    } catch (IOException e) {
578      throw getTIOError(e);
579    }
580  }
581
582  @Override
583  public boolean tableExists(TTableName tTableName) throws TIOError, TException {
584    try {
585      TableName tableName = tableNameFromThrift(tTableName);
586      return connectionCache.getAdmin().tableExists(tableName);
587    } catch (IOException e) {
588      throw getTIOError(e);
589    }
590  }
591
592  @Override
593  public List<TTableDescriptor> getTableDescriptorsByPattern(String regex, boolean includeSysTables)
594    throws TIOError, TException {
595    try {
596      Pattern pattern = (regex == null ? null : Pattern.compile(regex));
597      List<TableDescriptor> tableDescriptors =
598        connectionCache.getAdmin().listTableDescriptors(pattern, includeSysTables);
599      return tableDescriptorsFromHBase(tableDescriptors);
600    } catch (IOException e) {
601      throw getTIOError(e);
602    }
603  }
604
605  @Override
606  public List<TTableDescriptor> getTableDescriptorsByNamespace(String name)
607    throws TIOError, TException {
608    try {
609      List<TableDescriptor> descriptors =
610        connectionCache.getAdmin().listTableDescriptorsByNamespace(Bytes.toBytes(name));
611      return tableDescriptorsFromHBase(descriptors);
612    } catch (IOException e) {
613      throw getTIOError(e);
614    }
615  }
616
617  @Override
618  public List<TTableName> getTableNamesByPattern(String regex, boolean includeSysTables)
619    throws TIOError, TException {
620    try {
621      Pattern pattern = (regex == null ? null : Pattern.compile(regex));
622      TableName[] tableNames = connectionCache.getAdmin().listTableNames(pattern, includeSysTables);
623      return tableNamesFromHBase(tableNames);
624    } catch (IOException e) {
625      throw getTIOError(e);
626    }
627  }
628
629  @Override
630  public List<TTableName> getTableNamesByNamespace(String name) throws TIOError, TException {
631    try {
632      TableName[] tableNames = connectionCache.getAdmin().listTableNamesByNamespace(name);
633      return tableNamesFromHBase(tableNames);
634    } catch (IOException e) {
635      throw getTIOError(e);
636    }
637  }
638
639  @Override
640  public void createTable(TTableDescriptor desc, List<ByteBuffer> splitKeys)
641    throws TIOError, TException {
642    try {
643      TableDescriptor descriptor = tableDescriptorFromThrift(desc);
644      byte[][] split = splitKeyFromThrift(splitKeys);
645      if (split != null) {
646        connectionCache.getAdmin().createTable(descriptor, split);
647      } else {
648        connectionCache.getAdmin().createTable(descriptor);
649      }
650    } catch (IOException e) {
651      throw getTIOError(e);
652    }
653  }
654
655  @Override
656  public void deleteTable(TTableName tableName) throws TIOError, TException {
657    try {
658      TableName table = tableNameFromThrift(tableName);
659      connectionCache.getAdmin().deleteTable(table);
660    } catch (IOException e) {
661      throw getTIOError(e);
662    }
663  }
664
665  @Override
666  public void truncateTable(TTableName tableName, boolean preserveSplits)
667    throws TIOError, TException {
668    try {
669      TableName table = tableNameFromThrift(tableName);
670      connectionCache.getAdmin().truncateTable(table, preserveSplits);
671    } catch (IOException e) {
672      throw getTIOError(e);
673    }
674  }
675
676  @Override
677  public void enableTable(TTableName tableName) throws TIOError, TException {
678    try {
679      TableName table = tableNameFromThrift(tableName);
680      connectionCache.getAdmin().enableTable(table);
681    } catch (IOException e) {
682      throw getTIOError(e);
683    }
684  }
685
686  @Override
687  public void disableTable(TTableName tableName) throws TIOError, TException {
688    try {
689      TableName table = tableNameFromThrift(tableName);
690      connectionCache.getAdmin().disableTable(table);
691    } catch (IOException e) {
692      throw getTIOError(e);
693    }
694  }
695
696  @Override
697  public boolean isTableEnabled(TTableName tableName) throws TIOError, TException {
698    try {
699      TableName table = tableNameFromThrift(tableName);
700      return connectionCache.getAdmin().isTableEnabled(table);
701    } catch (IOException e) {
702      throw getTIOError(e);
703    }
704  }
705
706  @Override
707  public boolean isTableDisabled(TTableName tableName) throws TIOError, TException {
708    try {
709      TableName table = tableNameFromThrift(tableName);
710      return connectionCache.getAdmin().isTableDisabled(table);
711    } catch (IOException e) {
712      throw getTIOError(e);
713    }
714  }
715
716  @Override
717  public boolean isTableAvailable(TTableName tableName) throws TIOError, TException {
718    try {
719      TableName table = tableNameFromThrift(tableName);
720      return connectionCache.getAdmin().isTableAvailable(table);
721    } catch (IOException e) {
722      throw getTIOError(e);
723    }
724  }
725
726  @Override
727  public boolean isTableAvailableWithSplit(TTableName tableName, List<ByteBuffer> splitKeys)
728    throws TIOError, TException {
729    throw new NotImplementedException("isTableAvailableWithSplit not supported");
730  }
731
732  @Override
733  public void addColumnFamily(TTableName tableName, TColumnFamilyDescriptor column)
734    throws TIOError, TException {
735    try {
736      TableName table = tableNameFromThrift(tableName);
737      ColumnFamilyDescriptor columnFamilyDescriptor = columnFamilyDescriptorFromThrift(column);
738      connectionCache.getAdmin().addColumnFamily(table, columnFamilyDescriptor);
739    } catch (IOException e) {
740      throw getTIOError(e);
741    }
742  }
743
744  @Override
745  public void deleteColumnFamily(TTableName tableName, ByteBuffer column)
746    throws TIOError, TException {
747    try {
748      TableName table = tableNameFromThrift(tableName);
749      connectionCache.getAdmin().deleteColumnFamily(table, column.array());
750    } catch (IOException e) {
751      throw getTIOError(e);
752    }
753  }
754
755  @Override
756  public void modifyColumnFamily(TTableName tableName, TColumnFamilyDescriptor column)
757    throws TIOError, TException {
758    try {
759      TableName table = tableNameFromThrift(tableName);
760      ColumnFamilyDescriptor columnFamilyDescriptor = columnFamilyDescriptorFromThrift(column);
761      connectionCache.getAdmin().modifyColumnFamily(table, columnFamilyDescriptor);
762    } catch (IOException e) {
763      throw getTIOError(e);
764    }
765  }
766
767  @Override
768  public void modifyTable(TTableDescriptor desc) throws TIOError, TException {
769    try {
770      TableDescriptor descriptor = tableDescriptorFromThrift(desc);
771      connectionCache.getAdmin().modifyTable(descriptor);
772    } catch (IOException e) {
773      throw getTIOError(e);
774    }
775  }
776
777  @Override
778  public void createNamespace(TNamespaceDescriptor namespaceDesc) throws TIOError, TException {
779    try {
780      NamespaceDescriptor descriptor = namespaceDescriptorFromThrift(namespaceDesc);
781      connectionCache.getAdmin().createNamespace(descriptor);
782    } catch (IOException e) {
783      throw getTIOError(e);
784    }
785  }
786
787  @Override
788  public void modifyNamespace(TNamespaceDescriptor namespaceDesc) throws TIOError, TException {
789    try {
790      NamespaceDescriptor descriptor = namespaceDescriptorFromThrift(namespaceDesc);
791      connectionCache.getAdmin().modifyNamespace(descriptor);
792    } catch (IOException e) {
793      throw getTIOError(e);
794    }
795  }
796
797  @Override
798  public void deleteNamespace(String name) throws TIOError, TException {
799    try {
800      connectionCache.getAdmin().deleteNamespace(name);
801    } catch (IOException e) {
802      throw getTIOError(e);
803    }
804  }
805
806  @Override
807  public TNamespaceDescriptor getNamespaceDescriptor(String name) throws TIOError, TException {
808    try {
809      NamespaceDescriptor descriptor = connectionCache.getAdmin().getNamespaceDescriptor(name);
810      return namespaceDescriptorFromHBase(descriptor);
811    } catch (IOException e) {
812      throw getTIOError(e);
813    }
814  }
815
816  @Override
817  public List<String> listNamespaces() throws TIOError, TException {
818    try {
819      String[] namespaces = connectionCache.getAdmin().listNamespaces();
820      List<String> result = new ArrayList<>(namespaces.length);
821      for (String ns : namespaces) {
822        result.add(ns);
823      }
824      return result;
825    } catch (IOException e) {
826      throw getTIOError(e);
827    }
828  }
829
830  @Override
831  public TThriftServerType getThriftServerType() {
832    return TThriftServerType.TWO;
833  }
834
835  @Override
836  public String getClusterId() throws TException {
837    return connectionCache.getClusterId();
838  }
839
840  @Override
841  public List<TOnlineLogRecord> getSlowLogResponses(Set<TServerName> tServerNames,
842    TLogQueryFilter tLogQueryFilter) throws TIOError, TException {
843    try {
844      Set<ServerName> serverNames = ThriftUtilities.getServerNamesFromThrift(tServerNames);
845      LogQueryFilter logQueryFilter = ThriftUtilities.getSlowLogQueryFromThrift(tLogQueryFilter);
846      List<OnlineLogRecord> onlineLogRecords =
847        connectionCache.getAdmin().getSlowLogResponses(serverNames, logQueryFilter);
848      return ThriftUtilities.getSlowLogRecordsFromHBase(onlineLogRecords);
849    } catch (IOException e) {
850      throw getTIOError(e);
851    }
852  }
853
854  @Override
855  public List<Boolean> clearSlowLogResponses(Set<TServerName> tServerNames)
856    throws TIOError, TException {
857    Set<ServerName> serverNames = ThriftUtilities.getServerNamesFromThrift(tServerNames);
858    try {
859      return connectionCache.getAdmin().clearSlowLogResponses(serverNames);
860    } catch (IOException e) {
861      throw getTIOError(e);
862    }
863  }
864
865  @Override
866  public boolean grant(TAccessControlEntity info) throws TIOError, TException {
867    Permission.Action[] actions = ThriftUtilities.permissionActionsFromString(info.actions);
868    try {
869      if (info.scope == TPermissionScope.NAMESPACE) {
870        AccessControlClient.grant(connectionCache.getAdmin().getConnection(), info.getNsName(),
871          info.getUsername(), actions);
872      } else if (info.scope == TPermissionScope.TABLE) {
873        TableName tableName = TableName.valueOf(info.getTableName());
874        AccessControlClient.grant(connectionCache.getAdmin().getConnection(), tableName,
875          info.getUsername(), null, null, actions);
876      }
877    } catch (Throwable t) {
878      if (t instanceof IOException) {
879        throw getTIOError((IOException) t);
880      } else {
881        throw getTIOError(new DoNotRetryIOException(t.getMessage()));
882      }
883    }
884    return true;
885  }
886
887  @Override
888  public boolean revoke(TAccessControlEntity info) throws TIOError, TException {
889    Permission.Action[] actions = ThriftUtilities.permissionActionsFromString(info.actions);
890    try {
891      if (info.scope == TPermissionScope.NAMESPACE) {
892        AccessControlClient.revoke(connectionCache.getAdmin().getConnection(), info.getNsName(),
893          info.getUsername(), actions);
894      } else if (info.scope == TPermissionScope.TABLE) {
895        TableName tableName = TableName.valueOf(info.getTableName());
896        AccessControlClient.revoke(connectionCache.getAdmin().getConnection(), tableName,
897          info.getUsername(), null, null, actions);
898      }
899    } catch (Throwable t) {
900      if (t instanceof IOException) {
901        throw getTIOError((IOException) t);
902      } else {
903        throw getTIOError(new DoNotRetryIOException(t.getMessage()));
904      }
905    }
906    return true;
907  }
908
909  @Override
910  public List<TNamespaceDescriptor> listNamespaceDescriptors() throws TIOError, TException {
911    try {
912      NamespaceDescriptor[] descriptors = connectionCache.getAdmin().listNamespaceDescriptors();
913      return namespaceDescriptorsFromHBase(descriptors);
914    } catch (IOException e) {
915      throw getTIOError(e);
916    }
917  }
918}