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