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