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