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:
Michael Stack 2008-12-03 01:06:05 +00:00
parent 78165487d9
commit 914e6de8de
14 changed files with 226 additions and 111 deletions

View File

@ -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]

View File

@ -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);
}
}
);

View File

@ -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 {

View File

@ -62,8 +62,8 @@ public interface HBaseRPCProtocolVersion extends VersionedProtocol {
* <li>Version 10: initial version (had to be &gt 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;
}

View File

@ -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;
/**

View File

@ -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();

View File

@ -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);

View File

@ -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();

View File

@ -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(),

View File

@ -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()) {

View File

@ -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);
}
/**

View File

@ -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);

View File

@ -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++;

View File

@ -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);
}
}