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