View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.client;
20  
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.HashMap;
25  import java.util.List;
26  import java.util.Map;
27  import java.util.NavigableSet;
28  import java.util.Set;
29  import java.util.TreeMap;
30  import java.util.TreeSet;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.hbase.classification.InterfaceAudience;
35  import org.apache.hadoop.hbase.classification.InterfaceStability;
36  import org.apache.hadoop.hbase.HConstants;
37  import org.apache.hadoop.hbase.filter.Filter;
38  import org.apache.hadoop.hbase.io.TimeRange;
39  import org.apache.hadoop.hbase.security.access.Permission;
40  import org.apache.hadoop.hbase.security.visibility.Authorizations;
41  import org.apache.hadoop.hbase.util.Bytes;
42  
43  /**
44   * Used to perform Get operations on a single row.
45   * <p>
46   * To get everything for a row, instantiate a Get object with the row to get.
47   * To further narrow the scope of what to Get, use the methods 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, call {@link #setFilter(Filter) setFilter}.
65   */
66  @InterfaceAudience.Public
67  @InterfaceStability.Stable
68  public class Get extends Query
69    implements Row, Comparable<Row> {
70    private static final Log LOG = LogFactory.getLog(Get.class);
71  
72    private byte [] row = null;
73    private int maxVersions = 1;
74    private boolean cacheBlocks = true;
75    private int storeLimit = -1;
76    private int storeOffset = 0;
77    private TimeRange tr = new TimeRange();
78    private boolean checkExistenceOnly = false;
79    private boolean closestRowBefore = false;
80    private Map<byte [], NavigableSet<byte []>> familyMap =
81      new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
82  
83    /**
84     * Create a Get operation for the specified row.
85     * <p>
86     * If no further operations are done, this will get the latest version of
87     * all columns in all families of the specified row.
88     * @param row row key
89     */
90    public Get(byte [] row) {
91      Mutation.checkRow(row);
92      this.row = row;
93    }
94  
95    /**
96     * Copy-constructor
97     *
98     * @param get
99     */
100   public Get(Get get) {
101     this(get.getRow());
102     // from Query
103     this.setFilter(get.getFilter());
104     this.setReplicaId(get.getReplicaId());
105     this.setConsistency(get.getConsistency());
106     // from Get
107     this.cacheBlocks = get.getCacheBlocks();
108     this.maxVersions = get.getMaxVersions();
109     this.storeLimit = get.getMaxResultsPerColumnFamily();
110     this.storeOffset = get.getRowOffsetPerColumnFamily();
111     this.tr = get.getTimeRange();
112     this.checkExistenceOnly = get.isCheckExistenceOnly();
113     this.loadColumnFamiliesOnDemand = get.getLoadColumnFamiliesOnDemandValue();
114     this.closestRowBefore = get.isClosestRowBefore();
115     Map<byte[], NavigableSet<byte[]>> fams = get.getFamilyMap();
116     for (Map.Entry<byte[],NavigableSet<byte[]>> entry : fams.entrySet()) {
117       byte [] fam = entry.getKey();
118       NavigableSet<byte[]> cols = entry.getValue();
119       if (cols != null && cols.size() > 0) {
120         for (byte[] col : cols) {
121           addColumn(fam, col);
122         }
123       } else {
124         addFamily(fam);
125       }
126     }
127     for (Map.Entry<String, byte[]> attr : get.getAttributesMap().entrySet()) {
128       setAttribute(attr.getKey(), attr.getValue());
129     }
130     for (Map.Entry<byte[], TimeRange> entry : get.getColumnFamilyTimeRange().entrySet()) {
131       TimeRange tr = entry.getValue();
132       setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());
133     }
134   }
135 
136   public boolean isCheckExistenceOnly() {
137     return checkExistenceOnly;
138   }
139 
140   public Get setCheckExistenceOnly(boolean checkExistenceOnly) {
141     this.checkExistenceOnly = checkExistenceOnly;
142     return this;
143   }
144 
145   public boolean isClosestRowBefore() {
146     return closestRowBefore;
147   }
148 
149   public Get setClosestRowBefore(boolean closestRowBefore) {
150     this.closestRowBefore = closestRowBefore;
151     return this;
152   }
153 
154   /**
155    * Get all columns from the specified family.
156    * <p>
157    * Overrides previous calls to addColumn for this family.
158    * @param family family name
159    * @return the Get object
160    */
161   public Get addFamily(byte [] family) {
162     familyMap.remove(family);
163     familyMap.put(family, null);
164     return this;
165   }
166 
167   /**
168    * Get the column from the specific family with the specified qualifier.
169    * <p>
170    * Overrides previous calls to addFamily for this family.
171    * @param family family name
172    * @param qualifier column qualifier
173    * @return the Get objec
174    */
175   public Get addColumn(byte [] family, byte [] qualifier) {
176     NavigableSet<byte []> set = familyMap.get(family);
177     if(set == null) {
178       set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
179     }
180     if (qualifier == null) {
181       qualifier = HConstants.EMPTY_BYTE_ARRAY;
182     }
183     set.add(qualifier);
184     familyMap.put(family, set);
185     return this;
186   }
187 
188   /**
189    * Get versions of columns only within the specified timestamp range,
190    * [minStamp, maxStamp).
191    * @param minStamp minimum timestamp value, inclusive
192    * @param maxStamp maximum timestamp value, exclusive
193    * @throws IOException
194    * @return this for invocation chaining
195    */
196   public Get setTimeRange(long minStamp, long maxStamp) throws IOException {
197     tr = new TimeRange(minStamp, maxStamp);
198     return this;
199   }
200 
201   /**
202    * Get versions of columns with the specified timestamp.
203    * @param timestamp version timestamp
204    * @return this for invocation chaining
205    */
206   public Get setTimeStamp(long timestamp)
207   throws IOException {
208     try {
209       tr = new TimeRange(timestamp, timestamp+1);
210     } catch(Exception e) {
211       // This should never happen, unless integer overflow or something extremely wrong...
212       LOG.error("TimeRange failed, likely caused by integer overflow. ", e);
213       throw e;
214     }
215     return this;
216   }
217 
218   @Override public Get setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp) {
219     return (Get) super.setColumnFamilyTimeRange(cf, minStamp, maxStamp);
220   }
221 
222   /**
223    * Get all available versions.
224    * @return this for invocation chaining
225    */
226   public Get setMaxVersions() {
227     this.maxVersions = Integer.MAX_VALUE;
228     return this;
229   }
230 
231   /**
232    * Get up to the specified number of versions of each column.
233    * @param maxVersions maximum versions for each column
234    * @throws IOException if invalid number of versions
235    * @return this for invocation chaining
236    */
237   public Get setMaxVersions(int maxVersions) throws IOException {
238     if(maxVersions <= 0) {
239       throw new IOException("maxVersions must be positive");
240     }
241     this.maxVersions = maxVersions;
242     return this;
243   }
244 
245   public Get setLoadColumnFamiliesOnDemand(boolean value) {
246     return (Get) super.setLoadColumnFamiliesOnDemand(value);
247   }
248 
249   /**
250    * Set the maximum number of values to return per row per Column Family
251    * @param limit the maximum number of values returned / row / CF
252    * @return this for invocation chaining
253    */
254   public Get setMaxResultsPerColumnFamily(int limit) {
255     this.storeLimit = limit;
256     return this;
257   }
258 
259   /**
260    * Set offset for the row per Column Family. This offset is only within a particular row/CF
261    * combination. It gets reset back to zero when we move to the next row or CF.
262    * @param offset is the number of kvs that will be skipped.
263    * @return this for invocation chaining
264    */
265   public Get setRowOffsetPerColumnFamily(int offset) {
266     this.storeOffset = offset;
267     return this;
268   }
269 
270   @Override
271   public Get setFilter(Filter filter) {
272     super.setFilter(filter);
273     return this;
274   }
275 
276   /* Accessors */
277 
278   /**
279    * Set whether blocks should be cached for this Get.
280    * <p>
281    * This is true by default.  When true, default settings of the table and
282    * family are used (this will never override caching blocks if the block
283    * cache is disabled for that family or entirely).
284    *
285    * @param cacheBlocks if false, default settings are overridden and blocks
286    * will not be cached
287    */
288   public Get setCacheBlocks(boolean cacheBlocks) {
289     this.cacheBlocks = cacheBlocks;
290     return this;
291   }
292 
293   /**
294    * Get whether blocks should be cached for this Get.
295    * @return true if default caching should be used, false if blocks should not
296    * be cached
297    */
298   public boolean getCacheBlocks() {
299     return cacheBlocks;
300   }
301 
302   /**
303    * Method for retrieving the get's row
304    * @return row
305    */
306   @Override
307   public byte [] getRow() {
308     return this.row;
309   }
310 
311   /**
312    * Method for retrieving the get's maximum number of version
313    * @return the maximum number of version to fetch for this get
314    */
315   public int getMaxVersions() {
316     return this.maxVersions;
317   }
318 
319   /**
320    * Method for retrieving the get's maximum number of values
321    * to return per Column Family
322    * @return the maximum number of values to fetch per CF
323    */
324   public int getMaxResultsPerColumnFamily() {
325     return this.storeLimit;
326   }
327 
328   /**
329    * Method for retrieving the get's offset per row per column
330    * family (#kvs to be skipped)
331    * @return the row offset
332    */
333   public int getRowOffsetPerColumnFamily() {
334     return this.storeOffset;
335   }
336 
337   /**
338    * Method for retrieving the get's TimeRange
339    * @return timeRange
340    */
341   public TimeRange getTimeRange() {
342     return this.tr;
343   }
344 
345   /**
346    * Method for retrieving the keys in the familyMap
347    * @return keys in the current familyMap
348    */
349   public Set<byte[]> familySet() {
350     return this.familyMap.keySet();
351   }
352 
353   /**
354    * Method for retrieving the number of families to get from
355    * @return number of families
356    */
357   public int numFamilies() {
358     return this.familyMap.size();
359   }
360 
361   /**
362    * Method for checking if any families have been inserted into this Get
363    * @return true if familyMap is non empty false otherwise
364    */
365   public boolean hasFamilies() {
366     return !this.familyMap.isEmpty();
367   }
368 
369   /**
370    * Method for retrieving the get's familyMap
371    * @return familyMap
372    */
373   public Map<byte[],NavigableSet<byte[]>> getFamilyMap() {
374     return this.familyMap;
375   }
376 
377   /**
378    * Compile the table and column family (i.e. schema) information
379    * into a String. Useful for parsing and aggregation by debugging,
380    * logging, and administration tools.
381    * @return Map
382    */
383   @Override
384   public Map<String, Object> getFingerprint() {
385     Map<String, Object> map = new HashMap<String, Object>();
386     List<String> families = new ArrayList<String>();
387     map.put("families", families);
388     for (Map.Entry<byte [], NavigableSet<byte[]>> entry :
389       this.familyMap.entrySet()) {
390       families.add(Bytes.toStringBinary(entry.getKey()));
391     }
392     return map;
393   }
394 
395   /**
396    * Compile the details beyond the scope of getFingerprint (row, columns,
397    * timestamps, etc.) into a Map along with the fingerprinted information.
398    * Useful for debugging, logging, and administration tools.
399    * @param maxCols a limit on the number of columns output prior to truncation
400    * @return Map
401    */
402   @Override
403   public Map<String, Object> toMap(int maxCols) {
404     // we start with the fingerprint map and build on top of it.
405     Map<String, Object> map = getFingerprint();
406     // replace the fingerprint's simple list of families with a
407     // map from column families to lists of qualifiers and kv details
408     Map<String, List<String>> columns = new HashMap<String, List<String>>();
409     map.put("families", columns);
410     // add scalar information first
411     map.put("row", Bytes.toStringBinary(this.row));
412     map.put("maxVersions", this.maxVersions);
413     map.put("cacheBlocks", this.cacheBlocks);
414     List<Long> timeRange = new ArrayList<Long>();
415     timeRange.add(this.tr.getMin());
416     timeRange.add(this.tr.getMax());
417     map.put("timeRange", timeRange);
418     int colCount = 0;
419     // iterate through affected families and add details
420     for (Map.Entry<byte [], NavigableSet<byte[]>> entry :
421       this.familyMap.entrySet()) {
422       List<String> familyList = new ArrayList<String>();
423       columns.put(Bytes.toStringBinary(entry.getKey()), familyList);
424       if(entry.getValue() == null) {
425         colCount++;
426         --maxCols;
427         familyList.add("ALL");
428       } else {
429         colCount += entry.getValue().size();
430         if (maxCols <= 0) {
431           continue;
432         }
433         for (byte [] column : entry.getValue()) {
434           if (--maxCols <= 0) {
435             continue;
436           }
437           familyList.add(Bytes.toStringBinary(column));
438         }
439       }
440     }
441     map.put("totalColumns", colCount);
442     if (this.filter != null) {
443       map.put("filter", this.filter.toString());
444     }
445     // add the id if set
446     if (getId() != null) {
447       map.put("id", getId());
448     }
449     return map;
450   }
451 
452   //Row
453   @Override
454   public int compareTo(Row other) {
455     // TODO: This is wrong.  Can't have two gets the same just because on same row.
456     return Bytes.compareTo(this.getRow(), other.getRow());
457   }
458 
459   @Override
460   public int hashCode() {
461     // TODO: This is wrong.  Can't have two gets the same just because on same row.  But it
462     // matches how equals works currently and gets rid of the findbugs warning.
463     return Bytes.hashCode(this.getRow());
464   }
465 
466   @Override
467   public boolean equals(Object obj) {
468     if (this == obj) {
469       return true;
470     }
471     if (obj == null || getClass() != obj.getClass()) {
472       return false;
473     }
474     Row other = (Row) obj;
475     // TODO: This is wrong.  Can't have two gets the same just because on same row.
476     return compareTo(other) == 0;
477   }
478 
479   @Override
480   public Get setAttribute(String name, byte[] value) {
481     return (Get) super.setAttribute(name, value);
482   }
483 
484   @Override
485   public Get setId(String id) {
486     return (Get) super.setId(id);
487   }
488 
489   @Override
490   public Get setAuthorizations(Authorizations authorizations) {
491     return (Get) super.setAuthorizations(authorizations);
492   }
493 
494   @Override
495   public Get setACL(Map<String, Permission> perms) {
496     return (Get) super.setACL(perms);
497   }
498 
499   @Override
500   public Get setACL(String user, Permission perms) {
501     return (Get) super.setACL(user, perms);
502   }
503 
504   @Override
505   public Get setConsistency(Consistency consistency) {
506     return (Get) super.setConsistency(consistency);
507   }
508 
509   @Override
510   public Get setReplicaId(int Id) {
511     return (Get) super.setReplicaId(Id);
512   }
513 
514   @Override
515   public Get setIsolationLevel(IsolationLevel level) {
516       return (Get) super.setIsolationLevel(level);
517   }
518 
519 }