View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.thrift;
20  
21  import static org.apache.hadoop.hbase.util.Bytes.getBytes;
22  
23  import java.nio.ByteBuffer;
24  import java.util.ArrayList;
25  import java.util.List;
26  import java.util.TreeMap;
27  
28  import org.apache.hadoop.hbase.classification.InterfaceAudience;
29  import org.apache.hadoop.hbase.Cell;
30  import org.apache.hadoop.hbase.CellUtil;
31  import org.apache.hadoop.hbase.HColumnDescriptor;
32  import org.apache.hadoop.hbase.KeyValue;
33  import org.apache.hadoop.hbase.client.Append;
34  import org.apache.hadoop.hbase.client.Increment;
35  import org.apache.hadoop.hbase.client.Result;
36  import org.apache.hadoop.hbase.io.compress.Compression;
37  import org.apache.hadoop.hbase.regionserver.BloomType;
38  import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
39  import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
40  import org.apache.hadoop.hbase.thrift.generated.TAppend;
41  import org.apache.hadoop.hbase.thrift.generated.TCell;
42  import org.apache.hadoop.hbase.thrift.generated.TColumn;
43  import org.apache.hadoop.hbase.thrift.generated.TIncrement;
44  import org.apache.hadoop.hbase.thrift.generated.TRowResult;
45  import org.apache.hadoop.hbase.util.Bytes;
46  
47  @InterfaceAudience.Private
48  public class ThriftUtilities {
49  
50    /**
51     * This utility method creates a new Hbase HColumnDescriptor object based on a
52     * Thrift ColumnDescriptor "struct".
53     *
54     * @param in
55     *          Thrift ColumnDescriptor object
56     * @return HColumnDescriptor
57     * @throws IllegalArgument
58     */
59    static public HColumnDescriptor colDescFromThrift(ColumnDescriptor in)
60        throws IllegalArgument {
61      Compression.Algorithm comp =
62        Compression.getCompressionAlgorithmByName(in.compression.toLowerCase());
63      BloomType bt =
64        BloomType.valueOf(in.bloomFilterType);
65  
66      if (in.name == null || !in.name.hasRemaining()) {
67        throw new IllegalArgument("column name is empty");
68      }
69      byte [] parsedName = KeyValue.parseColumn(Bytes.getBytes(in.name))[0];
70      HColumnDescriptor col = new HColumnDescriptor(parsedName)
71          .setMaxVersions(in.maxVersions)
72          .setCompressionType(comp)
73          .setInMemory(in.inMemory)
74          .setBlockCacheEnabled(in.blockCacheEnabled)
75          .setTimeToLive(in.timeToLive > 0 ? in.timeToLive : Integer.MAX_VALUE)
76          .setBloomFilterType(bt);
77      return col;
78    }
79  
80    /**
81     * This utility method creates a new Thrift ColumnDescriptor "struct" based on
82     * an Hbase HColumnDescriptor object.
83     *
84     * @param in
85     *          Hbase HColumnDescriptor object
86     * @return Thrift ColumnDescriptor
87     */
88    static public ColumnDescriptor colDescFromHbase(HColumnDescriptor in) {
89      ColumnDescriptor col = new ColumnDescriptor();
90      col.name = ByteBuffer.wrap(Bytes.add(in.getName(), KeyValue.COLUMN_FAMILY_DELIM_ARRAY));
91      col.maxVersions = in.getMaxVersions();
92      col.compression = in.getCompression().toString();
93      col.inMemory = in.isInMemory();
94      col.blockCacheEnabled = in.isBlockCacheEnabled();
95      col.bloomFilterType = in.getBloomFilterType().toString();
96      return col;
97    }
98  
99    /**
100    * This utility method creates a list of Thrift TCell "struct" based on
101    * an Hbase Cell object. The empty list is returned if the input is null.
102    *
103    * @param in
104    *          Hbase Cell object
105    * @return Thrift TCell array
106    */
107   static public List<TCell> cellFromHBase(Cell in) {
108     List<TCell> list = new ArrayList<TCell>(1);
109     if (in != null) {
110       list.add(new TCell(ByteBuffer.wrap(CellUtil.cloneValue(in)), in.getTimestamp()));
111     }
112     return list;
113   }
114 
115   /**
116    * This utility method creates a list of Thrift TCell "struct" based on
117    * an Hbase Cell array. The empty list is returned if the input is null.
118    * @param in Hbase Cell array
119    * @return Thrift TCell array
120    */
121   static public List<TCell> cellFromHBase(Cell[] in) {
122     List<TCell> list = null;
123     if (in != null) {
124       list = new ArrayList<TCell>(in.length);
125       for (int i = 0; i < in.length; i++) {
126         list.add(new TCell(ByteBuffer.wrap(CellUtil.cloneValue(in[i])), in[i].getTimestamp()));
127       }
128     } else {
129       list = new ArrayList<TCell>(0);
130     }
131     return list;
132   }
133 
134   /**
135    * This utility method creates a list of Thrift TRowResult "struct" based on
136    * an Hbase RowResult object. The empty list is returned if the input is
137    * null.
138    *
139    * @param in
140    *          Hbase RowResult object
141    * @param sortColumns
142    *          This boolean dictates if row data is returned in a sorted order
143    *          sortColumns = True will set TRowResult's sortedColumns member
144    *                        which is an ArrayList of TColumn struct
145    *          sortColumns = False will set TRowResult's columns member which is
146    *                        a map of columnName and TCell struct
147    * @return Thrift TRowResult array
148    */
149   static public List<TRowResult> rowResultFromHBase(Result[] in, boolean sortColumns) {
150     List<TRowResult> results = new ArrayList<TRowResult>();
151     for ( Result result_ : in) {
152         if(result_ == null || result_.isEmpty()) {
153             continue;
154         }
155         TRowResult result = new TRowResult();
156         result.row = ByteBuffer.wrap(result_.getRow());
157         if (sortColumns) {
158           result.sortedColumns = new ArrayList<TColumn>();
159           for (Cell kv : result_.rawCells()) {
160             result.sortedColumns.add(new TColumn(
161                 ByteBuffer.wrap(KeyValue.makeColumn(CellUtil.cloneFamily(kv),
162                     CellUtil.cloneQualifier(kv))),
163                 new TCell(ByteBuffer.wrap(CellUtil.cloneValue(kv)), kv.getTimestamp())));
164           }
165         } else {
166           result.columns = new TreeMap<ByteBuffer, TCell>();
167           for (Cell kv : result_.rawCells()) {
168             result.columns.put(
169                 ByteBuffer.wrap(KeyValue.makeColumn(CellUtil.cloneFamily(kv),
170                     CellUtil.cloneQualifier(kv))),
171                 new TCell(ByteBuffer.wrap(CellUtil.cloneValue(kv)), kv.getTimestamp()));
172           }
173         }
174       results.add(result);
175     }
176     return results;
177   }
178 
179   /**
180    * This utility method creates a list of Thrift TRowResult "struct" based on
181    * an array of Hbase RowResult objects. The empty list is returned if the input is
182    * null.
183    *
184    * @param in
185    *          Array of Hbase RowResult objects
186    * @return Thrift TRowResult array
187    */
188   static public List<TRowResult> rowResultFromHBase(Result[] in) {
189     return rowResultFromHBase(in, false);
190   }
191 
192   static public List<TRowResult> rowResultFromHBase(Result in) {
193     Result [] result = { in };
194     return rowResultFromHBase(result);
195   }
196 
197   /**
198    * From a {@link TIncrement} create an {@link Increment}.
199    * @param tincrement the Thrift version of an increment
200    * @return an increment that the {@link TIncrement} represented.
201    */
202   public static Increment incrementFromThrift(TIncrement tincrement) {
203     Increment inc = new Increment(tincrement.getRow());
204     byte[][] famAndQf = KeyValue.parseColumn(tincrement.getColumn());
205     if (famAndQf.length != 2) return null;
206     inc.addColumn(famAndQf[0], famAndQf[1], tincrement.getAmmount());
207     return inc;
208   }
209 
210   /**
211    * From a {@link TAppend} create an {@link Append}.
212    * @param tappend the Thrift version of an append.
213    * @return an increment that the {@link TAppend} represented.
214    */
215   public static Append appendFromThrift(TAppend tappend) {
216     Append append = new Append(tappend.getRow());
217     List<ByteBuffer> columns = tappend.getColumns();
218     List<ByteBuffer> values = tappend.getValues();
219 
220     if (columns.size() != values.size()) {
221       throw new IllegalArgumentException(
222           "Sizes of columns and values in tappend object are not matching");
223     }
224 
225     int length = columns.size();
226 
227     for (int i = 0; i < length; i++) {
228       byte[][] famAndQf = KeyValue.parseColumn(getBytes(columns.get(i)));
229       append.add(famAndQf[0], famAndQf[1], getBytes(values.get(i)));
230     }
231     return append;
232   }
233 }