HBASE-847 new API: HTable.getRow with numVersion specified
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@722707 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
78165487d9
commit
914e6de8de
|
@ -151,6 +151,8 @@ Release 0.19.0 - Unreleased
|
|||
HBASE-1018 Regionservers should report detailed health to master
|
||||
HBASE-1034 Remove useless TestToString unit test
|
||||
HBASE-1030 Bit of polish on HBASE-1018
|
||||
HBASE-847 new API: HTable.getRow with numVersion specified
|
||||
(Doğacan Güney via Stack)
|
||||
|
||||
NEW FEATURES
|
||||
HBASE-875 Use MurmurHash instead of JenkinsHash [in bloomfilters]
|
||||
|
|
|
@ -423,6 +423,33 @@ public class HTable {
|
|||
public RowResult getRow(final byte [] row) throws IOException {
|
||||
return getRow(row, HConstants.LATEST_TIMESTAMP);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get more than one version of all columns for the specified row
|
||||
*
|
||||
* @param row row key
|
||||
* @param numVersions number of versions to return
|
||||
* @return RowResult is empty if row does not exist.
|
||||
* @throws IOException
|
||||
*/
|
||||
public RowResult getRow(final String row, final int numVersions)
|
||||
throws IOException {
|
||||
return getRow(Bytes.toBytes(row), null,
|
||||
HConstants.LATEST_TIMESTAMP, numVersions, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get more than one version of all columns for the specified row
|
||||
*
|
||||
* @param row row key
|
||||
* @param numVersions number of versions to return
|
||||
* @return RowResult is empty if row does not exist.
|
||||
* @throws IOException
|
||||
*/
|
||||
public RowResult getRow(final byte[] row, final int numVersions)
|
||||
throws IOException {
|
||||
return getRow(row, null, HConstants.LATEST_TIMESTAMP, numVersions, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all the data for the specified row at a specified timestamp
|
||||
|
@ -449,6 +476,26 @@ public class HTable {
|
|||
throws IOException {
|
||||
return getRow(row,null,ts);
|
||||
}
|
||||
|
||||
public RowResult getRow(final String row, final long ts,
|
||||
final int numVersions) throws IOException {
|
||||
return getRow(Bytes.toBytes(row), null, ts, numVersions, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get more than one version of all columns for the specified row
|
||||
* at a specified timestamp
|
||||
*
|
||||
* @param row row key
|
||||
* @param ts timestamp
|
||||
* @param numVersions number of versions to return
|
||||
* @return RowResult is empty if row does not exist.
|
||||
* @throws IOException
|
||||
*/
|
||||
public RowResult getRow(final byte[] row, final long timestamp,
|
||||
final int numVersions) throws IOException {
|
||||
return getRow(row, null, timestamp, numVersions, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get selected columns for the specified row at the latest timestamp
|
||||
|
@ -475,6 +522,35 @@ public class HTable {
|
|||
throws IOException {
|
||||
return getRow(row, columns, HConstants.LATEST_TIMESTAMP);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get more than one version of selected columns for the specified row
|
||||
*
|
||||
* @param row row key
|
||||
* @param columns Array of column names and families you want to retrieve.
|
||||
* @param numVersions number of versions to return
|
||||
* @return RowResult is empty if row does not exist.
|
||||
* @throws IOException
|
||||
*/
|
||||
public RowResult getRow(final String row, final String[] columns,
|
||||
final int numVersions) throws IOException {
|
||||
return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns),
|
||||
HConstants.LATEST_TIMESTAMP, numVersions, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get more than one version of selected columns for the specified row
|
||||
*
|
||||
* @param row row key
|
||||
* @param columns Array of column names and families you want to retrieve.
|
||||
* @param numVersions number of versions to return
|
||||
* @return RowResult is empty if row does not exist.
|
||||
* @throws IOException
|
||||
*/
|
||||
public RowResult getRow(final byte[] row, final byte[][] columns,
|
||||
final int numVersions) throws IOException {
|
||||
return getRow(row, columns, HConstants.LATEST_TIMESTAMP, numVersions, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get selected columns for the specified row at a specified timestamp
|
||||
|
@ -503,8 +579,16 @@ public class HTable {
|
|||
public RowResult getRow(final byte [] row, final byte [][] columns,
|
||||
final long ts)
|
||||
throws IOException {
|
||||
return getRow(row,columns,ts,null);
|
||||
return getRow(row,columns,ts,1,null);
|
||||
}
|
||||
|
||||
public RowResult getRow(final String row, final String[] columns,
|
||||
final long timestamp, final int numVersions, final RowLock rowLock)
|
||||
throws IOException {
|
||||
return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns), timestamp,
|
||||
numVersions, rowLock);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get selected columns for the specified row at a specified timestamp
|
||||
|
@ -518,7 +602,7 @@ public class HTable {
|
|||
* @throws IOException
|
||||
*/
|
||||
public RowResult getRow(final byte [] row, final byte [][] columns,
|
||||
final long ts, final RowLock rl)
|
||||
final long ts, final int numVersions, final RowLock rl)
|
||||
throws IOException {
|
||||
return connection.getRegionServerWithRetries(
|
||||
new ServerCallable<RowResult>(connection, tableName, row) {
|
||||
|
@ -528,7 +612,7 @@ public class HTable {
|
|||
lockId = rl.getLockId();
|
||||
}
|
||||
return server.getRow(location.getRegionInfo().getRegionName(), row,
|
||||
columns, ts, lockId);
|
||||
columns, ts, numVersions, lockId);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
|
|
@ -22,7 +22,12 @@ package org.apache.hadoop.hbase.io;
|
|||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
@ -33,14 +38,15 @@ import org.apache.hadoop.io.Writable;
|
|||
* the timestamp of a cell to a first-class value, making it easy to take
|
||||
* note of temporal data. Cell is used all the way from HStore up to HTable.
|
||||
*/
|
||||
public class Cell implements Writable, Iterable<Cell> {
|
||||
protected byte[][] values;
|
||||
protected long[] timestamps;
|
||||
public class Cell implements Writable, Iterable<Map.Entry<Long, byte[]>> {
|
||||
protected final SortedMap<Long, byte[]> valueMap =
|
||||
new TreeMap<Long, byte[]>(new Comparator<Long>() {
|
||||
public int compare(Long l1, Long l2) {
|
||||
return l2.compareTo(l1);
|
||||
}});
|
||||
|
||||
/** For Writable compatibility */
|
||||
public Cell() {
|
||||
values = null;
|
||||
timestamps = null;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -58,10 +64,7 @@ public class Cell implements Writable, Iterable<Cell> {
|
|||
* @param timestamp
|
||||
*/
|
||||
public Cell(byte[] value, long timestamp) {
|
||||
this.values = new byte[1][];
|
||||
this.values[0] = value;
|
||||
this.timestamps = new long[1];
|
||||
this.timestamps[0] = timestamp;
|
||||
valueMap.put(timestamp, value);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -69,16 +72,7 @@ public class Cell implements Writable, Iterable<Cell> {
|
|||
* @param ts array of timestamps
|
||||
*/
|
||||
public Cell(String[] vals, long[] ts) {
|
||||
if (vals.length != ts.length) {
|
||||
throw new IllegalArgumentException(
|
||||
"number of values must be the same as the number of timestamps");
|
||||
}
|
||||
this.values = new byte[vals.length][];
|
||||
this.timestamps = new long[ts.length];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
this.values[i] = Bytes.toBytes(vals[i]);
|
||||
this.timestamps[i] = ts[i];
|
||||
}
|
||||
this(Bytes.toByteArrays(vals), ts);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -90,38 +84,59 @@ public class Cell implements Writable, Iterable<Cell> {
|
|||
throw new IllegalArgumentException(
|
||||
"number of values must be the same as the number of timestamps");
|
||||
}
|
||||
this.values = new byte[vals.length][];
|
||||
this.timestamps = new long[ts.length];
|
||||
System.arraycopy(vals, 0, this.values, 0, vals.length);
|
||||
System.arraycopy(ts, 0, this.timestamps, 0, ts.length);
|
||||
for (int i = 0; i < vals.length; i++) {
|
||||
valueMap.put(ts[i], vals[i]);
|
||||
}
|
||||
}
|
||||
|
||||
/** @return the current cell's value */
|
||||
public byte[] getValue() {
|
||||
return values[0];
|
||||
return valueMap.get(valueMap.firstKey());
|
||||
}
|
||||
|
||||
/** @return the current cell's timestamp */
|
||||
public long getTimestamp() {
|
||||
return timestamps[0];
|
||||
return valueMap.firstKey();
|
||||
}
|
||||
|
||||
/** @return the number of values this cell holds */
|
||||
public int getNumValues() {
|
||||
return valueMap.size();
|
||||
}
|
||||
|
||||
/** Add values and timestamps of another cell into this cell
|
||||
* @param c Cell
|
||||
*/
|
||||
public void mergeCell(Cell c) {
|
||||
valueMap.putAll(c.valueMap);
|
||||
}
|
||||
|
||||
/** Add a new timestamp and value to this cell
|
||||
* @param val value
|
||||
* @param ts timestamp
|
||||
*/
|
||||
public void add(byte[] val, long ts) {
|
||||
valueMap.put(ts, val);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
if (this.values.length == 1) {
|
||||
return "timestamp=" + this.timestamps[0] + ", value=" +
|
||||
Bytes.toString(this.values[0]);
|
||||
if (valueMap.size() == 1) {
|
||||
return "timestamp=" + getTimestamp() + ", value=" +
|
||||
Bytes.toString(getValue());
|
||||
}
|
||||
StringBuilder s = new StringBuilder("{ ");
|
||||
for (int i = 0; i < this.values.length; i++) {
|
||||
int i = 0;
|
||||
for (Map.Entry<Long, byte[]> entry : valueMap.entrySet()) {
|
||||
if (i > 0) {
|
||||
s.append(", ");
|
||||
}
|
||||
s.append("[timestamp=");
|
||||
s.append(timestamps[i]);
|
||||
s.append(entry.getKey());
|
||||
s.append(", value=");
|
||||
s.append(Bytes.toString(values[i]));
|
||||
s.append(Bytes.toString(entry.getValue()));
|
||||
s.append("]");
|
||||
i++;
|
||||
}
|
||||
s.append(" }");
|
||||
return s.toString();
|
||||
|
@ -133,23 +148,18 @@ public class Cell implements Writable, Iterable<Cell> {
|
|||
|
||||
public void readFields(final DataInput in) throws IOException {
|
||||
int nvalues = in.readInt();
|
||||
this.timestamps = new long[nvalues];
|
||||
this.values = new byte[nvalues][];
|
||||
for (int i = 0; i < nvalues; i++) {
|
||||
this.timestamps[i] = in.readLong();
|
||||
}
|
||||
for (int i = 0; i < nvalues; i++) {
|
||||
this.values[i] = Bytes.readByteArray(in);
|
||||
long timestamp = in.readLong();
|
||||
byte[] value = Bytes.readByteArray(in);
|
||||
valueMap.put(timestamp, value);
|
||||
}
|
||||
}
|
||||
|
||||
public void write(final DataOutput out) throws IOException {
|
||||
out.writeInt(this.values.length);
|
||||
for (int i = 0; i < this.timestamps.length; i++) {
|
||||
out.writeLong(this.timestamps[i]);
|
||||
}
|
||||
for (int i = 0; i < this.values.length; i++) {
|
||||
Bytes.writeByteArray(out, this.values[i]);
|
||||
out.writeInt(valueMap.size());
|
||||
for (Map.Entry<Long, byte[]> entry : valueMap.entrySet()) {
|
||||
out.writeLong(entry.getKey());
|
||||
Bytes.writeByteArray(out, entry.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -157,22 +167,22 @@ public class Cell implements Writable, Iterable<Cell> {
|
|||
// Iterable
|
||||
//
|
||||
|
||||
public Iterator<Cell> iterator() {
|
||||
public Iterator<Entry<Long, byte[]>> iterator() {
|
||||
return new CellIterator();
|
||||
}
|
||||
private class CellIterator implements Iterator<Cell> {
|
||||
private int currentValue = 0;
|
||||
|
||||
private class CellIterator implements Iterator<Entry<Long, byte[]>> {
|
||||
private Iterator<Entry<Long, byte[]>> it;
|
||||
CellIterator() {
|
||||
it = valueMap.entrySet().iterator();
|
||||
}
|
||||
|
||||
public boolean hasNext() {
|
||||
return currentValue < values.length;
|
||||
return it.hasNext();
|
||||
}
|
||||
|
||||
public Cell next() {
|
||||
Cell c = new Cell(values[currentValue], timestamps[currentValue]);
|
||||
currentValue++;
|
||||
return c;
|
||||
public Entry<Long, byte[]> next() {
|
||||
return it.next();
|
||||
}
|
||||
|
||||
public void remove() throws UnsupportedOperationException {
|
||||
|
|
|
@ -62,8 +62,8 @@ public interface HBaseRPCProtocolVersion extends VersionedProtocol {
|
|||
* <li>Version 10: initial version (had to be > all other RPC versions</li>
|
||||
* <li>Version 11: Changed getClosestRowBefore signature.</li>
|
||||
* <li>Version 12: HServerLoad extensions (HBASE-1018).</li>
|
||||
* <li>Version 13: HBASE-847</li>
|
||||
* </ul>
|
||||
*/
|
||||
public static final long versionID = 12L;
|
||||
|
||||
public static final long versionID = 13L;
|
||||
}
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.NotServingRegionException;
|
|||
*
|
||||
*/
|
||||
public interface HRegionInterface extends HBaseRPCProtocolVersion {
|
||||
|
||||
/**
|
||||
* Get metainfo about an HRegion
|
||||
*
|
||||
|
@ -85,12 +84,14 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion {
|
|||
* @param row row key
|
||||
* @param columns columns to get
|
||||
* @param ts time stamp
|
||||
* @param numVersions number of versions
|
||||
* @param lockId lock id
|
||||
* @return map of values
|
||||
* @throws IOException
|
||||
*/
|
||||
public RowResult getRow(final byte [] regionName, final byte [] row,
|
||||
final byte[][] columns, final long ts, final long lockId)
|
||||
final byte[][] columns, final long ts,
|
||||
final int numVersions, final long lockId)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/**
|
||||
* Copyright 2008 The Apache Software Foundation
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
|
@ -963,7 +963,7 @@ public class HRegion implements HConstants {
|
|||
|
||||
/**
|
||||
* Fetch all the columns for the indicated row at a specified timestamp.
|
||||
* Returns a TreeMap that maps column names to values.
|
||||
* Returns a HbaseMapWritable that maps column names to values.
|
||||
*
|
||||
* We should eventually use Bloom filters here, to reduce running time. If
|
||||
* the database has many column families and is very sparse, then we could be
|
||||
|
@ -974,12 +974,14 @@ public class HRegion implements HConstants {
|
|||
* @param row
|
||||
* @param columns Array of columns you'd like to retrieve. When null, get all.
|
||||
* @param ts
|
||||
* @param numVersions number of versions to retrieve
|
||||
* @param lockid
|
||||
* @return Map<columnName, Cell> values
|
||||
* @return HbaseMapWritable<columnName, Cell> values
|
||||
* @throws IOException
|
||||
*/
|
||||
public Map<byte [], Cell> getFull(final byte [] row,
|
||||
final Set<byte []> columns, final long ts, final Integer lockid)
|
||||
public HbaseMapWritable<byte [], Cell> getFull(final byte [] row,
|
||||
final Set<byte []> columns, final long ts,
|
||||
final int numVersions, final Integer lockid)
|
||||
throws IOException {
|
||||
// Check columns passed
|
||||
if (columns != null) {
|
||||
|
@ -991,8 +993,8 @@ public class HRegion implements HConstants {
|
|||
Integer lid = getLock(lockid,row);
|
||||
HashSet<HStore> storeSet = new HashSet<HStore>();
|
||||
try {
|
||||
TreeMap<byte [], Cell> result =
|
||||
new TreeMap<byte [], Cell>(Bytes.BYTES_COMPARATOR);
|
||||
HbaseMapWritable<byte [], Cell> result =
|
||||
new HbaseMapWritable<byte [], Cell>();
|
||||
// Get the concerned columns or all of them
|
||||
if (columns != null) {
|
||||
for (byte[] bs : columns) {
|
||||
|
@ -1012,14 +1014,14 @@ public class HRegion implements HConstants {
|
|||
for (byte[] bs : columns) {
|
||||
if (HStoreKey.getFamilyDelimiterIndex(bs) == (bs.length - 1)) {
|
||||
HStore store = stores.get(Bytes.mapKey(HStoreKey.getFamily(bs)));
|
||||
store.getFull(key, null, result);
|
||||
store.getFull(key, null, numVersions, result);
|
||||
storeSet.remove(store);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (HStore targetStore: storeSet) {
|
||||
targetStore.getFull(key, columns, result);
|
||||
targetStore.getFull(key, columns, numVersions, result);
|
||||
}
|
||||
|
||||
return result;
|
||||
|
@ -1080,7 +1082,7 @@ public class HRegion implements HConstants {
|
|||
HbaseMapWritable<byte [], Cell> cells =
|
||||
new HbaseMapWritable<byte [], Cell>();
|
||||
// This will get all results for this store.
|
||||
store.getFull(key, null, cells);
|
||||
store.getFull(key, null, 1, cells);
|
||||
return new RowResult(key.getRow(), cells);
|
||||
} finally {
|
||||
splitsAndClosesLock.readLock().unlock();
|
||||
|
|
|
@ -1287,7 +1287,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
}
|
||||
|
||||
public RowResult getRow(final byte [] regionName, final byte [] row,
|
||||
final byte [][] columns, final long ts, final long lockId)
|
||||
final byte [][] columns, final long ts,
|
||||
final int numVersions, final long lockId)
|
||||
throws IOException {
|
||||
checkOpen();
|
||||
requestCount.incrementAndGet();
|
||||
|
@ -1300,13 +1301,11 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
}
|
||||
|
||||
HRegion region = getRegion(regionName);
|
||||
Map<byte [], Cell> map = region.getFull(row, columnSet, ts,
|
||||
getLockFromId(lockId));
|
||||
if (map == null || map.isEmpty())
|
||||
return null;
|
||||
HbaseMapWritable<byte [], Cell> result =
|
||||
new HbaseMapWritable<byte [], Cell>();
|
||||
result.putAll(map);
|
||||
region.getFull(row, columnSet,
|
||||
ts, numVersions, getLockFromId(lockId));
|
||||
if (result == null || result.isEmpty())
|
||||
return null;
|
||||
return new RowResult(row, result);
|
||||
} catch (IOException e) {
|
||||
checkOOME(e);
|
||||
|
|
|
@ -1308,8 +1308,10 @@ public class HStore implements HConstants {
|
|||
* The returned object should map column names to Cells.
|
||||
*/
|
||||
void getFull(HStoreKey key, final Set<byte []> columns,
|
||||
Map<byte [], Cell> results)
|
||||
final int numVersions, Map<byte [], Cell> results)
|
||||
throws IOException {
|
||||
int versions = versionsToReturn(numVersions);
|
||||
|
||||
Map<byte [], Long> deletes =
|
||||
new TreeMap<byte [], Long>(Bytes.BYTES_COMPARATOR);
|
||||
|
||||
|
@ -1321,7 +1323,7 @@ public class HStore implements HConstants {
|
|||
this.lock.readLock().lock();
|
||||
|
||||
// get from the memcache first.
|
||||
memcache.getFull(key, columns, deletes, results);
|
||||
memcache.getFull(key, columns, versions, deletes, results);
|
||||
|
||||
try {
|
||||
MapFile.Reader[] maparray = getReaders();
|
||||
|
@ -1332,7 +1334,7 @@ public class HStore implements HConstants {
|
|||
|
||||
// synchronize on the map so that no one else iterates it at the same
|
||||
// time
|
||||
getFullFromMapFile(map, key, columns, deletes, results);
|
||||
getFullFromMapFile(map, key, columns, versions, deletes, results);
|
||||
}
|
||||
} finally {
|
||||
this.lock.readLock().unlock();
|
||||
|
@ -1340,7 +1342,8 @@ public class HStore implements HConstants {
|
|||
}
|
||||
|
||||
private void getFullFromMapFile(MapFile.Reader map, HStoreKey key,
|
||||
Set<byte []> columns, Map<byte [], Long> deletes, Map<byte [], Cell> results)
|
||||
Set<byte []> columns, int numVersions, Map<byte [], Long> deletes,
|
||||
Map<byte [], Cell> results)
|
||||
throws IOException {
|
||||
synchronized(map) {
|
||||
long now = System.currentTimeMillis();
|
||||
|
@ -1354,14 +1357,17 @@ public class HStore implements HConstants {
|
|||
if (readkey == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
do {
|
||||
byte [] readcol = readkey.getColumn();
|
||||
|
||||
// if we're looking for this column (or all of them), and there isn't
|
||||
// already a value for this column in the results map, and the key we
|
||||
// already a value for this column in the results map or there is a value
|
||||
// but we haven't collected enough versions yet, and the key we
|
||||
// just read matches, then we'll consider it
|
||||
if ((columns == null || columns.contains(readcol))
|
||||
&& !results.containsKey(readcol)
|
||||
&& (!results.containsKey(readcol)
|
||||
|| results.get(readcol).getNumValues() < numVersions)
|
||||
&& key.matchesWithoutColumn(readkey)) {
|
||||
// if the value of the cell we're looking at right now is a delete,
|
||||
// we need to treat it differently
|
||||
|
@ -1380,8 +1386,13 @@ public class HStore implements HConstants {
|
|||
if (!(deletes.containsKey(readcol) &&
|
||||
deletes.get(readcol).longValue() >= readkey.getTimestamp())) {
|
||||
if (!isExpired(readkey, ttl, now)) {
|
||||
results.put(readcol,
|
||||
new Cell(readval.get(), readkey.getTimestamp()));
|
||||
if (!results.containsKey(readcol)) {
|
||||
results.put(readcol,
|
||||
new Cell(readval.get(), readkey.getTimestamp()));
|
||||
} else {
|
||||
results.get(readcol).add(readval.get(),
|
||||
readkey.getTimestamp());
|
||||
}
|
||||
// need to reinstantiate the readval so we can reuse it,
|
||||
// otherwise next iteration will destroy our result
|
||||
readval = new ImmutableBytesWritable();
|
||||
|
|
|
@ -285,19 +285,20 @@ class Memcache {
|
|||
* row and timestamp, but not a column name.
|
||||
* @param key
|
||||
* @param columns Pass null for all columns else the wanted subset.
|
||||
* @param numVersions number of versions to retrieve
|
||||
* @param deletes Map to accumulate deletes found.
|
||||
* @param results Where to stick row results found.
|
||||
*/
|
||||
void getFull(HStoreKey key, Set<byte []> columns, Map<byte [], Long> deletes,
|
||||
Map<byte [], Cell> results) {
|
||||
void getFull(HStoreKey key, Set<byte []> columns, int numVersions,
|
||||
Map<byte [], Long> deletes, Map<byte [], Cell> results) {
|
||||
this.lock.readLock().lock();
|
||||
try {
|
||||
// The synchronizations here are because internalGet iterates
|
||||
synchronized (this.memcache) {
|
||||
internalGetFull(this.memcache, key, columns, deletes, results);
|
||||
internalGetFull(this.memcache, key, columns, numVersions, deletes, results);
|
||||
}
|
||||
synchronized (this.snapshot) {
|
||||
internalGetFull(this.snapshot, key, columns, deletes, results);
|
||||
internalGetFull(this.snapshot, key, columns, numVersions, deletes, results);
|
||||
}
|
||||
} finally {
|
||||
this.lock.readLock().unlock();
|
||||
|
@ -305,7 +306,7 @@ class Memcache {
|
|||
}
|
||||
|
||||
private void internalGetFull(SortedMap<HStoreKey, byte[]> map, HStoreKey key,
|
||||
Set<byte []> columns, Map<byte [], Long> deletes,
|
||||
Set<byte []> columns, int numVersions, Map<byte [], Long> deletes,
|
||||
Map<byte [], Cell> results) {
|
||||
if (map.isEmpty() || key == null) {
|
||||
return;
|
||||
|
@ -316,7 +317,8 @@ class Memcache {
|
|||
for (Map.Entry<HStoreKey, byte []> es: tailMap.entrySet()) {
|
||||
HStoreKey itKey = es.getKey();
|
||||
byte [] itCol = itKey.getColumn();
|
||||
if (results.get(itCol) == null && key.matchesWithoutColumn(itKey)) {
|
||||
Cell cell = results.get(itCol);
|
||||
if ((cell == null || cell.getNumValues() < numVersions) && key.matchesWithoutColumn(itKey)) {
|
||||
if (columns == null || columns.contains(itKey.getColumn())) {
|
||||
byte [] val = tailMap.get(itKey);
|
||||
if (HLogEdit.isDeleted(val)) {
|
||||
|
@ -329,7 +331,11 @@ class Memcache {
|
|||
// Skip expired cells
|
||||
if (ttl == HConstants.FOREVER ||
|
||||
now < itKey.getTimestamp() + ttl) {
|
||||
results.put(itCol, new Cell(val, itKey.getTimestamp()));
|
||||
if (cell == null) {
|
||||
results.put(itCol, new Cell(val, itKey.getTimestamp()));
|
||||
} else {
|
||||
cell.add(val, itKey.getTimestamp());
|
||||
}
|
||||
} else {
|
||||
addVictim(victims, itKey);
|
||||
}
|
||||
|
@ -787,7 +793,7 @@ class Memcache {
|
|||
}
|
||||
key.setRow(this.currentRow);
|
||||
key.setVersion(this.timestamp);
|
||||
getFull(key, isWildcardScanner() ? null : this.columns, deletes,
|
||||
getFull(key, isWildcardScanner() ? null : this.columns, 1, deletes,
|
||||
rowResults);
|
||||
for (Map.Entry<byte [], Long> e: deletes.entrySet()) {
|
||||
rowResults.put(e.getKey(),
|
||||
|
|
|
@ -110,7 +110,7 @@ class IndexedRegion extends TransactionalRegion {
|
|||
|
||||
SortedMap<byte[], byte[]> newColumnValues = getColumnsFromBatchUpdate(batchUpdate);
|
||||
Map<byte[], Cell> oldColumnCells = super.getFull(batchUpdate.getRow(),
|
||||
neededColumns, HConstants.LATEST_TIMESTAMP, null);
|
||||
neededColumns, HConstants.LATEST_TIMESTAMP, 1, null);
|
||||
|
||||
// Handle delete batch updates. Go back and get the next older values
|
||||
for (BatchOperation op : batchUpdate) {
|
||||
|
@ -270,7 +270,7 @@ class IndexedRegion extends TransactionalRegion {
|
|||
Set<byte[]> neededColumns = getColumnsForIndexes(getIndexes());
|
||||
|
||||
Map<byte[], Cell> oldColumnCells = super.getFull(row,
|
||||
neededColumns, HConstants.LATEST_TIMESTAMP, null);
|
||||
neededColumns, HConstants.LATEST_TIMESTAMP, 1, null);
|
||||
SortedMap<byte[], byte[]> oldColumnValues = convertToValueMap(oldColumnCells);
|
||||
|
||||
|
||||
|
@ -281,7 +281,7 @@ class IndexedRegion extends TransactionalRegion {
|
|||
// Handle if there is still a version visible.
|
||||
if (ts != HConstants.LATEST_TIMESTAMP) {
|
||||
Map<byte[], Cell> currentColumnCells = super.getFull(row,
|
||||
neededColumns, ts, null);
|
||||
neededColumns, ts, 1, null);
|
||||
SortedMap<byte[], byte[]> currentColumnValues = convertToValueMap(currentColumnCells);
|
||||
|
||||
for (IndexSpecification indexSpec : getIndexes()) {
|
||||
|
@ -316,7 +316,7 @@ class IndexedRegion extends TransactionalRegion {
|
|||
|
||||
Set<byte[]> neededColumns = getColumnsForIndexes(indexesToUpdate);
|
||||
Map<byte[], Cell> oldColumnCells = super.getFull(row,
|
||||
neededColumns, HConstants.LATEST_TIMESTAMP, null);
|
||||
neededColumns, HConstants.LATEST_TIMESTAMP, 1, null);
|
||||
SortedMap<byte [], byte[]> oldColumnValues = convertToValueMap(oldColumnCells);
|
||||
|
||||
for (IndexSpecification indexSpec : indexesToUpdate) {
|
||||
|
@ -326,7 +326,7 @@ class IndexedRegion extends TransactionalRegion {
|
|||
// Handle if there is still a version visible.
|
||||
if (ts != HConstants.LATEST_TIMESTAMP) {
|
||||
Map<byte[], Cell> currentColumnCells = super.getFull(row,
|
||||
neededColumns, ts, null);
|
||||
neededColumns, ts, 1, null);
|
||||
SortedMap<byte[], byte[]> currentColumnValues = convertToValueMap(currentColumnCells);
|
||||
|
||||
for (IndexSpecification indexSpec : getIndexes()) {
|
||||
|
|
|
@ -309,12 +309,12 @@ public class TransactionalRegion extends HRegion {
|
|||
LOG.trace("cell: " + Bytes.toString(entry.getValue().getValue()));
|
||||
}
|
||||
|
||||
Map<byte[], Cell> internalResults = getFull(row, columns, ts, null);
|
||||
Map<byte[], Cell> internalResults = getFull(row, columns, ts, 1, null);
|
||||
internalResults.putAll(localCells);
|
||||
return internalResults;
|
||||
}
|
||||
|
||||
return getFull(row, columns, ts, null);
|
||||
return getFull(row, columns, ts, 1, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -434,7 +434,7 @@ public abstract class HBaseTestCase extends TestCase {
|
|||
* @throws IOException
|
||||
*/
|
||||
public Map<byte [], Cell> getFull(byte [] row) throws IOException {
|
||||
return region.getFull(row, null, HConstants.LATEST_TIMESTAMP, null);
|
||||
return region.getFull(row, null, HConstants.LATEST_TIMESTAMP, 1, null);
|
||||
}
|
||||
|
||||
public void flushcache() throws IOException {
|
||||
|
@ -555,7 +555,7 @@ public abstract class HBaseTestCase extends TestCase {
|
|||
protected void assertCellEquals(final HRegion region, final byte [] row,
|
||||
final byte [] column, final long timestamp, final String value)
|
||||
throws IOException {
|
||||
Map<byte [], Cell> result = region.getFull(row, null, timestamp, null);
|
||||
Map<byte [], Cell> result = region.getFull(row, null, timestamp, 1, null);
|
||||
Cell cell_value = result.get(column);
|
||||
if(value == null){
|
||||
assertEquals(column.toString() + " at timestamp " + timestamp, null, cell_value);
|
||||
|
|
|
@ -553,25 +553,25 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
|
|||
|
||||
// make sure we get all of them with standard getFull
|
||||
Map<byte [], Cell> result = region.getFull(row, null,
|
||||
HConstants.LATEST_TIMESTAMP, null);
|
||||
HConstants.LATEST_TIMESTAMP, 1, null);
|
||||
assertEquals(new String(result.get(COLUMNS[0]).getValue()), "column 0");
|
||||
assertEquals(new String(result.get(COLUMNS[1]).getValue()), "column 1");
|
||||
assertEquals(new String(result.get(COLUMNS[2]).getValue()), "column 2");
|
||||
|
||||
// try to get just one
|
||||
result = region.getFull(row, one, HConstants.LATEST_TIMESTAMP, null);
|
||||
result = region.getFull(row, one, HConstants.LATEST_TIMESTAMP, 1, null);
|
||||
assertEquals(new String(result.get(COLUMNS[0]).getValue()), "column 0");
|
||||
assertNull(result.get(COLUMNS[1]));
|
||||
assertNull(result.get(COLUMNS[2]));
|
||||
|
||||
// try to get all of them (specified)
|
||||
result = region.getFull(row, all, HConstants.LATEST_TIMESTAMP, null);
|
||||
result = region.getFull(row, all, HConstants.LATEST_TIMESTAMP, 1, null);
|
||||
assertEquals(new String(result.get(COLUMNS[0]).getValue()), "column 0");
|
||||
assertEquals(new String(result.get(COLUMNS[1]).getValue()), "column 1");
|
||||
assertEquals(new String(result.get(COLUMNS[2]).getValue()), "column 2");
|
||||
|
||||
// try to get none with empty column set
|
||||
result = region.getFull(row, none, HConstants.LATEST_TIMESTAMP, null);
|
||||
result = region.getFull(row, none, HConstants.LATEST_TIMESTAMP, 1, null);
|
||||
assertNull(result.get(COLUMNS[0]));
|
||||
assertNull(result.get(COLUMNS[1]));
|
||||
assertNull(result.get(COLUMNS[2]));
|
||||
|
@ -602,7 +602,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
|
|||
region.flushcache();
|
||||
|
||||
// assert that getFull gives us the older value
|
||||
results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP, null);
|
||||
results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
|
||||
assertEquals("olderValue", new String(results.get(COLUMNS[0]).getValue()));
|
||||
|
||||
// write a new value for the cell
|
||||
|
@ -614,7 +614,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
|
|||
region.flushcache();
|
||||
|
||||
// assert that getFull gives us the later value
|
||||
results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP, null);
|
||||
results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
|
||||
assertEquals("newerValue", new String(results.get(COLUMNS[0]).getValue()));
|
||||
|
||||
//
|
||||
|
@ -635,7 +635,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
|
|||
region.flushcache();
|
||||
|
||||
// assert i get both columns
|
||||
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, null);
|
||||
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
|
||||
assertEquals("Should have two columns in the results map", 2, results.size());
|
||||
assertEquals("column0 value", new String(results.get(cell1).getValue()));
|
||||
assertEquals("column1 value", new String(results.get(cell2).getValue()));
|
||||
|
@ -650,7 +650,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
|
|||
region.flushcache();
|
||||
|
||||
// assert i get the second column only
|
||||
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, null);
|
||||
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
|
||||
assertEquals("Should have one column in the results map", 1, results.size());
|
||||
assertNull("column0 value", results.get(cell1));
|
||||
assertEquals("column1 new value", new String(results.get(cell2).getValue()));
|
||||
|
@ -664,7 +664,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
|
|||
region.batchUpdate(batchUpdate, null);
|
||||
|
||||
// assert i get the third column only
|
||||
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, null);
|
||||
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
|
||||
assertEquals("Should have one column in the results map", 1, results.size());
|
||||
assertNull("column0 value", results.get(cell1));
|
||||
assertNull("column1 value", results.get(cell2));
|
||||
|
@ -685,7 +685,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
|
|||
private void assertColumnsPresent(final HRegion r, final byte [] row)
|
||||
throws IOException {
|
||||
Map<byte [], Cell> result =
|
||||
r.getFull(row, null, HConstants.LATEST_TIMESTAMP, null);
|
||||
r.getFull(row, null, HConstants.LATEST_TIMESTAMP, 1, null);
|
||||
int columnCount = 0;
|
||||
for (Map.Entry<byte [], Cell> e: result.entrySet()) {
|
||||
columnCount++;
|
||||
|
|
|
@ -188,7 +188,7 @@ public class TestHMemcache extends TestCase {
|
|||
new TreeMap<byte [], Cell>(Bytes.BYTES_COMPARATOR);
|
||||
TreeMap<byte [], Long> deletes =
|
||||
new TreeMap<byte [], Long>(Bytes.BYTES_COMPARATOR);
|
||||
this.hmemcache.getFull(hsk, null, deletes, all);
|
||||
this.hmemcache.getFull(hsk, null, 1, deletes, all);
|
||||
isExpectedRow(i, all);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue