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.util.Bytes.getBytes;
021
022import java.io.IOException;
023import java.nio.ByteBuffer;
024import java.util.ArrayList;
025import java.util.Collections;
026import java.util.HashSet;
027import java.util.List;
028import java.util.Map;
029import java.util.NavigableSet;
030import java.util.Set;
031import java.util.stream.Collectors;
032import org.apache.hadoop.hbase.Cell;
033import org.apache.hadoop.hbase.CellBuilderFactory;
034import org.apache.hadoop.hbase.CellBuilderType;
035import org.apache.hadoop.hbase.CellUtil;
036import org.apache.hadoop.hbase.CompareOperator;
037import org.apache.hadoop.hbase.ExtendedCell;
038import org.apache.hadoop.hbase.ExtendedCellBuilder;
039import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
040import org.apache.hadoop.hbase.HConstants;
041import org.apache.hadoop.hbase.HRegionLocation;
042import org.apache.hadoop.hbase.KeepDeletedCells;
043import org.apache.hadoop.hbase.NamespaceDescriptor;
044import org.apache.hadoop.hbase.PrivateCellUtil;
045import org.apache.hadoop.hbase.ServerName;
046import org.apache.hadoop.hbase.TableName;
047import org.apache.hadoop.hbase.client.Append;
048import org.apache.hadoop.hbase.client.ClientInternalHelper;
049import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
050import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
051import org.apache.hadoop.hbase.client.Consistency;
052import org.apache.hadoop.hbase.client.Delete;
053import org.apache.hadoop.hbase.client.Durability;
054import org.apache.hadoop.hbase.client.Get;
055import org.apache.hadoop.hbase.client.Increment;
056import org.apache.hadoop.hbase.client.LogQueryFilter;
057import org.apache.hadoop.hbase.client.Mutation;
058import org.apache.hadoop.hbase.client.OnlineLogRecord;
059import org.apache.hadoop.hbase.client.OperationWithAttributes;
060import org.apache.hadoop.hbase.client.Put;
061import org.apache.hadoop.hbase.client.RegionInfo;
062import org.apache.hadoop.hbase.client.Result;
063import org.apache.hadoop.hbase.client.RowMutations;
064import org.apache.hadoop.hbase.client.Scan;
065import org.apache.hadoop.hbase.client.Scan.ReadType;
066import org.apache.hadoop.hbase.client.TableDescriptor;
067import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
068import org.apache.hadoop.hbase.exceptions.DeserializationException;
069import org.apache.hadoop.hbase.filter.Filter;
070import org.apache.hadoop.hbase.filter.ParseFilter;
071import org.apache.hadoop.hbase.io.TimeRange;
072import org.apache.hadoop.hbase.io.compress.Compression;
073import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
074import org.apache.hadoop.hbase.regionserver.BloomType;
075import org.apache.hadoop.hbase.security.access.Permission;
076import org.apache.hadoop.hbase.security.visibility.Authorizations;
077import org.apache.hadoop.hbase.security.visibility.CellVisibility;
078import org.apache.hadoop.hbase.thrift2.generated.TAppend;
079import org.apache.hadoop.hbase.thrift2.generated.TAuthorization;
080import org.apache.hadoop.hbase.thrift2.generated.TBloomFilterType;
081import org.apache.hadoop.hbase.thrift2.generated.TCellVisibility;
082import org.apache.hadoop.hbase.thrift2.generated.TColumn;
083import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
084import org.apache.hadoop.hbase.thrift2.generated.TColumnIncrement;
085import org.apache.hadoop.hbase.thrift2.generated.TColumnValue;
086import org.apache.hadoop.hbase.thrift2.generated.TCompareOperator;
087import org.apache.hadoop.hbase.thrift2.generated.TCompressionAlgorithm;
088import org.apache.hadoop.hbase.thrift2.generated.TConsistency;
089import org.apache.hadoop.hbase.thrift2.generated.TDataBlockEncoding;
090import org.apache.hadoop.hbase.thrift2.generated.TDelete;
091import org.apache.hadoop.hbase.thrift2.generated.TDeleteType;
092import org.apache.hadoop.hbase.thrift2.generated.TDurability;
093import org.apache.hadoop.hbase.thrift2.generated.TFilterByOperator;
094import org.apache.hadoop.hbase.thrift2.generated.TGet;
095import org.apache.hadoop.hbase.thrift2.generated.THRegionInfo;
096import org.apache.hadoop.hbase.thrift2.generated.THRegionLocation;
097import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
098import org.apache.hadoop.hbase.thrift2.generated.TKeepDeletedCells;
099import org.apache.hadoop.hbase.thrift2.generated.TLogQueryFilter;
100import org.apache.hadoop.hbase.thrift2.generated.TLogType;
101import org.apache.hadoop.hbase.thrift2.generated.TMutation;
102import org.apache.hadoop.hbase.thrift2.generated.TNamespaceDescriptor;
103import org.apache.hadoop.hbase.thrift2.generated.TOnlineLogRecord;
104import org.apache.hadoop.hbase.thrift2.generated.TPut;
105import org.apache.hadoop.hbase.thrift2.generated.TReadType;
106import org.apache.hadoop.hbase.thrift2.generated.TResult;
107import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
108import org.apache.hadoop.hbase.thrift2.generated.TScan;
109import org.apache.hadoop.hbase.thrift2.generated.TServerName;
110import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
111import org.apache.hadoop.hbase.thrift2.generated.TTableName;
112import org.apache.hadoop.hbase.thrift2.generated.TTimeRange;
113import org.apache.hadoop.hbase.util.Bytes;
114import org.apache.yetus.audience.InterfaceAudience;
115
116import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
117import org.apache.hbase.thirdparty.org.apache.commons.collections4.MapUtils;
118
119import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
120import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
121
122@InterfaceAudience.Private
123public final class ThriftUtilities {
124
125  private final static ExtendedCell[] EMPTY_CELL_ARRAY = new ExtendedCell[0];
126  private final static Result EMPTY_RESULT = Result.create(EMPTY_CELL_ARRAY);
127  private final static Result EMPTY_RESULT_STALE = Result.create(EMPTY_CELL_ARRAY, null, true);
128
129  private ThriftUtilities() {
130    throw new UnsupportedOperationException("Can't initialize class");
131  }
132
133  /**
134   * Creates a {@link Get} (HBase) from a {@link TGet} (Thrift). This ignores any timestamps set on
135   * {@link TColumn} objects.
136   * @param in the <code>TGet</code> to convert
137   * @return <code>Get</code> object
138   * @throws IOException if an invalid time range or max version parameter is given
139   */
140  public static Get getFromThrift(TGet in) throws IOException {
141    Get out = new Get(in.getRow());
142
143    // Timestamp overwrites time range if both are set
144    if (in.isSetTimestamp()) {
145      out.setTimestamp(in.getTimestamp());
146    } else if (in.isSetTimeRange()) {
147      out.setTimeRange(in.getTimeRange().getMinStamp(), in.getTimeRange().getMaxStamp());
148    }
149
150    if (in.isSetMaxVersions()) {
151      out.readVersions(in.getMaxVersions());
152    }
153
154    if (in.isSetFilterString()) {
155      ParseFilter parseFilter = new ParseFilter();
156      out.setFilter(parseFilter.parseFilterString(in.getFilterString()));
157    }
158
159    if (in.isSetAttributes()) {
160      addAttributes(out, in.getAttributes());
161    }
162
163    if (in.isSetAuthorizations()) {
164      out.setAuthorizations(new Authorizations(in.getAuthorizations().getLabels()));
165    }
166
167    if (in.isSetConsistency()) {
168      out.setConsistency(consistencyFromThrift(in.getConsistency()));
169    }
170
171    if (in.isSetTargetReplicaId()) {
172      out.setReplicaId(in.getTargetReplicaId());
173    }
174
175    if (in.isSetCacheBlocks()) {
176      out.setCacheBlocks(in.isCacheBlocks());
177    }
178    if (in.isSetStoreLimit()) {
179      out.setMaxResultsPerColumnFamily(in.getStoreLimit());
180    }
181    if (in.isSetStoreOffset()) {
182      out.setRowOffsetPerColumnFamily(in.getStoreOffset());
183    }
184    if (in.isSetExistence_only()) {
185      out.setCheckExistenceOnly(in.isExistence_only());
186    }
187
188    if (in.isSetColumns()) {
189      for (TColumn column : in.getColumns()) {
190        if (column.isSetQualifier()) {
191          out.addColumn(column.getFamily(), column.getQualifier());
192        } else {
193          out.addFamily(column.getFamily());
194        }
195      }
196    }
197
198    if (in.isSetFilterBytes()) {
199      out.setFilter(filterFromThrift(in.getFilterBytes()));
200    }
201    return out;
202  }
203
204  /**
205   * Converts multiple {@link TGet}s (Thrift) into a list of {@link Get}s (HBase).
206   * @param in list of <code>TGet</code>s to convert
207   * @return list of <code>Get</code> objects
208   * @throws IOException if an invalid time range or max version parameter is given
209   * @see #getFromThrift(TGet)
210   */
211  public static List<Get> getsFromThrift(List<TGet> in) throws IOException {
212    List<Get> out = new ArrayList<>(in.size());
213    for (TGet get : in) {
214      out.add(getFromThrift(get));
215    }
216    return out;
217  }
218
219  /**
220   * Creates a {@link TResult} (Thrift) from a {@link Result} (HBase).
221   * @param in the <code>Result</code> to convert
222   * @return converted result, returns an empty result if the input is <code>null</code>
223   */
224  public static TResult resultFromHBase(Result in) {
225    ExtendedCell[] raw = ClientInternalHelper.getExtendedRawCells(in);
226    TResult out = new TResult();
227    byte[] row = in.getRow();
228    if (row != null) {
229      out.setRow(in.getRow());
230    }
231    List<TColumnValue> columnValues = new ArrayList<>(raw.length);
232    for (ExtendedCell kv : raw) {
233      TColumnValue col = new TColumnValue();
234      col.setFamily(CellUtil.cloneFamily(kv));
235      col.setQualifier(CellUtil.cloneQualifier(kv));
236      col.setTimestamp(kv.getTimestamp());
237      col.setValue(CellUtil.cloneValue(kv));
238      col.setType(kv.getType().getCode());
239      if (kv.getTagsLength() > 0) {
240        col.setTags(PrivateCellUtil.cloneTags(kv));
241      }
242      columnValues.add(col);
243    }
244    out.setColumnValues(columnValues);
245
246    out.setStale(in.isStale());
247
248    out.setPartial(in.mayHaveMoreCellsInRow());
249    return out;
250  }
251
252  /**
253   * Converts multiple {@link Result}s (HBase) into a list of {@link TResult}s (Thrift).
254   * @param in array of <code>Result</code>s to convert
255   * @return list of converted <code>TResult</code>s
256   * @see #resultFromHBase(Result)
257   */
258  public static List<TResult> resultsFromHBase(Result[] in) {
259    List<TResult> out = new ArrayList<>(in.length);
260    for (Result result : in) {
261      out.add(resultFromHBase(result));
262    }
263    return out;
264  }
265
266  /**
267   * Creates a {@link Put} (HBase) from a {@link TPut} (Thrift)
268   * @param in the <code>TPut</code> to convert
269   * @return converted <code>Put</code>
270   */
271  public static Put putFromThrift(TPut in) {
272    Put out;
273
274    if (in.isSetTimestamp()) {
275      out = new Put(in.getRow(), in.getTimestamp());
276    } else {
277      out = new Put(in.getRow());
278    }
279
280    if (in.isSetDurability()) {
281      out.setDurability(durabilityFromThrift(in.getDurability()));
282    }
283
284    for (TColumnValue columnValue : in.getColumnValues()) {
285      try {
286        if (columnValue.isSetTimestamp()) {
287          out.add(CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(out.getRow())
288            .setFamily(columnValue.getFamily()).setQualifier(columnValue.getQualifier())
289            .setTimestamp(columnValue.getTimestamp()).setType(Cell.Type.Put)
290            .setValue(columnValue.getValue()).build());
291        } else {
292          out.add(CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(out.getRow())
293            .setFamily(columnValue.getFamily()).setQualifier(columnValue.getQualifier())
294            .setTimestamp(out.getTimestamp()).setType(Cell.Type.Put)
295            .setValue(columnValue.getValue()).build());
296        }
297      } catch (IOException e) {
298        throw new IllegalArgumentException((e));
299      }
300    }
301
302    if (in.isSetAttributes()) {
303      addAttributes(out, in.getAttributes());
304    }
305
306    if (in.getCellVisibility() != null) {
307      out.setCellVisibility(new CellVisibility(in.getCellVisibility().getExpression()));
308    }
309
310    return out;
311  }
312
313  /**
314   * Converts multiple {@link TPut}s (Thrift) into a list of {@link Put}s (HBase).
315   * @param in list of <code>TPut</code>s to convert
316   * @return list of converted <code>Put</code>s
317   * @see #putFromThrift(TPut)
318   */
319  public static List<Put> putsFromThrift(List<TPut> in) {
320    List<Put> out = new ArrayList<>(in.size());
321    for (TPut put : in) {
322      out.add(putFromThrift(put));
323    }
324    return out;
325  }
326
327  /**
328   * Creates a {@link Delete} (HBase) from a {@link TDelete} (Thrift).
329   * @param in the <code>TDelete</code> to convert
330   * @return converted <code>Delete</code>
331   */
332  public static Delete deleteFromThrift(TDelete in) {
333    Delete out;
334
335    if (in.isSetColumns()) {
336      out = new Delete(in.getRow());
337      for (TColumn column : in.getColumns()) {
338        if (in.isSetDeleteType()) {
339          switch (in.getDeleteType()) {
340            case DELETE_COLUMN:
341              if (column.isSetTimestamp()) {
342                out.addColumn(column.getFamily(), column.getQualifier(), column.getTimestamp());
343              } else {
344                out.addColumn(column.getFamily(), column.getQualifier());
345              }
346              break;
347            case DELETE_COLUMNS:
348              if (column.isSetTimestamp()) {
349                out.addColumns(column.getFamily(), column.getQualifier(), column.getTimestamp());
350              } else {
351                out.addColumns(column.getFamily(), column.getQualifier());
352              }
353              break;
354            case DELETE_FAMILY:
355              if (column.isSetTimestamp()) {
356                out.addFamily(column.getFamily(), column.getTimestamp());
357              } else {
358                out.addFamily(column.getFamily());
359              }
360              break;
361            case DELETE_FAMILY_VERSION:
362              if (column.isSetTimestamp()) {
363                out.addFamilyVersion(column.getFamily(), column.getTimestamp());
364              } else {
365                throw new IllegalArgumentException(
366                  "Timestamp is required for TDelete with DeleteFamilyVersion type");
367              }
368              break;
369            default:
370              throw new IllegalArgumentException("DeleteType is required for TDelete");
371          }
372        } else {
373          throw new IllegalArgumentException("DeleteType is required for TDelete");
374        }
375      }
376    } else {
377      if (in.isSetTimestamp()) {
378        out = new Delete(in.getRow(), in.getTimestamp());
379      } else {
380        out = new Delete(in.getRow());
381      }
382    }
383
384    if (in.isSetAttributes()) {
385      addAttributes(out, in.getAttributes());
386    }
387
388    if (in.isSetDurability()) {
389      out.setDurability(durabilityFromThrift(in.getDurability()));
390    }
391
392    return out;
393  }
394
395  /**
396   * Converts multiple {@link TDelete}s (Thrift) into a list of {@link Delete}s (HBase).
397   * @param in list of <code>TDelete</code>s to convert
398   * @return list of converted <code>Delete</code>s
399   * @see #deleteFromThrift(TDelete)
400   */
401
402  public static List<Delete> deletesFromThrift(List<TDelete> in) {
403    List<Delete> out = new ArrayList<>(in.size());
404    for (TDelete delete : in) {
405      out.add(deleteFromThrift(delete));
406    }
407    return out;
408  }
409
410  public static TDeleteType deleteTypeFromHBase(Cell.Type type) {
411    switch (type) {
412      case Delete:
413        return TDeleteType.DELETE_COLUMN;
414      case DeleteColumn:
415        return TDeleteType.DELETE_COLUMNS;
416      case DeleteFamily:
417        return TDeleteType.DELETE_FAMILY;
418      case DeleteFamilyVersion:
419        return TDeleteType.DELETE_FAMILY_VERSION;
420      default:
421        throw new IllegalArgumentException("Unknow delete type " + type);
422    }
423  }
424
425  public static TDelete deleteFromHBase(Delete in) {
426    TDelete out = new TDelete(ByteBuffer.wrap(in.getRow()));
427
428    List<TColumn> columns = new ArrayList<>(in.getFamilyCellMap().entrySet().size());
429    long rowTimestamp = in.getTimestamp();
430    if (rowTimestamp != HConstants.LATEST_TIMESTAMP) {
431      out.setTimestamp(rowTimestamp);
432    }
433
434    for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
435      out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
436        ByteBuffer.wrap(attribute.getValue()));
437    }
438    if (in.getDurability() != Durability.USE_DEFAULT) {
439      out.setDurability(durabilityFromHBase(in.getDurability()));
440    }
441    // Delete the whole row
442    if (in.getFamilyCellMap().size() == 0) {
443      return out;
444    }
445    TDeleteType type = null;
446    for (Map.Entry<byte[], List<Cell>> familyEntry : in.getFamilyCellMap().entrySet()) {
447      byte[] family = familyEntry.getKey();
448      TColumn column = new TColumn(ByteBuffer.wrap(familyEntry.getKey()));
449      for (Cell cell : familyEntry.getValue()) {
450        TDeleteType cellDeleteType = deleteTypeFromHBase(cell.getType());
451        if (type == null) {
452          type = cellDeleteType;
453        } else if (type != cellDeleteType) {
454          throw new RuntimeException("Only the same delete type is supported, but two delete type "
455            + "is founded, one is " + type + " the other one is " + cellDeleteType);
456        }
457        byte[] qualifier = CellUtil.cloneQualifier(cell);
458        long timestamp = cell.getTimestamp();
459        column.setFamily(family);
460        if (qualifier != null) {
461          column.setQualifier(qualifier);
462        }
463        if (timestamp != HConstants.LATEST_TIMESTAMP) {
464          column.setTimestamp(timestamp);
465        }
466      }
467      columns.add(column);
468    }
469    out.setColumns(columns);
470    out.setDeleteType(type);
471
472    return out;
473  }
474
475  /**
476   * Creates a {@link RowMutations} (HBase) from a {@link TRowMutations} (Thrift)
477   * @param in the <code>TRowMutations</code> to convert
478   * @return converted <code>RowMutations</code>
479   */
480  public static RowMutations rowMutationsFromThrift(TRowMutations in) throws IOException {
481    List<TMutation> mutations = in.getMutations();
482    RowMutations out = new RowMutations(in.getRow(), mutations.size());
483    for (TMutation mutation : mutations) {
484      if (mutation.isSetPut()) {
485        out.add(putFromThrift(mutation.getPut()));
486      }
487      if (mutation.isSetDeleteSingle()) {
488        out.add(deleteFromThrift(mutation.getDeleteSingle()));
489      }
490    }
491    return out;
492  }
493
494  public static Scan scanFromThrift(TScan in) throws IOException {
495    Scan out = new Scan();
496
497    if (in.isSetStartRow()) {
498      out.withStartRow(in.getStartRow());
499    }
500    if (in.isSetStopRow()) {
501      out.withStopRow(in.getStopRow());
502    }
503    if (in.isSetCaching()) {
504      out.setCaching(in.getCaching());
505    }
506    if (in.isSetMaxVersions()) {
507      out.readVersions(in.getMaxVersions());
508    }
509
510    if (in.isSetColumns()) {
511      for (TColumn column : in.getColumns()) {
512        if (column.isSetQualifier()) {
513          out.addColumn(column.getFamily(), column.getQualifier());
514        } else {
515          out.addFamily(column.getFamily());
516        }
517      }
518    }
519
520    TTimeRange timeRange = in.getTimeRange();
521    if (timeRange != null && timeRange.isSetMinStamp() && timeRange.isSetMaxStamp()) {
522      out.setTimeRange(timeRange.getMinStamp(), timeRange.getMaxStamp());
523    }
524
525    if (in.isSetBatchSize()) {
526      out.setBatch(in.getBatchSize());
527    }
528
529    if (in.isSetFilterString()) {
530      ParseFilter parseFilter = new ParseFilter();
531      out.setFilter(parseFilter.parseFilterString(in.getFilterString()));
532    }
533
534    if (in.isSetAttributes()) {
535      addAttributes(out, in.getAttributes());
536    }
537
538    if (in.isSetAuthorizations()) {
539      out.setAuthorizations(new Authorizations(in.getAuthorizations().getLabels()));
540    }
541
542    if (in.isSetReversed()) {
543      out.setReversed(in.isReversed());
544    }
545
546    if (in.isSetCacheBlocks()) {
547      out.setCacheBlocks(in.isCacheBlocks());
548    }
549
550    if (in.isSetColFamTimeRangeMap()) {
551      Map<ByteBuffer, TTimeRange> colFamTimeRangeMap = in.getColFamTimeRangeMap();
552      if (MapUtils.isNotEmpty(colFamTimeRangeMap)) {
553        for (Map.Entry<ByteBuffer, TTimeRange> entry : colFamTimeRangeMap.entrySet()) {
554          out.setColumnFamilyTimeRange(Bytes.toBytes(entry.getKey()),
555            entry.getValue().getMinStamp(), entry.getValue().getMaxStamp());
556        }
557      }
558    }
559
560    if (in.isSetReadType()) {
561      out.setReadType(readTypeFromThrift(in.getReadType()));
562    }
563
564    if (in.isSetLimit()) {
565      out.setLimit(in.getLimit());
566    }
567
568    if (in.isSetConsistency()) {
569      out.setConsistency(consistencyFromThrift(in.getConsistency()));
570    }
571
572    if (in.isSetTargetReplicaId()) {
573      out.setReplicaId(in.getTargetReplicaId());
574    }
575
576    if (in.isSetFilterBytes()) {
577      out.setFilter(filterFromThrift(in.getFilterBytes()));
578    }
579
580    return out;
581  }
582
583  public static byte[] filterFromHBase(Filter filter) throws IOException {
584    FilterProtos.Filter filterPB = ProtobufUtil.toFilter(filter);
585    return filterPB.toByteArray();
586  }
587
588  public static Filter filterFromThrift(byte[] filterBytes) throws IOException {
589    FilterProtos.Filter filterPB = FilterProtos.Filter.parseFrom(filterBytes);
590    return ProtobufUtil.toFilter(filterPB);
591  }
592
593  public static TScan scanFromHBase(Scan in) throws IOException {
594    TScan out = new TScan();
595    out.setStartRow(in.getStartRow());
596    out.setStopRow(in.getStopRow());
597    out.setCaching(in.getCaching());
598    out.setMaxVersions(in.getMaxVersions());
599    for (Map.Entry<byte[], NavigableSet<byte[]>> family : in.getFamilyMap().entrySet()) {
600
601      if (family.getValue() != null && !family.getValue().isEmpty()) {
602        for (byte[] qualifier : family.getValue()) {
603          TColumn column = new TColumn();
604          column.setFamily(family.getKey());
605          column.setQualifier(qualifier);
606          out.addToColumns(column);
607        }
608      } else {
609        TColumn column = new TColumn();
610        column.setFamily(family.getKey());
611        out.addToColumns(column);
612      }
613    }
614    TTimeRange tTimeRange = new TTimeRange();
615    tTimeRange.setMinStamp(in.getTimeRange().getMin()).setMaxStamp(in.getTimeRange().getMax());
616    out.setTimeRange(tTimeRange);
617    out.setBatchSize(in.getBatch());
618
619    for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
620      out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
621        ByteBuffer.wrap(attribute.getValue()));
622    }
623
624    try {
625      Authorizations authorizations = in.getAuthorizations();
626      if (authorizations != null) {
627        TAuthorization tAuthorization = new TAuthorization();
628        tAuthorization.setLabels(authorizations.getLabels());
629        out.setAuthorizations(tAuthorization);
630      }
631    } catch (DeserializationException e) {
632      throw new RuntimeException(e);
633    }
634
635    out.setReversed(in.isReversed());
636    out.setCacheBlocks(in.getCacheBlocks());
637    out.setReadType(readTypeFromHBase(in.getReadType()));
638    out.setLimit(in.getLimit());
639    out.setConsistency(consistencyFromHBase(in.getConsistency()));
640    out.setTargetReplicaId(in.getReplicaId());
641    for (Map.Entry<byte[], TimeRange> entry : in.getColumnFamilyTimeRange().entrySet()) {
642      if (entry.getValue() != null) {
643        TTimeRange timeRange = new TTimeRange();
644        timeRange.setMinStamp(entry.getValue().getMin()).setMaxStamp(entry.getValue().getMax());
645        out.putToColFamTimeRangeMap(ByteBuffer.wrap(entry.getKey()), timeRange);
646      }
647    }
648    if (in.getFilter() != null) {
649      try {
650        out.setFilterBytes(filterFromHBase(in.getFilter()));
651      } catch (IOException ioE) {
652        throw new RuntimeException(ioE);
653      }
654    }
655    return out;
656  }
657
658  public static Increment incrementFromThrift(TIncrement in) throws IOException {
659    Increment out = new Increment(in.getRow());
660    for (TColumnIncrement column : in.getColumns()) {
661      out.addColumn(column.getFamily(), column.getQualifier(), column.getAmount());
662    }
663
664    if (in.isSetAttributes()) {
665      addAttributes(out, in.getAttributes());
666    }
667
668    if (in.isSetDurability()) {
669      out.setDurability(durabilityFromThrift(in.getDurability()));
670    }
671
672    if (in.getCellVisibility() != null) {
673      out.setCellVisibility(new CellVisibility(in.getCellVisibility().getExpression()));
674    }
675
676    if (in.isSetReturnResults()) {
677      out.setReturnResults(in.isReturnResults());
678    }
679
680    return out;
681  }
682
683  public static Append appendFromThrift(TAppend append) throws IOException {
684    Append out = new Append(append.getRow());
685    for (TColumnValue column : append.getColumns()) {
686      out.addColumn(column.getFamily(), column.getQualifier(), column.getValue());
687    }
688
689    if (append.isSetAttributes()) {
690      addAttributes(out, append.getAttributes());
691    }
692
693    if (append.isSetDurability()) {
694      out.setDurability(durabilityFromThrift(append.getDurability()));
695    }
696
697    if (append.getCellVisibility() != null) {
698      out.setCellVisibility(new CellVisibility(append.getCellVisibility().getExpression()));
699    }
700
701    if (append.isSetReturnResults()) {
702      out.setReturnResults(append.isReturnResults());
703    }
704
705    return out;
706  }
707
708  public static THRegionLocation regionLocationFromHBase(HRegionLocation hrl) {
709    RegionInfo hri = hrl.getRegion();
710    ServerName serverName = hrl.getServerName();
711
712    THRegionInfo thRegionInfo = new THRegionInfo();
713    THRegionLocation thRegionLocation = new THRegionLocation();
714    TServerName tServerName = new TServerName();
715
716    tServerName.setHostName(serverName.getHostname());
717    tServerName.setPort(serverName.getPort());
718    tServerName.setStartCode(serverName.getStartcode());
719
720    thRegionInfo.setTableName(hri.getTable().getName());
721    thRegionInfo.setEndKey(hri.getEndKey());
722    thRegionInfo.setStartKey(hri.getStartKey());
723    thRegionInfo.setOffline(hri.isOffline());
724    thRegionInfo.setSplit(hri.isSplit());
725    thRegionInfo.setReplicaId(hri.getReplicaId());
726
727    thRegionLocation.setRegionInfo(thRegionInfo);
728    thRegionLocation.setServerName(tServerName);
729
730    return thRegionLocation;
731  }
732
733  public static List<THRegionLocation> regionLocationsFromHBase(List<HRegionLocation> locations) {
734    List<THRegionLocation> tlocations = new ArrayList<>(locations.size());
735    for (HRegionLocation hrl : locations) {
736      tlocations.add(regionLocationFromHBase(hrl));
737    }
738    return tlocations;
739  }
740
741  /**
742   * Adds all the attributes into the Operation object
743   */
744  private static void addAttributes(OperationWithAttributes op,
745    Map<ByteBuffer, ByteBuffer> attributes) {
746    if (attributes == null || attributes.isEmpty()) {
747      return;
748    }
749    for (Map.Entry<ByteBuffer, ByteBuffer> entry : attributes.entrySet()) {
750      String name = Bytes.toStringBinary(getBytes(entry.getKey()));
751      byte[] value = getBytes(entry.getValue());
752      op.setAttribute(name, value);
753    }
754  }
755
756  private static Durability durabilityFromThrift(TDurability tDurability) {
757    switch (tDurability.getValue()) {
758      case 0:
759        return Durability.USE_DEFAULT;
760      case 1:
761        return Durability.SKIP_WAL;
762      case 2:
763        return Durability.ASYNC_WAL;
764      case 3:
765        return Durability.SYNC_WAL;
766      case 4:
767        return Durability.FSYNC_WAL;
768      default:
769        return Durability.USE_DEFAULT;
770    }
771  }
772
773  public static CompareOperator compareOpFromThrift(TCompareOperator tCompareOp) {
774    switch (tCompareOp.getValue()) {
775      case 0:
776        return CompareOperator.LESS;
777      case 1:
778        return CompareOperator.LESS_OR_EQUAL;
779      case 2:
780        return CompareOperator.EQUAL;
781      case 3:
782        return CompareOperator.NOT_EQUAL;
783      case 4:
784        return CompareOperator.GREATER_OR_EQUAL;
785      case 5:
786        return CompareOperator.GREATER;
787      case 6:
788        return CompareOperator.NO_OP;
789      default:
790        return null;
791    }
792  }
793
794  private static ReadType readTypeFromThrift(TReadType tReadType) {
795    switch (tReadType.getValue()) {
796      case 1:
797        return ReadType.DEFAULT;
798      case 2:
799        return ReadType.STREAM;
800      case 3:
801        return ReadType.PREAD;
802      default:
803        return null;
804    }
805  }
806
807  private static TReadType readTypeFromHBase(ReadType readType) {
808    switch (readType) {
809      case DEFAULT:
810        return TReadType.DEFAULT;
811      case STREAM:
812        return TReadType.STREAM;
813      case PREAD:
814        return TReadType.PREAD;
815      default:
816        return TReadType.DEFAULT;
817    }
818  }
819
820  private static Consistency consistencyFromThrift(TConsistency tConsistency) {
821    switch (tConsistency.getValue()) {
822      case 1:
823        return Consistency.STRONG;
824      case 2:
825        return Consistency.TIMELINE;
826      default:
827        return Consistency.STRONG;
828    }
829  }
830
831  public static TableName tableNameFromThrift(TTableName tableName) {
832    return TableName.valueOf(tableName.getNs(), tableName.getQualifier());
833  }
834
835  public static TableName[] tableNamesArrayFromThrift(List<TTableName> tableNames) {
836    TableName[] out = new TableName[tableNames.size()];
837    int index = 0;
838    for (TTableName tableName : tableNames) {
839      out[index++] = tableNameFromThrift(tableName);
840    }
841    return out;
842  }
843
844  public static List<TableName> tableNamesFromThrift(List<TTableName> tableNames) {
845    List<TableName> out = new ArrayList<>(tableNames.size());
846    for (TTableName tableName : tableNames) {
847      out.add(tableNameFromThrift(tableName));
848    }
849    return out;
850  }
851
852  public static TTableName tableNameFromHBase(TableName table) {
853    TTableName tableName = new TTableName();
854    tableName.setNs(table.getNamespace());
855    tableName.setQualifier(table.getQualifier());
856    return tableName;
857  }
858
859  public static List<TTableName> tableNamesFromHBase(List<TableName> in) {
860    List<TTableName> out = new ArrayList<>(in.size());
861    for (TableName tableName : in) {
862      out.add(tableNameFromHBase(tableName));
863    }
864    return out;
865  }
866
867  public static List<TTableName> tableNamesFromHBase(TableName[] in) {
868    List<TTableName> out = new ArrayList<>(in.length);
869    for (TableName tableName : in) {
870      out.add(tableNameFromHBase(tableName));
871    }
872    return out;
873  }
874
875  public static byte[][] splitKeyFromThrift(List<ByteBuffer> in) {
876    if (in == null || in.size() == 0) {
877      return null;
878    }
879    byte[][] out = new byte[in.size()][];
880    int index = 0;
881    for (ByteBuffer key : in) {
882      out[index++] = key.array();
883    }
884    return out;
885  }
886
887  public static BloomType bloomFilterFromThrift(TBloomFilterType in) {
888    switch (in.getValue()) {
889      case 0:
890        return BloomType.NONE;
891      case 1:
892        return BloomType.ROW;
893      case 2:
894        return BloomType.ROWCOL;
895      case 3:
896        return BloomType.ROWPREFIX_FIXED_LENGTH;
897      default:
898        return BloomType.ROW;
899    }
900  }
901
902  public static Compression.Algorithm compressionAlgorithmFromThrift(TCompressionAlgorithm in) {
903    switch (in.getValue()) {
904      case 0:
905        return Compression.Algorithm.LZO;
906      case 1:
907        return Compression.Algorithm.GZ;
908      case 2:
909        return Compression.Algorithm.NONE;
910      case 3:
911        return Compression.Algorithm.SNAPPY;
912      case 4:
913        return Compression.Algorithm.LZ4;
914      case 5:
915        return Compression.Algorithm.BZIP2;
916      case 6:
917        return Compression.Algorithm.ZSTD;
918      default:
919        return Compression.Algorithm.NONE;
920    }
921  }
922
923  public static DataBlockEncoding dataBlockEncodingFromThrift(TDataBlockEncoding in) {
924    switch (in.getValue()) {
925      case 0:
926        return DataBlockEncoding.NONE;
927      case 2:
928        return DataBlockEncoding.PREFIX;
929      case 3:
930        return DataBlockEncoding.DIFF;
931      case 4:
932        return DataBlockEncoding.FAST_DIFF;
933      case 7:
934        return DataBlockEncoding.ROW_INDEX_V1;
935      default:
936        return DataBlockEncoding.NONE;
937    }
938  }
939
940  public static KeepDeletedCells keepDeletedCellsFromThrift(TKeepDeletedCells in) {
941    switch (in.getValue()) {
942      case 0:
943        return KeepDeletedCells.FALSE;
944      case 1:
945        return KeepDeletedCells.TRUE;
946      case 2:
947        return KeepDeletedCells.TTL;
948      default:
949        return KeepDeletedCells.FALSE;
950    }
951  }
952
953  public static ColumnFamilyDescriptor
954    columnFamilyDescriptorFromThrift(TColumnFamilyDescriptor in) {
955    ColumnFamilyDescriptorBuilder builder = ColumnFamilyDescriptorBuilder.newBuilder(in.getName());
956
957    if (in.isSetAttributes()) {
958      for (Map.Entry<ByteBuffer, ByteBuffer> attribute : in.getAttributes().entrySet()) {
959        builder.setValue(attribute.getKey().array(), attribute.getValue().array());
960      }
961    }
962    if (in.isSetConfiguration()) {
963      for (Map.Entry<String, String> conf : in.getConfiguration().entrySet()) {
964        builder.setConfiguration(conf.getKey(), conf.getValue());
965      }
966    }
967    if (in.isSetBlockSize()) {
968      builder.setBlocksize(in.getBlockSize());
969    }
970    if (in.isSetBloomnFilterType()) {
971      builder.setBloomFilterType(bloomFilterFromThrift(in.getBloomnFilterType()));
972    }
973    if (in.isSetCompressionType()) {
974      builder.setCompressionType(compressionAlgorithmFromThrift(in.getCompressionType()));
975    }
976    if (in.isSetDfsReplication()) {
977      builder.setDFSReplication(in.getDfsReplication());
978    }
979    if (in.isSetDataBlockEncoding()) {
980      builder.setDataBlockEncoding(dataBlockEncodingFromThrift(in.getDataBlockEncoding()));
981    }
982    if (in.isSetKeepDeletedCells()) {
983      builder.setKeepDeletedCells(keepDeletedCellsFromThrift(in.getKeepDeletedCells()));
984    }
985    if (in.isSetMaxVersions()) {
986      builder.setMaxVersions(in.getMaxVersions());
987    }
988    if (in.isSetMinVersions()) {
989      builder.setMinVersions(in.getMinVersions());
990    }
991    if (in.isSetScope()) {
992      builder.setScope(in.getScope());
993    }
994    if (in.isSetTimeToLive()) {
995      builder.setTimeToLive(in.getTimeToLive());
996    }
997    if (in.isSetBlockCacheEnabled()) {
998      builder.setBlockCacheEnabled(in.isBlockCacheEnabled());
999    }
1000    if (in.isSetCacheBloomsOnWrite()) {
1001      builder.setCacheBloomsOnWrite(in.isCacheBloomsOnWrite());
1002    }
1003    if (in.isSetCacheDataOnWrite()) {
1004      builder.setCacheDataOnWrite(in.isCacheDataOnWrite());
1005    }
1006    if (in.isSetCacheIndexesOnWrite()) {
1007      builder.setCacheIndexesOnWrite(in.isCacheIndexesOnWrite());
1008    }
1009    if (in.isSetCompressTags()) {
1010      builder.setCompressTags(in.isCompressTags());
1011    }
1012    if (in.isSetEvictBlocksOnClose()) {
1013      builder.setEvictBlocksOnClose(in.isEvictBlocksOnClose());
1014    }
1015    if (in.isSetInMemory()) {
1016      builder.setInMemory(in.isInMemory());
1017    }
1018
1019    return builder.build();
1020  }
1021
1022  public static NamespaceDescriptor namespaceDescriptorFromThrift(TNamespaceDescriptor in) {
1023    NamespaceDescriptor.Builder builder = NamespaceDescriptor.create(in.getName());
1024    if (in.isSetConfiguration()) {
1025      for (Map.Entry<String, String> conf : in.getConfiguration().entrySet()) {
1026        builder.addConfiguration(conf.getKey(), conf.getValue());
1027      }
1028    }
1029    return builder.build();
1030  }
1031
1032  public static TNamespaceDescriptor namespaceDescriptorFromHBase(NamespaceDescriptor in) {
1033    TNamespaceDescriptor out = new TNamespaceDescriptor();
1034    out.setName(in.getName());
1035    for (Map.Entry<String, String> conf : in.getConfiguration().entrySet()) {
1036      out.putToConfiguration(conf.getKey(), conf.getValue());
1037    }
1038    return out;
1039  }
1040
1041  public static List<TNamespaceDescriptor> namespaceDescriptorsFromHBase(NamespaceDescriptor[] in) {
1042    List<TNamespaceDescriptor> out = new ArrayList<>(in.length);
1043    for (NamespaceDescriptor descriptor : in) {
1044      out.add(namespaceDescriptorFromHBase(descriptor));
1045    }
1046    return out;
1047  }
1048
1049  public static TableDescriptor tableDescriptorFromThrift(TTableDescriptor in) {
1050    TableDescriptorBuilder builder =
1051      TableDescriptorBuilder.newBuilder(tableNameFromThrift(in.getTableName()));
1052    for (TColumnFamilyDescriptor column : in.getColumns()) {
1053      builder.setColumnFamily(columnFamilyDescriptorFromThrift(column));
1054    }
1055    if (in.isSetAttributes()) {
1056      for (Map.Entry<ByteBuffer, ByteBuffer> attribute : in.getAttributes().entrySet()) {
1057        builder.setValue(attribute.getKey().array(), attribute.getValue().array());
1058      }
1059    }
1060    if (in.isSetDurability()) {
1061      builder.setDurability(durabilityFromThrift(in.getDurability()));
1062    }
1063    return builder.build();
1064  }
1065
1066  public static List<TableDescriptor> tableDescriptorsFromThrift(List<TTableDescriptor> in) {
1067    List<TableDescriptor> out = new ArrayList<>();
1068    for (TTableDescriptor tableDescriptor : in) {
1069      out.add(tableDescriptorFromThrift(tableDescriptor));
1070    }
1071    return out;
1072  }
1073
1074  private static TDurability durabilityFromHBase(Durability durability) {
1075    switch (durability) {
1076      case USE_DEFAULT:
1077        return TDurability.USE_DEFAULT;
1078      case SKIP_WAL:
1079        return TDurability.SKIP_WAL;
1080      case ASYNC_WAL:
1081        return TDurability.ASYNC_WAL;
1082      case SYNC_WAL:
1083        return TDurability.SYNC_WAL;
1084      case FSYNC_WAL:
1085        return TDurability.FSYNC_WAL;
1086      default:
1087        return null;
1088    }
1089  }
1090
1091  public static TTableDescriptor tableDescriptorFromHBase(TableDescriptor in) {
1092    TTableDescriptor out = new TTableDescriptor();
1093    out.setTableName(tableNameFromHBase(in.getTableName()));
1094    Map<Bytes, Bytes> attributes = in.getValues();
1095    for (Map.Entry<Bytes, Bytes> attribute : attributes.entrySet()) {
1096      out.putToAttributes(ByteBuffer.wrap(attribute.getKey().get()),
1097        ByteBuffer.wrap(attribute.getValue().get()));
1098    }
1099    for (ColumnFamilyDescriptor column : in.getColumnFamilies()) {
1100      out.addToColumns(columnFamilyDescriptorFromHBase(column));
1101    }
1102    out.setDurability(durabilityFromHBase(in.getDurability()));
1103    return out;
1104  }
1105
1106  public static List<TTableDescriptor> tableDescriptorsFromHBase(List<TableDescriptor> in) {
1107    List<TTableDescriptor> out = new ArrayList<>(in.size());
1108    for (TableDescriptor descriptor : in) {
1109      out.add(tableDescriptorFromHBase(descriptor));
1110    }
1111    return out;
1112  }
1113
1114  public static List<TTableDescriptor> tableDescriptorsFromHBase(TableDescriptor[] in) {
1115    List<TTableDescriptor> out = new ArrayList<>(in.length);
1116    for (TableDescriptor descriptor : in) {
1117      out.add(tableDescriptorFromHBase(descriptor));
1118    }
1119    return out;
1120  }
1121
1122  public static TBloomFilterType bloomFilterFromHBase(BloomType in) {
1123    switch (in) {
1124      case NONE:
1125        return TBloomFilterType.NONE;
1126      case ROW:
1127        return TBloomFilterType.ROW;
1128      case ROWCOL:
1129        return TBloomFilterType.ROWCOL;
1130      case ROWPREFIX_FIXED_LENGTH:
1131        return TBloomFilterType.ROWPREFIX_FIXED_LENGTH;
1132      default:
1133        return TBloomFilterType.ROW;
1134    }
1135  }
1136
1137  public static TCompressionAlgorithm compressionAlgorithmFromHBase(Compression.Algorithm in) {
1138    switch (in) {
1139      case LZO:
1140        return TCompressionAlgorithm.LZO;
1141      case GZ:
1142        return TCompressionAlgorithm.GZ;
1143      case NONE:
1144        return TCompressionAlgorithm.NONE;
1145      case SNAPPY:
1146        return TCompressionAlgorithm.SNAPPY;
1147      case LZ4:
1148        return TCompressionAlgorithm.LZ4;
1149      case BZIP2:
1150        return TCompressionAlgorithm.BZIP2;
1151      case ZSTD:
1152        return TCompressionAlgorithm.ZSTD;
1153      default:
1154        return TCompressionAlgorithm.NONE;
1155    }
1156  }
1157
1158  public static TDataBlockEncoding dataBlockEncodingFromHBase(DataBlockEncoding in) {
1159    switch (in) {
1160      case NONE:
1161        return TDataBlockEncoding.NONE;
1162      case PREFIX:
1163        return TDataBlockEncoding.PREFIX;
1164      case DIFF:
1165        return TDataBlockEncoding.DIFF;
1166      case FAST_DIFF:
1167        return TDataBlockEncoding.FAST_DIFF;
1168      case ROW_INDEX_V1:
1169        return TDataBlockEncoding.ROW_INDEX_V1;
1170      default:
1171        return TDataBlockEncoding.NONE;
1172    }
1173  }
1174
1175  public static TKeepDeletedCells keepDeletedCellsFromHBase(KeepDeletedCells in) {
1176    switch (in) {
1177      case FALSE:
1178        return TKeepDeletedCells.FALSE;
1179      case TRUE:
1180        return TKeepDeletedCells.TRUE;
1181      case TTL:
1182        return TKeepDeletedCells.TTL;
1183      default:
1184        return TKeepDeletedCells.FALSE;
1185    }
1186  }
1187
1188  public static TColumnFamilyDescriptor columnFamilyDescriptorFromHBase(ColumnFamilyDescriptor in) {
1189    TColumnFamilyDescriptor out = new TColumnFamilyDescriptor();
1190    out.setName(in.getName());
1191    for (Map.Entry<Bytes, Bytes> attribute : in.getValues().entrySet()) {
1192      out.putToAttributes(ByteBuffer.wrap(attribute.getKey().get()),
1193        ByteBuffer.wrap(attribute.getValue().get()));
1194    }
1195    for (Map.Entry<String, String> conf : in.getConfiguration().entrySet()) {
1196      out.putToConfiguration(conf.getKey(), conf.getValue());
1197    }
1198    out.setBlockSize(in.getBlocksize());
1199    out.setBloomnFilterType(bloomFilterFromHBase(in.getBloomFilterType()));
1200    out.setCompressionType(compressionAlgorithmFromHBase(in.getCompressionType()));
1201    out.setDfsReplication(in.getDFSReplication());
1202    out.setDataBlockEncoding(dataBlockEncodingFromHBase(in.getDataBlockEncoding()));
1203    out.setKeepDeletedCells(keepDeletedCellsFromHBase(in.getKeepDeletedCells()));
1204    out.setMaxVersions(in.getMaxVersions());
1205    out.setMinVersions(in.getMinVersions());
1206    out.setScope(in.getScope());
1207    out.setTimeToLive(in.getTimeToLive());
1208    out.setBlockCacheEnabled(in.isBlockCacheEnabled());
1209    out.setCacheBloomsOnWrite(in.isCacheBloomsOnWrite());
1210    out.setCacheDataOnWrite(in.isCacheDataOnWrite());
1211    out.setCacheIndexesOnWrite(in.isCacheIndexesOnWrite());
1212    out.setCompressTags(in.isCompressTags());
1213    out.setEvictBlocksOnClose(in.isEvictBlocksOnClose());
1214    out.setInMemory(in.isInMemory());
1215    return out;
1216  }
1217
1218  private static TConsistency consistencyFromHBase(Consistency consistency) {
1219    switch (consistency) {
1220      case STRONG:
1221        return TConsistency.STRONG;
1222      case TIMELINE:
1223        return TConsistency.TIMELINE;
1224      default:
1225        return TConsistency.STRONG;
1226    }
1227  }
1228
1229  public static TGet getFromHBase(Get in) {
1230    TGet out = new TGet();
1231    out.setRow(in.getRow());
1232
1233    TTimeRange tTimeRange = new TTimeRange();
1234    tTimeRange.setMaxStamp(in.getTimeRange().getMax()).setMinStamp(in.getTimeRange().getMin());
1235    out.setTimeRange(tTimeRange);
1236    out.setMaxVersions(in.getMaxVersions());
1237
1238    for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
1239      out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
1240        ByteBuffer.wrap(attribute.getValue()));
1241    }
1242    try {
1243      Authorizations authorizations = in.getAuthorizations();
1244      if (authorizations != null) {
1245        TAuthorization tAuthorization = new TAuthorization();
1246        tAuthorization.setLabels(authorizations.getLabels());
1247        out.setAuthorizations(tAuthorization);
1248      }
1249    } catch (DeserializationException e) {
1250      throw new RuntimeException(e);
1251    }
1252    out.setConsistency(consistencyFromHBase(in.getConsistency()));
1253    out.setTargetReplicaId(in.getReplicaId());
1254    out.setCacheBlocks(in.getCacheBlocks());
1255    out.setStoreLimit(in.getMaxResultsPerColumnFamily());
1256    out.setStoreOffset(in.getRowOffsetPerColumnFamily());
1257    out.setExistence_only(in.isCheckExistenceOnly());
1258    for (Map.Entry<byte[], NavigableSet<byte[]>> family : in.getFamilyMap().entrySet()) {
1259
1260      if (family.getValue() != null && !family.getValue().isEmpty()) {
1261        for (byte[] qualifier : family.getValue()) {
1262          TColumn column = new TColumn();
1263          column.setFamily(family.getKey());
1264          column.setQualifier(qualifier);
1265          out.addToColumns(column);
1266        }
1267      } else {
1268        TColumn column = new TColumn();
1269        column.setFamily(family.getKey());
1270        out.addToColumns(column);
1271      }
1272    }
1273    if (in.getFilter() != null) {
1274      try {
1275        out.setFilterBytes(filterFromHBase(in.getFilter()));
1276      } catch (IOException ioE) {
1277        throw new RuntimeException(ioE);
1278      }
1279    }
1280    return out;
1281  }
1282
1283  public static Cell toCell(ExtendedCellBuilder cellBuilder, byte[] row, TColumnValue columnValue) {
1284    return cellBuilder.clear().setRow(row).setFamily(columnValue.getFamily())
1285      .setQualifier(columnValue.getQualifier()).setTimestamp(columnValue.getTimestamp())
1286      .setType(columnValue.getType()).setValue(columnValue.getValue())
1287      .setTags(columnValue.getTags()).build();
1288  }
1289
1290  public static Result resultFromThrift(TResult in) {
1291    if (in == null) {
1292      return null;
1293    }
1294    if (!in.isSetColumnValues() || in.getColumnValues().isEmpty()) {
1295      return in.isStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT;
1296    }
1297    List<Cell> cells = new ArrayList<>(in.getColumnValues().size());
1298    ExtendedCellBuilder builder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
1299    for (TColumnValue columnValue : in.getColumnValues()) {
1300      cells.add(toCell(builder, in.getRow(), columnValue));
1301    }
1302    return Result.create(cells, null, in.isStale(), in.isPartial());
1303  }
1304
1305  public static TPut putFromHBase(Put in) {
1306    TPut out = new TPut();
1307    out.setRow(in.getRow());
1308    if (in.getTimestamp() != HConstants.LATEST_TIMESTAMP) {
1309      out.setTimestamp(in.getTimestamp());
1310    }
1311    if (in.getDurability() != Durability.USE_DEFAULT) {
1312      out.setDurability(durabilityFromHBase(in.getDurability()));
1313    }
1314    for (Map.Entry<byte[], List<ExtendedCell>> entry : ClientInternalHelper
1315      .getExtendedFamilyCellMap(in).entrySet()) {
1316      byte[] family = entry.getKey();
1317      for (ExtendedCell cell : entry.getValue()) {
1318        TColumnValue columnValue = new TColumnValue();
1319        columnValue.setFamily(family).setQualifier(CellUtil.cloneQualifier(cell))
1320          .setType(cell.getType().getCode()).setTimestamp(cell.getTimestamp())
1321          .setValue(CellUtil.cloneValue(cell));
1322        if (cell.getTagsLength() != 0) {
1323          columnValue.setTags(PrivateCellUtil.cloneTags(cell));
1324        }
1325        out.addToColumnValues(columnValue);
1326      }
1327    }
1328    for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
1329      out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
1330        ByteBuffer.wrap(attribute.getValue()));
1331    }
1332    try {
1333      CellVisibility cellVisibility = in.getCellVisibility();
1334      if (cellVisibility != null) {
1335        TCellVisibility tCellVisibility = new TCellVisibility();
1336        tCellVisibility.setExpression(cellVisibility.getExpression());
1337        out.setCellVisibility(tCellVisibility);
1338      }
1339    } catch (DeserializationException e) {
1340      throw new RuntimeException(e);
1341    }
1342    return out;
1343  }
1344
1345  public static List<TPut> putsFromHBase(List<Put> in) {
1346    List<TPut> out = new ArrayList<>(in.size());
1347    for (Put put : in) {
1348      out.add(putFromHBase(put));
1349    }
1350    return out;
1351  }
1352
1353  public static NamespaceDescriptor[]
1354    namespaceDescriptorsFromThrift(List<TNamespaceDescriptor> in) {
1355    NamespaceDescriptor[] out = new NamespaceDescriptor[in.size()];
1356    int index = 0;
1357    for (TNamespaceDescriptor descriptor : in) {
1358      out[index++] = namespaceDescriptorFromThrift(descriptor);
1359    }
1360    return out;
1361  }
1362
1363  public static List<TDelete> deletesFromHBase(List<Delete> in) {
1364    List<TDelete> out = new ArrayList<>(in.size());
1365    for (Delete delete : in) {
1366      out.add(deleteFromHBase(delete));
1367    }
1368    return out;
1369  }
1370
1371  public static TAppend appendFromHBase(Append in) throws IOException {
1372    TAppend out = new TAppend();
1373    out.setRow(in.getRow());
1374
1375    if (in.getDurability() != Durability.USE_DEFAULT) {
1376      out.setDurability(durabilityFromHBase(in.getDurability()));
1377    }
1378    for (Map.Entry<byte[], List<ExtendedCell>> entry : ClientInternalHelper
1379      .getExtendedFamilyCellMap(in).entrySet()) {
1380      byte[] family = entry.getKey();
1381      for (ExtendedCell cell : entry.getValue()) {
1382        TColumnValue columnValue = new TColumnValue();
1383        columnValue.setFamily(family).setQualifier(CellUtil.cloneQualifier(cell))
1384          .setType(cell.getType().getCode()).setTimestamp(cell.getTimestamp())
1385          .setValue(CellUtil.cloneValue(cell));
1386        if (cell.getTagsLength() != 0) {
1387          columnValue.setTags(PrivateCellUtil.cloneTags(cell));
1388        }
1389        out.addToColumns(columnValue);
1390      }
1391    }
1392    for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
1393      out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
1394        ByteBuffer.wrap(attribute.getValue()));
1395    }
1396    try {
1397      CellVisibility cellVisibility = in.getCellVisibility();
1398      if (cellVisibility != null) {
1399        TCellVisibility tCellVisibility = new TCellVisibility();
1400        tCellVisibility.setExpression(cellVisibility.getExpression());
1401        out.setCellVisibility(tCellVisibility);
1402      }
1403    } catch (DeserializationException e) {
1404      throw new RuntimeException(e);
1405    }
1406    out.setReturnResults(in.isReturnResults());
1407    return out;
1408  }
1409
1410  public static TIncrement incrementFromHBase(Increment in) throws IOException {
1411    TIncrement out = new TIncrement();
1412    out.setRow(in.getRow());
1413
1414    if (in.getDurability() != Durability.USE_DEFAULT) {
1415      out.setDurability(durabilityFromHBase(in.getDurability()));
1416    }
1417    for (Map.Entry<byte[], List<Cell>> entry : in.getFamilyCellMap().entrySet()) {
1418      byte[] family = entry.getKey();
1419      for (Cell cell : entry.getValue()) {
1420        TColumnIncrement columnValue = new TColumnIncrement();
1421        columnValue.setFamily(family).setQualifier(CellUtil.cloneQualifier(cell));
1422        columnValue.setAmount(
1423          Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
1424        out.addToColumns(columnValue);
1425      }
1426    }
1427    for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
1428      out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
1429        ByteBuffer.wrap(attribute.getValue()));
1430    }
1431    try {
1432      CellVisibility cellVisibility = in.getCellVisibility();
1433      if (cellVisibility != null) {
1434        TCellVisibility tCellVisibility = new TCellVisibility();
1435        tCellVisibility.setExpression(cellVisibility.getExpression());
1436        out.setCellVisibility(tCellVisibility);
1437      }
1438    } catch (DeserializationException e) {
1439      throw new RuntimeException(e);
1440    }
1441    out.setReturnResults(in.isReturnResults());
1442    return out;
1443  }
1444
1445  public static TRowMutations rowMutationsFromHBase(RowMutations in) {
1446    TRowMutations tRowMutations = new TRowMutations();
1447    tRowMutations.setRow(in.getRow());
1448    for (Mutation mutation : in.getMutations()) {
1449      TMutation tMutation = new TMutation();
1450      if (mutation instanceof Put) {
1451        tMutation.setPut(ThriftUtilities.putFromHBase((Put) mutation));
1452      } else if (mutation instanceof Delete) {
1453        tMutation.setDeleteSingle(ThriftUtilities.deleteFromHBase((Delete) mutation));
1454      } else {
1455        throw new IllegalArgumentException(
1456          "Only Put and Delete is supported in mutateRow, but muation=" + mutation);
1457      }
1458      tRowMutations.addToMutations(tMutation);
1459    }
1460    return tRowMutations;
1461  }
1462
1463  public static TCompareOperator compareOpFromHBase(CompareOperator compareOp) {
1464    switch (compareOp) {
1465      case LESS:
1466        return TCompareOperator.LESS;
1467      case LESS_OR_EQUAL:
1468        return TCompareOperator.LESS_OR_EQUAL;
1469      case EQUAL:
1470        return TCompareOperator.EQUAL;
1471      case NOT_EQUAL:
1472        return TCompareOperator.NOT_EQUAL;
1473      case GREATER_OR_EQUAL:
1474        return TCompareOperator.GREATER_OR_EQUAL;
1475      case GREATER:
1476        return TCompareOperator.GREATER;
1477      case NO_OP:
1478        return TCompareOperator.NO_OP;
1479      default:
1480        return null;
1481    }
1482  }
1483
1484  public static List<ByteBuffer> splitKeyFromHBase(byte[][] in) {
1485    if (in == null || in.length == 0) {
1486      return null;
1487    }
1488    List<ByteBuffer> out = new ArrayList<>(in.length);
1489    for (byte[] key : in) {
1490      out.add(ByteBuffer.wrap(key));
1491    }
1492    return out;
1493  }
1494
1495  public static Result[] resultsFromThrift(List<TResult> in) {
1496    Result[] out = new Result[in.size()];
1497    int index = 0;
1498    for (TResult tResult : in) {
1499      out[index++] = resultFromThrift(tResult);
1500    }
1501    return out;
1502  }
1503
1504  public static List<TGet> getsFromHBase(List<Get> in) {
1505    List<TGet> out = new ArrayList<>(in.size());
1506    for (Get get : in) {
1507      out.add(getFromHBase(get));
1508    }
1509    return out;
1510  }
1511
1512  public static Set<TServerName> getServerNamesFromHBase(Set<ServerName> serverNames) {
1513    if (CollectionUtils.isEmpty(serverNames)) {
1514      return Collections.emptySet();
1515    }
1516    return serverNames.stream().map(serverName -> {
1517      TServerName tServerName = new TServerName();
1518      tServerName.setHostName(serverName.getHostname());
1519      tServerName.setPort(serverName.getPort());
1520      tServerName.setStartCode(serverName.getStartcode());
1521      return tServerName;
1522    }).collect(Collectors.toSet());
1523  }
1524
1525  public static Set<ServerName> getServerNamesFromThrift(Set<TServerName> tServerNames) {
1526    if (CollectionUtils.isEmpty(tServerNames)) {
1527      return Collections.emptySet();
1528    }
1529    return tServerNames.stream().map(tServerName -> ServerName.valueOf(tServerName.getHostName(),
1530      tServerName.getPort(), tServerName.getStartCode())).collect(Collectors.toSet());
1531  }
1532
1533  public static TLogQueryFilter getSlowLogQueryFromHBase(LogQueryFilter logQueryFilter) {
1534    TLogQueryFilter tLogQueryFilter = new TLogQueryFilter();
1535    tLogQueryFilter.setRegionName(logQueryFilter.getRegionName());
1536    tLogQueryFilter.setClientAddress(logQueryFilter.getClientAddress());
1537    tLogQueryFilter.setTableName(logQueryFilter.getTableName());
1538    tLogQueryFilter.setUserName(logQueryFilter.getUserName());
1539    tLogQueryFilter.setLimit(logQueryFilter.getLimit());
1540    TLogType tLogType = gettLogTypeFromHBase(logQueryFilter);
1541    tLogQueryFilter.setLogType(tLogType);
1542    TFilterByOperator tFilterByOperator = getTFilterByFromHBase(logQueryFilter);
1543    tLogQueryFilter.setFilterByOperator(tFilterByOperator);
1544    return tLogQueryFilter;
1545  }
1546
1547  private static TLogType gettLogTypeFromHBase(final LogQueryFilter logQueryFilter) {
1548    TLogType tLogType;
1549    switch (logQueryFilter.getType()) {
1550      case SLOW_LOG: {
1551        tLogType = TLogType.SLOW_LOG;
1552        break;
1553      }
1554      case LARGE_LOG: {
1555        tLogType = TLogType.LARGE_LOG;
1556        break;
1557      }
1558      default: {
1559        tLogType = TLogType.SLOW_LOG;
1560      }
1561    }
1562    return tLogType;
1563  }
1564
1565  private static TFilterByOperator getTFilterByFromHBase(final LogQueryFilter logQueryFilter) {
1566    TFilterByOperator tFilterByOperator;
1567    switch (logQueryFilter.getFilterByOperator()) {
1568      case AND: {
1569        tFilterByOperator = TFilterByOperator.AND;
1570        break;
1571      }
1572      case OR: {
1573        tFilterByOperator = TFilterByOperator.OR;
1574        break;
1575      }
1576      default: {
1577        tFilterByOperator = TFilterByOperator.OR;
1578      }
1579    }
1580    return tFilterByOperator;
1581  }
1582
1583  public static LogQueryFilter getSlowLogQueryFromThrift(TLogQueryFilter tLogQueryFilter) {
1584    LogQueryFilter logQueryFilter = new LogQueryFilter();
1585    logQueryFilter.setRegionName(tLogQueryFilter.getRegionName());
1586    logQueryFilter.setClientAddress(tLogQueryFilter.getClientAddress());
1587    logQueryFilter.setTableName(tLogQueryFilter.getTableName());
1588    logQueryFilter.setUserName(tLogQueryFilter.getUserName());
1589    logQueryFilter.setLimit(tLogQueryFilter.getLimit());
1590    LogQueryFilter.Type type = getLogTypeFromThrift(tLogQueryFilter);
1591    logQueryFilter.setType(type);
1592    LogQueryFilter.FilterByOperator filterByOperator = getFilterByFromThrift(tLogQueryFilter);
1593    logQueryFilter.setFilterByOperator(filterByOperator);
1594    return logQueryFilter;
1595  }
1596
1597  private static LogQueryFilter.Type
1598    getLogTypeFromThrift(final TLogQueryFilter tSlowLogQueryFilter) {
1599    LogQueryFilter.Type type;
1600    switch (tSlowLogQueryFilter.getLogType()) {
1601      case SLOW_LOG: {
1602        type = LogQueryFilter.Type.SLOW_LOG;
1603        break;
1604      }
1605      case LARGE_LOG: {
1606        type = LogQueryFilter.Type.LARGE_LOG;
1607        break;
1608      }
1609      default: {
1610        type = LogQueryFilter.Type.SLOW_LOG;
1611      }
1612    }
1613    return type;
1614  }
1615
1616  private static LogQueryFilter.FilterByOperator
1617    getFilterByFromThrift(final TLogQueryFilter tLogQueryFilter) {
1618    LogQueryFilter.FilterByOperator filterByOperator;
1619    switch (tLogQueryFilter.getFilterByOperator()) {
1620      case AND: {
1621        filterByOperator = LogQueryFilter.FilterByOperator.AND;
1622        break;
1623      }
1624      case OR: {
1625        filterByOperator = LogQueryFilter.FilterByOperator.OR;
1626        break;
1627      }
1628      default: {
1629        filterByOperator = LogQueryFilter.FilterByOperator.OR;
1630      }
1631    }
1632    return filterByOperator;
1633  }
1634
1635  public static List<TOnlineLogRecord>
1636    getSlowLogRecordsFromHBase(List<OnlineLogRecord> onlineLogRecords) {
1637    if (CollectionUtils.isEmpty(onlineLogRecords)) {
1638      return Collections.emptyList();
1639    }
1640    return onlineLogRecords.stream().map(slowLogRecord -> {
1641      TOnlineLogRecord tOnlineLogRecord = new TOnlineLogRecord();
1642      tOnlineLogRecord.setCallDetails(slowLogRecord.getCallDetails());
1643      tOnlineLogRecord.setClientAddress(slowLogRecord.getClientAddress());
1644      tOnlineLogRecord.setMethodName(slowLogRecord.getMethodName());
1645      tOnlineLogRecord.setMultiGetsCount(slowLogRecord.getMultiGetsCount());
1646      tOnlineLogRecord.setMultiMutationsCount(slowLogRecord.getMultiMutationsCount());
1647      tOnlineLogRecord.setMultiServiceCalls(slowLogRecord.getMultiServiceCalls());
1648      tOnlineLogRecord.setParam(slowLogRecord.getParam());
1649      tOnlineLogRecord.setProcessingTime(slowLogRecord.getProcessingTime());
1650      tOnlineLogRecord.setQueueTime(slowLogRecord.getQueueTime());
1651      tOnlineLogRecord.setRegionName(slowLogRecord.getRegionName());
1652      tOnlineLogRecord.setResponseSize(slowLogRecord.getResponseSize());
1653      tOnlineLogRecord.setBlockBytesScanned(slowLogRecord.getBlockBytesScanned());
1654      tOnlineLogRecord.setFsReadTime(slowLogRecord.getFsReadTime());
1655      tOnlineLogRecord.setServerClass(slowLogRecord.getServerClass());
1656      tOnlineLogRecord.setStartTime(slowLogRecord.getStartTime());
1657      tOnlineLogRecord.setUserName(slowLogRecord.getUserName());
1658      return tOnlineLogRecord;
1659    }).collect(Collectors.toList());
1660  }
1661
1662  public static List<OnlineLogRecord>
1663    getSlowLogRecordsFromThrift(List<TOnlineLogRecord> tOnlineLogRecords) {
1664    if (CollectionUtils.isEmpty(tOnlineLogRecords)) {
1665      return Collections.emptyList();
1666    }
1667    return tOnlineLogRecords.stream()
1668      .map(tSlowLogRecord -> new OnlineLogRecord.OnlineLogRecordBuilder()
1669        .setCallDetails(tSlowLogRecord.getCallDetails())
1670        .setClientAddress(tSlowLogRecord.getClientAddress())
1671        .setMethodName(tSlowLogRecord.getMethodName())
1672        .setMultiGetsCount(tSlowLogRecord.getMultiGetsCount())
1673        .setMultiMutationsCount(tSlowLogRecord.getMultiMutationsCount())
1674        .setMultiServiceCalls(tSlowLogRecord.getMultiServiceCalls())
1675        .setParam(tSlowLogRecord.getParam()).setProcessingTime(tSlowLogRecord.getProcessingTime())
1676        .setQueueTime(tSlowLogRecord.getQueueTime()).setRegionName(tSlowLogRecord.getRegionName())
1677        .setResponseSize(tSlowLogRecord.getResponseSize())
1678        .setBlockBytesScanned(tSlowLogRecord.getBlockBytesScanned())
1679        .setServerClass(tSlowLogRecord.getServerClass()).setStartTime(tSlowLogRecord.getStartTime())
1680        .setUserName(tSlowLogRecord.getUserName()).build())
1681      .collect(Collectors.toList());
1682  }
1683
1684  public static Permission.Action[] permissionActionsFromString(String permission_actions) {
1685    Set<Permission.Action> actions = new HashSet<>();
1686    for (char c : permission_actions.toCharArray()) {
1687      switch (c) {
1688        case 'R':
1689          actions.add(Permission.Action.READ);
1690          break;
1691        case 'W':
1692          actions.add(Permission.Action.WRITE);
1693          break;
1694        case 'C':
1695          actions.add(Permission.Action.CREATE);
1696          break;
1697        case 'X':
1698          actions.add(Permission.Action.EXEC);
1699          break;
1700        case 'A':
1701          actions.add(Permission.Action.ADMIN);
1702          break;
1703        default:
1704          break;
1705      }
1706    }
1707    return actions.toArray(new Permission.Action[0]);
1708  }
1709
1710}