001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.client;
019
020import java.io.IOException;
021import java.nio.ByteBuffer;
022import java.util.ArrayList;
023import java.util.HashMap;
024import java.util.List;
025import java.util.Map;
026import java.util.NavigableSet;
027import java.util.Set;
028import java.util.TreeMap;
029import java.util.TreeSet;
030import org.apache.hadoop.hbase.HConstants;
031import org.apache.hadoop.hbase.filter.Filter;
032import org.apache.hadoop.hbase.io.TimeRange;
033import org.apache.hadoop.hbase.security.access.Permission;
034import org.apache.hadoop.hbase.security.visibility.Authorizations;
035import org.apache.hadoop.hbase.util.Bytes;
036import org.apache.yetus.audience.InterfaceAudience;
037import org.slf4j.Logger;
038import org.slf4j.LoggerFactory;
039
040/**
041 * Used to perform Get operations on a single row.
042 * <p>
043 * To get everything for a row, instantiate a Get object with the row to get. To further narrow the
044 * scope of what to Get, use the methods below.
045 * <p>
046 * To get all columns from specific families, execute {@link #addFamily(byte[]) addFamily} for each
047 * family to retrieve.
048 * <p>
049 * To get specific columns, execute {@link #addColumn(byte[], byte[]) addColumn} for each column to
050 * retrieve.
051 * <p>
052 * To only retrieve columns within a specific range of version timestamps, execute
053 * {@link #setTimeRange(long, long) setTimeRange}.
054 * <p>
055 * To only retrieve columns with a specific timestamp, execute {@link #setTimestamp(long)
056 * setTimestamp}.
057 * <p>
058 * To limit the number of versions of each column to be returned, execute
059 * {@link #setMaxVersions(int) setMaxVersions}.
060 * <p>
061 * To add a filter, call {@link #setFilter(Filter) setFilter}.
062 */
063@InterfaceAudience.Public
064public class Get extends Query implements Row {
065  private static final Logger LOG = LoggerFactory.getLogger(Get.class);
066
067  private byte[] row = null;
068  private int maxVersions = 1;
069  private boolean cacheBlocks = true;
070  private int storeLimit = -1;
071  private int storeOffset = 0;
072  private TimeRange tr = TimeRange.allTime();
073  private boolean checkExistenceOnly = false;
074  private boolean closestRowBefore = false;
075  private Map<byte[], NavigableSet<byte[]>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
076
077  /**
078   * Create a Get operation for the specified row.
079   * <p>
080   * If no further operations are done, this will get the latest version of all columns in all
081   * families of the specified row.
082   * @param row row key
083   */
084  public Get(byte[] row) {
085    Mutation.checkRow(row);
086    this.row = row;
087  }
088
089  /**
090   * Copy-constructor n
091   */
092  public Get(Get get) {
093    this(get.getRow());
094    // from Query
095    this.setFilter(get.getFilter());
096    this.setReplicaId(get.getReplicaId());
097    this.setConsistency(get.getConsistency());
098    // from Get
099    this.cacheBlocks = get.getCacheBlocks();
100    this.maxVersions = get.getMaxVersions();
101    this.storeLimit = get.getMaxResultsPerColumnFamily();
102    this.storeOffset = get.getRowOffsetPerColumnFamily();
103    this.tr = get.getTimeRange();
104    this.checkExistenceOnly = get.isCheckExistenceOnly();
105    this.loadColumnFamiliesOnDemand = get.getLoadColumnFamiliesOnDemandValue();
106    Map<byte[], NavigableSet<byte[]>> fams = get.getFamilyMap();
107    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : fams.entrySet()) {
108      byte[] fam = entry.getKey();
109      NavigableSet<byte[]> cols = entry.getValue();
110      if (cols != null && cols.size() > 0) {
111        for (byte[] col : cols) {
112          addColumn(fam, col);
113        }
114      } else {
115        addFamily(fam);
116      }
117    }
118    for (Map.Entry<String, byte[]> attr : get.getAttributesMap().entrySet()) {
119      setAttribute(attr.getKey(), attr.getValue());
120    }
121    for (Map.Entry<byte[], TimeRange> entry : get.getColumnFamilyTimeRange().entrySet()) {
122      TimeRange tr = entry.getValue();
123      setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());
124    }
125    super.setPriority(get.getPriority());
126  }
127
128  /**
129   * Create a Get operation for the specified row. nnn
130   */
131  public Get(byte[] row, int rowOffset, int rowLength) {
132    Mutation.checkRow(row, rowOffset, rowLength);
133    this.row = Bytes.copy(row, rowOffset, rowLength);
134  }
135
136  /**
137   * Create a Get operation for the specified row. n
138   */
139  public Get(ByteBuffer row) {
140    Mutation.checkRow(row);
141    this.row = new byte[row.remaining()];
142    row.get(this.row);
143  }
144
145  public boolean isCheckExistenceOnly() {
146    return checkExistenceOnly;
147  }
148
149  public Get setCheckExistenceOnly(boolean checkExistenceOnly) {
150    this.checkExistenceOnly = checkExistenceOnly;
151    return this;
152  }
153
154  /**
155   * This will always return the default value which is false as client cannot set the value to this
156   * property any more.
157   * @deprecated since 2.0.0 and will be removed in 3.0.0
158   */
159  @Deprecated
160  public boolean isClosestRowBefore() {
161    return closestRowBefore;
162  }
163
164  /**
165   * This is not used any more and does nothing. Use reverse scan instead.
166   * @deprecated since 2.0.0 and will be removed in 3.0.0
167   */
168  @Deprecated
169  public Get setClosestRowBefore(boolean closestRowBefore) {
170    // do Nothing
171    return this;
172  }
173
174  /**
175   * Get all columns from the specified family.
176   * <p>
177   * Overrides previous calls to addColumn for this family.
178   * @param family family name
179   * @return the Get object
180   */
181  public Get addFamily(byte[] family) {
182    familyMap.remove(family);
183    familyMap.put(family, null);
184    return this;
185  }
186
187  /**
188   * Get the column from the specific family with the specified qualifier.
189   * <p>
190   * Overrides previous calls to addFamily for this family.
191   * @param family    family name
192   * @param qualifier column qualifier
193   * @return the Get objec
194   */
195  public Get addColumn(byte[] family, byte[] qualifier) {
196    NavigableSet<byte[]> set = familyMap.get(family);
197    if (set == null) {
198      set = new TreeSet<>(Bytes.BYTES_COMPARATOR);
199      familyMap.put(family, set);
200    }
201    if (qualifier == null) {
202      qualifier = HConstants.EMPTY_BYTE_ARRAY;
203    }
204    set.add(qualifier);
205    return this;
206  }
207
208  /**
209   * Get versions of columns only within the specified timestamp range, [minStamp, maxStamp).
210   * @param minStamp minimum timestamp value, inclusive
211   * @param maxStamp maximum timestamp value, exclusive n * @return this for invocation chaining
212   */
213  public Get setTimeRange(long minStamp, long maxStamp) throws IOException {
214    tr = new TimeRange(minStamp, maxStamp);
215    return this;
216  }
217
218  /**
219   * Get versions of columns with the specified timestamp.
220   * @param timestamp version timestamp
221   * @return this for invocation chaining
222   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
223   *             {@link #setTimestamp(long)} instead
224   */
225  @Deprecated
226  public Get setTimeStamp(long timestamp) throws IOException {
227    return this.setTimestamp(timestamp);
228  }
229
230  /**
231   * Get versions of columns with the specified timestamp.
232   * @param timestamp version timestamp
233   * @return this for invocation chaining
234   */
235  public Get setTimestamp(long timestamp) {
236    try {
237      tr = new TimeRange(timestamp, timestamp + 1);
238    } catch (Exception e) {
239      // This should never happen, unless integer overflow or something extremely wrong...
240      LOG.error("TimeRange failed, likely caused by integer overflow. ", e);
241      throw e;
242    }
243
244    return this;
245  }
246
247  @Override
248  public Get setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp) {
249    return (Get) super.setColumnFamilyTimeRange(cf, minStamp, maxStamp);
250  }
251
252  /**
253   * Get all available versions.
254   * @return this for invocation chaining
255   * @deprecated It is easy to misunderstand with column family's max versions, so use
256   *             {@link #readAllVersions()} instead.
257   */
258  @Deprecated
259  public Get setMaxVersions() {
260    return readAllVersions();
261  }
262
263  /**
264   * Get up to the specified number of versions of each column.
265   * @param maxVersions maximum versions for each column
266   * @throws IOException if invalid number of versions
267   * @return this for invocation chaining
268   * @deprecated It is easy to misunderstand with column family's max versions, so use
269   *             {@link #readVersions(int)} instead.
270   */
271  @Deprecated
272  public Get setMaxVersions(int maxVersions) throws IOException {
273    return readVersions(maxVersions);
274  }
275
276  /**
277   * Get all available versions.
278   * @return this for invocation chaining
279   */
280  public Get readAllVersions() {
281    this.maxVersions = Integer.MAX_VALUE;
282    return this;
283  }
284
285  /**
286   * Get up to the specified number of versions of each column.
287   * @param versions specified number of versions for each column
288   * @throws IOException if invalid number of versions
289   * @return this for invocation chaining
290   */
291  public Get readVersions(int versions) throws IOException {
292    if (versions <= 0) {
293      throw new IOException("versions must be positive");
294    }
295    this.maxVersions = versions;
296    return this;
297  }
298
299  @Override
300  public Get setLoadColumnFamiliesOnDemand(boolean value) {
301    return (Get) super.setLoadColumnFamiliesOnDemand(value);
302  }
303
304  /**
305   * Set the maximum number of values to return per row per Column Family
306   * @param limit the maximum number of values returned / row / CF
307   * @return this for invocation chaining
308   */
309  public Get setMaxResultsPerColumnFamily(int limit) {
310    this.storeLimit = limit;
311    return this;
312  }
313
314  /**
315   * Set offset for the row per Column Family. This offset is only within a particular row/CF
316   * combination. It gets reset back to zero when we move to the next row or CF.
317   * @param offset is the number of kvs that will be skipped.
318   * @return this for invocation chaining
319   */
320  public Get setRowOffsetPerColumnFamily(int offset) {
321    this.storeOffset = offset;
322    return this;
323  }
324
325  @Override
326  public Get setFilter(Filter filter) {
327    super.setFilter(filter);
328    return this;
329  }
330
331  /* Accessors */
332
333  /**
334   * Set whether blocks should be cached for this Get.
335   * <p>
336   * This is true by default. When true, default settings of the table and family are used (this
337   * will never override caching blocks if the block cache is disabled for that family or entirely).
338   * @param cacheBlocks if false, default settings are overridden and blocks will not be cached
339   */
340  public Get setCacheBlocks(boolean cacheBlocks) {
341    this.cacheBlocks = cacheBlocks;
342    return this;
343  }
344
345  /**
346   * Get whether blocks should be cached for this Get.
347   * @return true if default caching should be used, false if blocks should not be cached
348   */
349  public boolean getCacheBlocks() {
350    return cacheBlocks;
351  }
352
353  /**
354   * Method for retrieving the get's row n
355   */
356  @Override
357  public byte[] getRow() {
358    return this.row;
359  }
360
361  /**
362   * Method for retrieving the get's maximum number of version
363   * @return the maximum number of version to fetch for this get
364   */
365  public int getMaxVersions() {
366    return this.maxVersions;
367  }
368
369  /**
370   * Method for retrieving the get's maximum number of values to return per Column Family
371   * @return the maximum number of values to fetch per CF
372   */
373  public int getMaxResultsPerColumnFamily() {
374    return this.storeLimit;
375  }
376
377  /**
378   * Method for retrieving the get's offset per row per column family (#kvs to be skipped)
379   * @return the row offset
380   */
381  public int getRowOffsetPerColumnFamily() {
382    return this.storeOffset;
383  }
384
385  /**
386   * Method for retrieving the get's TimeRange n
387   */
388  public TimeRange getTimeRange() {
389    return this.tr;
390  }
391
392  /**
393   * Method for retrieving the keys in the familyMap
394   * @return keys in the current familyMap
395   */
396  public Set<byte[]> familySet() {
397    return this.familyMap.keySet();
398  }
399
400  /**
401   * Method for retrieving the number of families to get from
402   * @return number of families
403   */
404  public int numFamilies() {
405    return this.familyMap.size();
406  }
407
408  /**
409   * Method for checking if any families have been inserted into this Get
410   * @return true if familyMap is non empty false otherwise
411   */
412  public boolean hasFamilies() {
413    return !this.familyMap.isEmpty();
414  }
415
416  /**
417   * Method for retrieving the get's familyMap n
418   */
419  public Map<byte[], NavigableSet<byte[]>> getFamilyMap() {
420    return this.familyMap;
421  }
422
423  /**
424   * Compile the table and column family (i.e. schema) information into a String. Useful for parsing
425   * and aggregation by debugging, logging, and administration tools. n
426   */
427  @Override
428  public Map<String, Object> getFingerprint() {
429    Map<String, Object> map = new HashMap<>();
430    List<String> families = new ArrayList<>(this.familyMap.entrySet().size());
431    map.put("families", families);
432    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : this.familyMap.entrySet()) {
433      families.add(Bytes.toStringBinary(entry.getKey()));
434    }
435    return map;
436  }
437
438  /**
439   * Compile the details beyond the scope of getFingerprint (row, columns, timestamps, etc.) into a
440   * Map along with the fingerprinted information. Useful for debugging, logging, and administration
441   * tools.
442   * @param maxCols a limit on the number of columns output prior to truncation n
443   */
444  @Override
445  public Map<String, Object> toMap(int maxCols) {
446    // we start with the fingerprint map and build on top of it.
447    Map<String, Object> map = getFingerprint();
448    // replace the fingerprint's simple list of families with a
449    // map from column families to lists of qualifiers and kv details
450    Map<String, List<String>> columns = new HashMap<>();
451    map.put("families", columns);
452    // add scalar information first
453    map.put("row", Bytes.toStringBinary(this.row));
454    map.put("maxVersions", this.maxVersions);
455    map.put("cacheBlocks", this.cacheBlocks);
456    List<Long> timeRange = new ArrayList<>(2);
457    timeRange.add(this.tr.getMin());
458    timeRange.add(this.tr.getMax());
459    map.put("timeRange", timeRange);
460    int colCount = 0;
461    // iterate through affected families and add details
462    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : this.familyMap.entrySet()) {
463      List<String> familyList = new ArrayList<>();
464      columns.put(Bytes.toStringBinary(entry.getKey()), familyList);
465      if (entry.getValue() == null) {
466        colCount++;
467        --maxCols;
468        familyList.add("ALL");
469      } else {
470        colCount += entry.getValue().size();
471        if (maxCols <= 0) {
472          continue;
473        }
474        for (byte[] column : entry.getValue()) {
475          if (--maxCols <= 0) {
476            continue;
477          }
478          familyList.add(Bytes.toStringBinary(column));
479        }
480      }
481    }
482    map.put("totalColumns", colCount);
483    if (this.filter != null) {
484      map.put("filter", this.filter.toString());
485    }
486    // add the id if set
487    if (getId() != null) {
488      map.put("id", getId());
489    }
490    return map;
491  }
492
493  // Row
494  @Override
495  public int compareTo(Row other) {
496    // TODO: This is wrong. Can't have two gets the same just because on same row.
497    return Bytes.compareTo(this.getRow(), other.getRow());
498  }
499
500  @Override
501  public int hashCode() {
502    // TODO: This is wrong. Can't have two gets the same just because on same row. But it
503    // matches how equals works currently and gets rid of the findbugs warning.
504    return Bytes.hashCode(this.getRow());
505  }
506
507  @Override
508  public boolean equals(Object obj) {
509    if (this == obj) {
510      return true;
511    }
512    if (!(obj instanceof Row)) {
513      return false;
514    }
515    Row other = (Row) obj;
516    // TODO: This is wrong. Can't have two gets the same just because on same row.
517    return compareTo(other) == 0;
518  }
519
520  @Override
521  public Get setAttribute(String name, byte[] value) {
522    return (Get) super.setAttribute(name, value);
523  }
524
525  @Override
526  public Get setId(String id) {
527    return (Get) super.setId(id);
528  }
529
530  @Override
531  public Get setAuthorizations(Authorizations authorizations) {
532    return (Get) super.setAuthorizations(authorizations);
533  }
534
535  @Override
536  public Get setACL(Map<String, Permission> perms) {
537    return (Get) super.setACL(perms);
538  }
539
540  @Override
541  public Get setACL(String user, Permission perms) {
542    return (Get) super.setACL(user, perms);
543  }
544
545  @Override
546  public Get setConsistency(Consistency consistency) {
547    return (Get) super.setConsistency(consistency);
548  }
549
550  @Override
551  public Get setReplicaId(int Id) {
552    return (Get) super.setReplicaId(Id);
553  }
554
555  @Override
556  public Get setIsolationLevel(IsolationLevel level) {
557    return (Get) super.setIsolationLevel(level);
558  }
559
560  @Override
561  public Get setPriority(int priority) {
562    return (Get) super.setPriority(priority);
563  }
564}