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 {@link #readVersions(int)
059 * readVersions}.
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 Map<byte[], NavigableSet<byte[]>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
075
076  /**
077   * Create a Get operation for the specified row.
078   * <p>
079   * If no further operations are done, this will get the latest version of all columns in all
080   * families of the specified row.
081   * @param row row key
082   */
083  public Get(byte[] row) {
084    Mutation.checkRow(row);
085    this.row = row;
086  }
087
088  /**
089   * Copy-constructor
090   */
091  public Get(Get get) {
092    this(get.getRow());
093    // from Query
094    this.setFilter(get.getFilter());
095    this.setReplicaId(get.getReplicaId());
096    this.setConsistency(get.getConsistency());
097    // from Get
098    this.cacheBlocks = get.getCacheBlocks();
099    this.maxVersions = get.getMaxVersions();
100    this.storeLimit = get.getMaxResultsPerColumnFamily();
101    this.storeOffset = get.getRowOffsetPerColumnFamily();
102    this.tr = get.getTimeRange();
103    this.checkExistenceOnly = get.isCheckExistenceOnly();
104    this.loadColumnFamiliesOnDemand = get.getLoadColumnFamiliesOnDemandValue();
105    Map<byte[], NavigableSet<byte[]>> fams = get.getFamilyMap();
106    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : fams.entrySet()) {
107      byte[] fam = entry.getKey();
108      NavigableSet<byte[]> cols = entry.getValue();
109      if (cols != null && cols.size() > 0) {
110        for (byte[] col : cols) {
111          addColumn(fam, col);
112        }
113      } else {
114        addFamily(fam);
115      }
116    }
117    for (Map.Entry<String, byte[]> attr : get.getAttributesMap().entrySet()) {
118      setAttribute(attr.getKey(), attr.getValue());
119    }
120    for (Map.Entry<byte[], TimeRange> entry : get.getColumnFamilyTimeRange().entrySet()) {
121      TimeRange tr = entry.getValue();
122      setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());
123    }
124    super.setPriority(get.getPriority());
125  }
126
127  /**
128   * Create a Get operation for the specified row.
129   */
130  public Get(byte[] row, int rowOffset, int rowLength) {
131    Mutation.checkRow(row, rowOffset, rowLength);
132    this.row = Bytes.copy(row, rowOffset, rowLength);
133  }
134
135  /**
136   * Create a Get operation for the specified row.
137   */
138  public Get(ByteBuffer row) {
139    Mutation.checkRow(row);
140    this.row = new byte[row.remaining()];
141    row.get(this.row);
142  }
143
144  public boolean isCheckExistenceOnly() {
145    return checkExistenceOnly;
146  }
147
148  public Get setCheckExistenceOnly(boolean checkExistenceOnly) {
149    this.checkExistenceOnly = checkExistenceOnly;
150    return this;
151  }
152
153  /**
154   * Get all columns from the specified family.
155   * <p>
156   * Overrides previous calls to addColumn for this family.
157   * @param family family name
158   * @return the Get object
159   */
160  public Get addFamily(byte[] family) {
161    familyMap.remove(family);
162    familyMap.put(family, null);
163    return this;
164  }
165
166  /**
167   * Get the column from the specific family with the specified qualifier.
168   * <p>
169   * Overrides previous calls to addFamily for this family.
170   * @param family    family name
171   * @param qualifier column qualifier
172   * @return the Get objec
173   */
174  public Get addColumn(byte[] family, byte[] qualifier) {
175    NavigableSet<byte[]> set = familyMap.get(family);
176    if (set == null) {
177      set = new TreeSet<>(Bytes.BYTES_COMPARATOR);
178      familyMap.put(family, set);
179    }
180    if (qualifier == null) {
181      qualifier = HConstants.EMPTY_BYTE_ARRAY;
182    }
183    set.add(qualifier);
184    return this;
185  }
186
187  /**
188   * Get versions of columns only within the specified timestamp range, [minStamp, maxStamp).
189   * @param minStamp minimum timestamp value, inclusive
190   * @param maxStamp maximum timestamp value, exclusive
191   * @return this for invocation chaining
192   */
193  public Get setTimeRange(long minStamp, long maxStamp) throws IOException {
194    tr = TimeRange.between(minStamp, maxStamp);
195    return this;
196  }
197
198  /**
199   * Get versions of columns with the specified timestamp.
200   * @param timestamp version timestamp
201   * @return this for invocation chaining
202   */
203  public Get setTimestamp(long timestamp) {
204    try {
205      tr = TimeRange.at(timestamp);
206    } catch (Exception e) {
207      // This should never happen, unless integer overflow or something extremely wrong...
208      LOG.error("TimeRange failed, likely caused by integer overflow. ", e);
209      throw e;
210    }
211    return this;
212  }
213
214  @Override
215  public Get setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp) {
216    return (Get) super.setColumnFamilyTimeRange(cf, minStamp, maxStamp);
217  }
218
219  /**
220   * Get all available versions.
221   * @return this for invocation chaining
222   */
223  public Get readAllVersions() {
224    this.maxVersions = Integer.MAX_VALUE;
225    return this;
226  }
227
228  /**
229   * Get up to the specified number of versions of each column.
230   * @param versions specified number of versions for each column
231   * @throws IOException if invalid number of versions
232   * @return this for invocation chaining
233   */
234  public Get readVersions(int versions) throws IOException {
235    if (versions <= 0) {
236      throw new IOException("versions must be positive");
237    }
238    this.maxVersions = versions;
239    return this;
240  }
241
242  @Override
243  public Get setLoadColumnFamiliesOnDemand(boolean value) {
244    return (Get) super.setLoadColumnFamiliesOnDemand(value);
245  }
246
247  /**
248   * Set the maximum number of values to return per row per Column Family
249   * @param limit the maximum number of values returned / row / CF
250   * @return this for invocation chaining
251   */
252  public Get setMaxResultsPerColumnFamily(int limit) {
253    this.storeLimit = limit;
254    return this;
255  }
256
257  /**
258   * Set offset for the row per Column Family. This offset is only within a particular row/CF
259   * combination. It gets reset back to zero when we move to the next row or CF.
260   * @param offset is the number of kvs that will be skipped.
261   * @return this for invocation chaining
262   */
263  public Get setRowOffsetPerColumnFamily(int offset) {
264    this.storeOffset = offset;
265    return this;
266  }
267
268  @Override
269  public Get setFilter(Filter filter) {
270    super.setFilter(filter);
271    return this;
272  }
273
274  /* Accessors */
275
276  /**
277   * Set whether blocks should be cached for this Get.
278   * <p>
279   * This is true by default. When true, default settings of the table and family are used (this
280   * will never override caching blocks if the block cache is disabled for that family or entirely).
281   * @param cacheBlocks if false, default settings are overridden and blocks will not be cached
282   */
283  public Get setCacheBlocks(boolean cacheBlocks) {
284    this.cacheBlocks = cacheBlocks;
285    return this;
286  }
287
288  /**
289   * Get whether blocks should be cached for this Get.
290   * @return true if default caching should be used, false if blocks should not be cached
291   */
292  public boolean getCacheBlocks() {
293    return cacheBlocks;
294  }
295
296  /**
297   * Method for retrieving the get's row
298   */
299  @Override
300  public byte[] getRow() {
301    return this.row;
302  }
303
304  /**
305   * Method for retrieving the get's maximum number of version
306   * @return the maximum number of version to fetch for this get
307   */
308  public int getMaxVersions() {
309    return this.maxVersions;
310  }
311
312  /**
313   * Method for retrieving the get's maximum number of values to return per Column Family
314   * @return the maximum number of values to fetch per CF
315   */
316  public int getMaxResultsPerColumnFamily() {
317    return this.storeLimit;
318  }
319
320  /**
321   * Method for retrieving the get's offset per row per column family (#kvs to be skipped)
322   * @return the row offset
323   */
324  public int getRowOffsetPerColumnFamily() {
325    return this.storeOffset;
326  }
327
328  /**
329   * Method for retrieving the get's TimeRange
330   */
331  public TimeRange getTimeRange() {
332    return this.tr;
333  }
334
335  /**
336   * Method for retrieving the keys in the familyMap
337   * @return keys in the current familyMap
338   */
339  public Set<byte[]> familySet() {
340    return this.familyMap.keySet();
341  }
342
343  /**
344   * Method for retrieving the number of families to get from
345   * @return number of families
346   */
347  public int numFamilies() {
348    return this.familyMap.size();
349  }
350
351  /**
352   * Method for checking if any families have been inserted into this Get
353   * @return true if familyMap is non empty false otherwise
354   */
355  public boolean hasFamilies() {
356    return !this.familyMap.isEmpty();
357  }
358
359  /**
360   * Method for retrieving the get's familyMap
361   */
362  public Map<byte[], NavigableSet<byte[]>> getFamilyMap() {
363    return this.familyMap;
364  }
365
366  /**
367   * Compile the table and column family (i.e. schema) information into a String. Useful for parsing
368   * and aggregation by debugging, logging, and administration tools.
369   */
370  @Override
371  public Map<String, Object> getFingerprint() {
372    Map<String, Object> map = new HashMap<>();
373    List<String> families = new ArrayList<>(this.familyMap.entrySet().size());
374    map.put("families", families);
375    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : this.familyMap.entrySet()) {
376      families.add(Bytes.toStringBinary(entry.getKey()));
377    }
378    return map;
379  }
380
381  /**
382   * Compile the details beyond the scope of getFingerprint (row, columns, timestamps, etc.) into a
383   * Map along with the fingerprinted information. Useful for debugging, logging, and administration
384   * tools.
385   * @param maxCols a limit on the number of columns output prior to truncation
386   */
387  @Override
388  public Map<String, Object> toMap(int maxCols) {
389    // we start with the fingerprint map and build on top of it.
390    Map<String, Object> map = getFingerprint();
391    // replace the fingerprint's simple list of families with a
392    // map from column families to lists of qualifiers and kv details
393    Map<String, List<String>> columns = new HashMap<>();
394    map.put("families", columns);
395    // add scalar information first
396    map.put("row", Bytes.toStringBinary(this.row));
397    map.put("maxVersions", this.maxVersions);
398    map.put("cacheBlocks", this.cacheBlocks);
399    List<Long> timeRange = new ArrayList<>(2);
400    timeRange.add(this.tr.getMin());
401    timeRange.add(this.tr.getMax());
402    map.put("timeRange", timeRange);
403    int colCount = 0;
404    // iterate through affected families and add details
405    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : this.familyMap.entrySet()) {
406      List<String> familyList = new ArrayList<>();
407      columns.put(Bytes.toStringBinary(entry.getKey()), familyList);
408      if (entry.getValue() == null) {
409        colCount++;
410        --maxCols;
411        familyList.add("ALL");
412      } else {
413        colCount += entry.getValue().size();
414        if (maxCols <= 0) {
415          continue;
416        }
417        for (byte[] column : entry.getValue()) {
418          if (--maxCols <= 0) {
419            continue;
420          }
421          familyList.add(Bytes.toStringBinary(column));
422        }
423      }
424    }
425    map.put("totalColumns", colCount);
426    if (this.filter != null) {
427      map.put("filter", this.filter.toString());
428    }
429    // add the id if set
430    if (getId() != null) {
431      map.put("id", getId());
432    }
433    return map;
434  }
435
436  @Override
437  public int hashCode() {
438    // TODO: This is wrong. Can't have two gets the same just because on same row. But it
439    // matches how equals works currently and gets rid of the findbugs warning.
440    return Bytes.hashCode(this.getRow());
441  }
442
443  @Override
444  public boolean equals(Object obj) {
445    if (this == obj) {
446      return true;
447    }
448    if (!(obj instanceof Row)) {
449      return false;
450    }
451    Row other = (Row) obj;
452    // TODO: This is wrong. Can't have two gets the same just because on same row.
453    return Row.COMPARATOR.compare(this, other) == 0;
454  }
455
456  @Override
457  public Get setAttribute(String name, byte[] value) {
458    return (Get) super.setAttribute(name, value);
459  }
460
461  @Override
462  public Get setId(String id) {
463    return (Get) super.setId(id);
464  }
465
466  @Override
467  public Get setAuthorizations(Authorizations authorizations) {
468    return (Get) super.setAuthorizations(authorizations);
469  }
470
471  @Override
472  public Get setACL(Map<String, Permission> perms) {
473    return (Get) super.setACL(perms);
474  }
475
476  @Override
477  public Get setACL(String user, Permission perms) {
478    return (Get) super.setACL(user, perms);
479  }
480
481  @Override
482  public Get setConsistency(Consistency consistency) {
483    return (Get) super.setConsistency(consistency);
484  }
485
486  @Override
487  public Get setReplicaId(int Id) {
488    return (Get) super.setReplicaId(Id);
489  }
490
491  @Override
492  public Get setIsolationLevel(IsolationLevel level) {
493    return (Get) super.setIsolationLevel(level);
494  }
495
496  @Override
497  public Get setPriority(int priority) {
498    return (Get) super.setPriority(priority);
499  }
500}