View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.client;
21  
22  import org.apache.hadoop.conf.Configuration;
23  import org.apache.hadoop.hbase.KeyValue;
24  import org.apache.hadoop.hbase.filter.Filter;
25  import org.apache.hadoop.hbase.io.TimeRange;
26  import org.apache.hadoop.hbase.util.Bytes;
27  import org.apache.hadoop.io.Writable;
28  import org.apache.hadoop.io.WritableFactories;
29  
30  import java.io.DataInput;
31  import java.io.DataOutput;
32  import java.io.IOException;
33  import java.util.ArrayList;
34  import java.util.HashMap;
35  import java.util.List;
36  import java.util.Map;
37  import java.util.NavigableSet;
38  import java.util.Set;
39  import java.util.TreeMap;
40  import java.util.TreeSet;
41  
42  /**
43   * Used to perform Get operations on a single row.
44   * <p>
45   * To get everything for a row, instantiate a Get object with the row to get.
46   * To further define the scope of what to get, perform additional methods as
47   * outlined below.
48   * <p>
49   * To get all columns from specific families, execute {@link #addFamily(byte[]) addFamily}
50   * for each family to retrieve.
51   * <p>
52   * To get specific columns, execute {@link #addColumn(byte[], byte[]) addColumn}
53   * for each column to retrieve.
54   * <p>
55   * To only retrieve columns within a specific range of version timestamps,
56   * execute {@link #setTimeRange(long, long) setTimeRange}.
57   * <p>
58   * To only retrieve columns with a specific timestamp, execute
59   * {@link #setTimeStamp(long) setTimestamp}.
60   * <p>
61   * To limit the number of versions of each column to be returned, execute
62   * {@link #setMaxVersions(int) setMaxVersions}.
63   * <p>
64   * To add a filter, execute {@link #setFilter(Filter) setFilter}.
65   */
66  public class Get extends OperationWithAttributes
67    implements Writable, Row, Comparable<Row> {
68    private static final byte GET_VERSION = (byte)2;
69  
70    private byte [] row = null;
71    private long lockId = -1L;
72    private int maxVersions = 1;
73    private boolean cacheBlocks = true;
74    private Filter filter = null;
75    private TimeRange tr = new TimeRange();
76    private Map<byte [], NavigableSet<byte []>> familyMap =
77      new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
78  
79    /** Constructor for Writable.  DO NOT USE */
80    public Get() {}
81  
82    /**
83     * Create a Get operation for the specified row.
84     * <p>
85     * If no further operations are done, this will get the latest version of
86     * all columns in all families of the specified row.
87     * @param row row key
88     */
89    public Get(byte [] row) {
90      this(row, null);
91    }
92  
93    /**
94     * Create a Get operation for the specified row, using an existing row lock.
95     * <p>
96     * If no further operations are done, this will get the latest version of
97     * all columns in all families of the specified row.
98     * @param row row key
99     * @param rowLock previously acquired row lock, or null
100    */
101   public Get(byte [] row, RowLock rowLock) {
102     this.row = row;
103     if(rowLock != null) {
104       this.lockId = rowLock.getLockId();
105     }
106   }
107 
108   /**
109    * Get all columns from the specified family.
110    * <p>
111    * Overrides previous calls to addColumn for this family.
112    * @param family family name
113    * @return the Get object
114    */
115   public Get addFamily(byte [] family) {
116     familyMap.remove(family);
117     familyMap.put(family, null);
118     return this;
119   }
120 
121   /**
122    * Get the column from the specific family with the specified qualifier.
123    * <p>
124    * Overrides previous calls to addFamily for this family.
125    * @param family family name
126    * @param qualifier column qualifier
127    * @return the Get objec
128    */
129   public Get addColumn(byte [] family, byte [] qualifier) {
130     NavigableSet<byte []> set = familyMap.get(family);
131     if(set == null) {
132       set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
133     }
134     set.add(qualifier);
135     familyMap.put(family, set);
136     return this;
137   }
138 
139   /**
140    * Get versions of columns only within the specified timestamp range,
141    * [minStamp, maxStamp).
142    * @param minStamp minimum timestamp value, inclusive
143    * @param maxStamp maximum timestamp value, exclusive
144    * @throws IOException if invalid time range
145    * @return this for invocation chaining
146    */
147   public Get setTimeRange(long minStamp, long maxStamp)
148   throws IOException {
149     tr = new TimeRange(minStamp, maxStamp);
150     return this;
151   }
152 
153   /**
154    * Get versions of columns with the specified timestamp.
155    * @param timestamp version timestamp
156    * @return this for invocation chaining
157    */
158   public Get setTimeStamp(long timestamp) {
159     try {
160       tr = new TimeRange(timestamp, timestamp+1);
161     } catch(IOException e) {
162       // Will never happen
163     }
164     return this;
165   }
166 
167   /**
168    * Get all available versions.
169    * @return this for invocation chaining
170    */
171   public Get setMaxVersions() {
172     this.maxVersions = Integer.MAX_VALUE;
173     return this;
174   }
175 
176   /**
177    * Get up to the specified number of versions of each column.
178    * @param maxVersions maximum versions for each column
179    * @throws IOException if invalid number of versions
180    * @return this for invocation chaining
181    */
182   public Get setMaxVersions(int maxVersions) throws IOException {
183     if(maxVersions <= 0) {
184       throw new IOException("maxVersions must be positive");
185     }
186     this.maxVersions = maxVersions;
187     return this;
188   }
189 
190   /**
191    * Apply the specified server-side filter when performing the Get.
192    * Only {@link Filter#filterKeyValue(KeyValue)} is called AFTER all tests
193    * for ttl, column match, deletes and max versions have been run.
194    * @param filter filter to run on the server
195    * @return this for invocation chaining
196    */
197   public Get setFilter(Filter filter) {
198     this.filter = filter;
199     return this;
200   }
201 
202   /* Accessors */
203 
204   /**
205    * @return Filter
206    */
207   public Filter getFilter() {
208     return this.filter;
209   }
210 
211   /**
212    * Set whether blocks should be cached for this Get.
213    * <p>
214    * This is true by default.  When true, default settings of the table and
215    * family are used (this will never override caching blocks if the block
216    * cache is disabled for that family or entirely).
217    *
218    * @param cacheBlocks if false, default settings are overridden and blocks
219    * will not be cached
220    */
221   public void setCacheBlocks(boolean cacheBlocks) {
222     this.cacheBlocks = cacheBlocks;
223   }
224 
225   /**
226    * Get whether blocks should be cached for this Get.
227    * @return true if default caching should be used, false if blocks should not
228    * be cached
229    */
230   public boolean getCacheBlocks() {
231     return cacheBlocks;
232   }
233 
234   /**
235    * Method for retrieving the get's row
236    * @return row
237    */
238   public byte [] getRow() {
239     return this.row;
240   }
241 
242   /**
243    * Method for retrieving the get's RowLock
244    * @return RowLock
245    */
246   public RowLock getRowLock() {
247     return new RowLock(this.row, this.lockId);
248   }
249 
250   /**
251    * Method for retrieving the get's lockId
252    * @return lockId
253    */
254   public long getLockId() {
255     return this.lockId;
256   }
257 
258   /**
259    * Method for retrieving the get's maximum number of version
260    * @return the maximum number of version to fetch for this get
261    */
262   public int getMaxVersions() {
263     return this.maxVersions;
264   }
265 
266   /**
267    * Method for retrieving the get's TimeRange
268    * @return timeRange
269    */
270   public TimeRange getTimeRange() {
271     return this.tr;
272   }
273 
274   /**
275    * Method for retrieving the keys in the familyMap
276    * @return keys in the current familyMap
277    */
278   public Set<byte[]> familySet() {
279     return this.familyMap.keySet();
280   }
281 
282   /**
283    * Method for retrieving the number of families to get from
284    * @return number of families
285    */
286   public int numFamilies() {
287     return this.familyMap.size();
288   }
289 
290   /**
291    * Method for checking if any families have been inserted into this Get
292    * @return true if familyMap is non empty false otherwise
293    */
294   public boolean hasFamilies() {
295     return !this.familyMap.isEmpty();
296   }
297 
298   /**
299    * Method for retrieving the get's familyMap
300    * @return familyMap
301    */
302   public Map<byte[],NavigableSet<byte[]>> getFamilyMap() {
303     return this.familyMap;
304   }
305 
306   /**
307    * Compile the table and column family (i.e. schema) information
308    * into a String. Useful for parsing and aggregation by debugging,
309    * logging, and administration tools.
310    * @return Map
311    */
312   @Override
313   public Map<String, Object> getFingerprint() {
314     Map<String, Object> map = new HashMap<String, Object>();
315     List<String> families = new ArrayList<String>();
316     map.put("families", families);
317     for (Map.Entry<byte [], NavigableSet<byte[]>> entry :
318       this.familyMap.entrySet()) {
319       families.add(Bytes.toStringBinary(entry.getKey()));
320     }
321     return map;
322   }
323 
324   /**
325    * Compile the details beyond the scope of getFingerprint (row, columns,
326    * timestamps, etc.) into a Map along with the fingerprinted information.
327    * Useful for debugging, logging, and administration tools.
328    * @param maxCols a limit on the number of columns output prior to truncation
329    * @return Map
330    */
331   @Override
332   public Map<String, Object> toMap(int maxCols) {
333     // we start with the fingerprint map and build on top of it.
334     Map<String, Object> map = getFingerprint();
335     // replace the fingerprint's simple list of families with a 
336     // map from column families to lists of qualifiers and kv details
337     Map<String, List<String>> columns = new HashMap<String, List<String>>();
338     map.put("families", columns);
339     // add scalar information first
340     map.put("row", Bytes.toStringBinary(this.row));
341     map.put("maxVersions", this.maxVersions);
342     map.put("cacheBlocks", this.cacheBlocks);
343     List<Long> timeRange = new ArrayList<Long>();
344     timeRange.add(this.tr.getMin());
345     timeRange.add(this.tr.getMax());
346     map.put("timeRange", timeRange);
347     int colCount = 0;
348     // iterate through affected families and add details
349     for (Map.Entry<byte [], NavigableSet<byte[]>> entry :
350       this.familyMap.entrySet()) {
351       List<String> familyList = new ArrayList<String>();
352       columns.put(Bytes.toStringBinary(entry.getKey()), familyList);
353       if(entry.getValue() == null) {
354         colCount++;
355         --maxCols;
356         familyList.add("ALL");
357       } else {
358         colCount += entry.getValue().size();
359         if (maxCols <= 0) {
360           continue;
361         }
362         for (byte [] column : entry.getValue()) {
363           if (--maxCols <= 0) {
364             continue;
365           }
366           familyList.add(Bytes.toStringBinary(column));
367         }
368       }   
369     }   
370     map.put("totalColumns", colCount);
371     return map;
372   }
373 
374   //Row
375   public int compareTo(Row other) {
376     return Bytes.compareTo(this.getRow(), other.getRow());
377   }
378   
379   //Writable
380   public void readFields(final DataInput in)
381   throws IOException {
382     int version = in.readByte();
383     if (version > GET_VERSION) {
384       throw new IOException("unsupported version");
385     }
386     this.row = Bytes.readByteArray(in);
387     this.lockId = in.readLong();
388     this.maxVersions = in.readInt();
389     boolean hasFilter = in.readBoolean();
390     if (hasFilter) {
391       this.filter = (Filter)createForName(Bytes.toString(Bytes.readByteArray(in)));
392       this.filter.readFields(in);
393     }
394     this.cacheBlocks = in.readBoolean();
395     this.tr = new TimeRange();
396     tr.readFields(in);
397     int numFamilies = in.readInt();
398     this.familyMap =
399       new TreeMap<byte [],NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
400     for(int i=0; i<numFamilies; i++) {
401       byte [] family = Bytes.readByteArray(in);
402       boolean hasColumns = in.readBoolean();
403       NavigableSet<byte []> set = null;
404       if(hasColumns) {
405         int numColumns = in.readInt();
406         set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
407         for(int j=0; j<numColumns; j++) {
408           byte [] qualifier = Bytes.readByteArray(in);
409           set.add(qualifier);
410         }
411       }
412       this.familyMap.put(family, set);
413     }
414     readAttributes(in);
415   }
416 
417   public void write(final DataOutput out)
418   throws IOException {
419     out.writeByte(GET_VERSION);
420     Bytes.writeByteArray(out, this.row);
421     out.writeLong(this.lockId);
422     out.writeInt(this.maxVersions);
423     if(this.filter == null) {
424       out.writeBoolean(false);
425     } else {
426       out.writeBoolean(true);
427       Bytes.writeByteArray(out, Bytes.toBytes(filter.getClass().getName()));
428       filter.write(out);
429     }
430     out.writeBoolean(this.cacheBlocks);
431     tr.write(out);
432     out.writeInt(familyMap.size());
433     for(Map.Entry<byte [], NavigableSet<byte []>> entry :
434       familyMap.entrySet()) {
435       Bytes.writeByteArray(out, entry.getKey());
436       NavigableSet<byte []> columnSet = entry.getValue();
437       if(columnSet == null) {
438         out.writeBoolean(false);
439       } else {
440         out.writeBoolean(true);
441         out.writeInt(columnSet.size());
442         for(byte [] qualifier : columnSet) {
443           Bytes.writeByteArray(out, qualifier);
444         }
445       }
446     }
447     writeAttributes(out);
448   }
449 
450   @SuppressWarnings("unchecked")
451   private Writable createForName(String className) {
452     try {
453       Class<? extends Writable> clazz =
454         (Class<? extends Writable>) Class.forName(className);
455       return WritableFactories.newInstance(clazz, new Configuration());
456     } catch (ClassNotFoundException e) {
457       throw new RuntimeException("Can't find class " + className);
458     }
459   }
460 }