HBASE-4436 Remove trivial 0.90 deprecated code from 0.92 and trunk.

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1190675 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-10-28 23:40:32 +00:00
parent 8c6f8b4b96
commit 9586812380
21 changed files with 67 additions and 343 deletions

View File

@ -721,6 +721,8 @@ Release 0.92.0 - Unreleased
HBASE-4656 Note how dfs.support.append has to be enabled in 0.20.205.0
clusters
HBASE-4699 Cleanup the UIs
HBASE-4552 Remove trivial 0.90 deprecated code from 0.92 and trunk.
(Jonathan Hsieh)
NEW FEATURES
HBASE-2001 Coprocessors: Colocate user code with regions (Mingjie Lai via

View File

@ -55,22 +55,6 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
// Version 8 -- reintroduction of bloom filters, changed from boolean to enum
private static final byte COLUMN_DESCRIPTOR_VERSION = (byte)8;
/**
* The type of compression.
* @see org.apache.hadoop.io.SequenceFile.Writer
* @deprecated Compression now means which compression library
* rather than 'what' to compress.
*/
@Deprecated
public static enum CompressionType {
/** Do not compress records. */
NONE,
/** Compress values only, each separately. */
RECORD,
/** Compress sequences of records together in blocks. */
BLOCK
}
public static final String COMPRESSION = "COMPRESSION";
public static final String COMPRESSION_COMPACT = "COMPRESSION_COMPACT";
public static final String BLOCKCACHE = "BLOCKCACHE";

View File

@ -1736,20 +1736,6 @@ public class KeyValue implements Writable, HeapSize {
return new KeyValue(row, null, null, ts, Type.Maximum);
}
/**
* @param row - row key (arbitrary byte array)
* @param c column - {@link #parseColumn(byte[])} is called to split
* the column.
* @param ts - timestamp
* @return First possible key on passed <code>row</code>, column and timestamp
* @deprecated
*/
public static KeyValue createFirstOnRow(final byte [] row, final byte [] c,
final long ts) {
byte [][] split = parseColumn(c);
return new KeyValue(row, split[0], split[1], ts, Type.Maximum);
}
/**
* Create a KeyValue for the specified row, family and qualifier that would be
* smaller than all other possible KeyValues that have the same row,family,qualifier.

View File

@ -280,32 +280,4 @@ public class Delete extends Mutation
}
writeAttributes(out);
}
/**
* Delete all versions of the specified column, given in
* <code>family:qualifier</code> notation, and with a timestamp less than
* or equal to the specified timestamp.
* @param column colon-delimited family and qualifier
* @param timestamp maximum version timestamp
* @deprecated use {@link #deleteColumn(byte[], byte[], long)} instead
* @return this for invocation chaining
*/
public Delete deleteColumns(byte [] column, long timestamp) {
byte [][] parts = KeyValue.parseColumn(column);
this.deleteColumns(parts[0], parts[1], timestamp);
return this;
}
/**
* Delete the latest version of the specified column, given in
* <code>family:qualifier</code> notation.
* @param column colon-delimited family and qualifier
* @deprecated use {@link #deleteColumn(byte[], byte[])} instead
* @return this for invocation chaining
*/
public Delete deleteColumn(byte [] column) {
byte [][] parts = KeyValue.parseColumn(column);
this.deleteColumn(parts[0], parts[1], HConstants.LATEST_TIMESTAMP);
return this;
}
}

View File

@ -457,41 +457,4 @@ public class Get extends OperationWithAttributes
throw new RuntimeException("Can't find class " + className);
}
}
/**
* Adds an array of columns specified the old format, family:qualifier.
* <p>
* Overrides previous calls to addFamily for any families in the input.
* @param columns array of columns, formatted as <pre>family:qualifier</pre>
* @deprecated issue multiple {@link #addColumn(byte[], byte[])} instead
* @return this for invocation chaining
*/
@SuppressWarnings({"deprecation"})
public Get addColumns(byte [][] columns) {
if (columns == null) return this;
for (byte[] column : columns) {
try {
addColumn(column);
} catch (Exception ignored) {
}
}
return this;
}
/**
*
* @param column Old format column.
* @return This.
* @deprecated use {@link #addColumn(byte[], byte[])} instead
*/
public Get addColumn(final byte [] column) {
if (column == null) return this;
byte [][] split = KeyValue.parseColumn(column);
if (split.length > 1 && split[1] != null && split[1].length > 0) {
addColumn(split[0], split[1]);
} else {
addFamily(split[0]);
}
return this;
}
}

View File

@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.NotServingRegionException;
@ -951,22 +950,6 @@ public class HBaseAdmin implements Abortable, Closeable {
}
}
/**
* Modify an existing column family on a table.
* Asynchronous operation.
*
* @param tableName name of table
* @param columnName name of column to be modified
* @param descriptor new column descriptor to use
* @throws IOException if a remote or network exception occurs
* @deprecated The <code>columnName</code> is redundant. Use {@link #addColumn(String, HColumnDescriptor)}
*/
public void modifyColumn(final String tableName, final String columnName,
HColumnDescriptor descriptor)
throws IOException {
modifyColumn(tableName, descriptor);
}
/**
* Modify an existing column family on a table.
* Asynchronous operation.
@ -980,22 +963,6 @@ public class HBaseAdmin implements Abortable, Closeable {
modifyColumn(Bytes.toBytes(tableName), descriptor);
}
/**
* Modify an existing column family on a table.
* Asynchronous operation.
*
* @param tableName name of table
* @param columnName name of column to be modified
* @param descriptor new column descriptor to use
* @throws IOException if a remote or network exception occurs
* @deprecated The <code>columnName</code> is redundant. Use {@link #modifyColumn(byte[], HColumnDescriptor)}
*/
public void modifyColumn(final byte [] tableName, final byte [] columnName,
HColumnDescriptor descriptor)
throws IOException {
modifyColumn(tableName, descriptor);
}
/**
* Modify an existing column family on a table.
* Asynchronous operation.
@ -1308,24 +1275,6 @@ public class HBaseAdmin implements Abortable, Closeable {
getMaster().move(encodedRegionName, destServerName);
}
/**
* Tries to assign a region. Region could be reassigned to the same server.
*
* @param regionName
* Region name to assign.
* @param force
* True to force assign.
* @throws MasterNotRunningException
* @throws ZooKeeperConnectionException
* @throws IOException
* @deprecated The <code>force</code> is unused.Use {@link #assign(byte[])}
*/
public void assign(final byte[] regionName, final boolean force)
throws MasterNotRunningException, ZooKeeperConnectionException,
IOException {
getMaster().assign(regionName, force);
}
/**
* @param regionName
* Region name to assign.
@ -1524,18 +1473,6 @@ public class HBaseAdmin implements Abortable, Closeable {
}
}
/**
* Stop the designated regionserver.
* @throws IOException if a remote or network exception occurs
* @deprecated Use {@link #stopRegionServer(String)}
*/
public synchronized void stopRegionServer(final HServerAddress hsa)
throws IOException {
HRegionInterface rs =
this.connection.getHRegionConnection(hsa);
rs.stop("Called by admin client " + this.connection.toString());
}
/**
* Stop the designated regionserver
* @param hostnamePort Hostname and port delimited by a <code>:</code> as in

View File

@ -283,7 +283,7 @@ public interface HConnection extends Abortable, Closeable {
* @throws IOException if there are problems talking to META. Per-item
* exceptions are stored in the results array.
*/
public void processBatch(List<Row> actions, final byte[] tableName,
public void processBatch(List<? extends Row> actions, final byte[] tableName,
ExecutorService pool, Object[] results)
throws IOException, InterruptedException;
@ -327,20 +327,6 @@ public interface HConnection extends Abortable, Closeable {
final Batch.Call<T,R> call,
final Batch.Callback<R> callback) throws IOException, Throwable;
/**
* Process a batch of Puts.
*
* @param list The collection of actions. The list is mutated: all successful Puts
* are removed from the list.
* @param tableName Name of the hbase table
* @param pool Thread pool for parallel execution
* @throws IOException
* @deprecated Use HConnectionManager::processBatch instead.
*/
public void processBatchOfPuts(List<Put> list,
final byte[] tableName, ExecutorService pool)
throws IOException;
/**
* Enable or disable region cache prefetch for the table. It will be
* applied for the given table's all HTable instances within this

View File

@ -1301,7 +1301,7 @@ public class HConnectionManager {
};
}
public void processBatch(List<Row> list,
public void processBatch(List<? extends Row> list,
final byte[] tableName,
ExecutorService pool,
Object[] results) throws IOException, InterruptedException {
@ -1549,31 +1549,6 @@ public class HConnectionManager {
}
}
/**
* @deprecated Use HConnectionManager::processBatch instead.
*/
public void processBatchOfPuts(List<Put> list,
final byte[] tableName,
ExecutorService pool) throws IOException {
Object[] results = new Object[list.size()];
try {
processBatch((List) list, tableName, pool, results);
} catch (InterruptedException e) {
throw new IOException(e);
} finally {
// mutate list so that it is empty for complete success, or contains only failed records
// results are returned in the same order as the requests in list
// walk the list backwards, so we can remove from list without impacting the indexes of earlier members
for (int i = results.length - 1; i>=0; i--) {
if (results[i] instanceof Result) {
// successful Puts are removed from the list here.
list.remove(i);
}
}
}
}
private Throwable translateException(Throwable t) throws IOException {
if (t instanceof UndeclaredThrowableException) {
t = t.getCause();

View File

@ -873,7 +873,23 @@ public class HTable implements HTableInterface, Closeable {
@Override
public void flushCommits() throws IOException {
try {
this.connection.processBatchOfPuts(writeBuffer, tableName, pool);
Object[] results = new Object[writeBuffer.size()];
try {
this.connection.processBatch(writeBuffer, tableName, pool, results);
} catch (InterruptedException e) {
throw new IOException(e);
} finally {
// mutate list so that it is empty for complete success, or contains
// only failed records results are returned in the same order as the
// requests in list walk the list backwards, so we can remove from list
// without impacting the indexes of earlier members
for (int i = results.length - 1; i>=0; i--) {
if (results[i] instanceof Result) {
// successful Puts are removed from the list here.
writeBuffer.remove(i);
}
}
}
} finally {
if (clearBufferOnFail) {
writeBuffer.clear();

View File

@ -406,19 +406,4 @@ public class Put extends Mutation
}
writeAttributes(out);
}
/**
* Add the specified column and value, with the specified timestamp as
* its version to this Put operation.
* @param column Old style column name with family and qualifier put together
* with a colon.
* @param ts version timestamp
* @param value column value
* @deprecated use {@link #add(byte[], byte[], long, byte[])} instead
* @return true
*/
public Put add(byte [] column, long ts, byte [] value) {
byte [][] parts = KeyValue.parseColumn(column);
return add(parts[0], parts[1], ts, value);
}
}

View File

@ -20,18 +20,6 @@
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.SplitKeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.WritableWithSize;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.Writable;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
@ -43,6 +31,13 @@ import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.SplitKeyValue;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.WritableWithSize;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Writable;
/**
* Single row result of a {@link Get} or {@link Scan} query.<p>
*
@ -168,19 +163,6 @@ public class Result implements Writable, WritableWithSize {
return isEmpty()? null: Arrays.asList(raw());
}
/**
* Returns a sorted array of KeyValues in this Result.
* <p>
* Since HBase 0.20.5 this is equivalent to {@link #raw}. Use
* {@link #raw} instead.
*
* @return sorted array of KeyValues
* @deprecated
*/
public KeyValue[] sorted() {
return raw(); // side effect of loading this.kvs
}
/**
* Return the KeyValues for the specific column. The KeyValues are sorted in
* the {@link KeyValue#COMPARATOR} order. That implies the first entry in
@ -399,32 +381,6 @@ public class Result implements Writable, WritableWithSize {
return returnMap;
}
private Map.Entry<Long,byte[]> getKeyValue(byte[] family, byte[] qualifier) {
if(this.familyMap == null) {
getMap();
}
if(isEmpty()) {
return null;
}
NavigableMap<byte [], NavigableMap<Long, byte[]>> qualifierMap =
familyMap.get(family);
if(qualifierMap == null) {
return null;
}
NavigableMap<Long, byte[]> versionMap =
getVersionMap(qualifierMap, qualifier);
if(versionMap == null) {
return null;
}
return versionMap.firstEntry();
}
private NavigableMap<Long, byte[]> getVersionMap(
NavigableMap<byte [], NavigableMap<Long, byte[]>> qualifierMap, byte [] qualifier) {
return qualifier != null?
qualifierMap.get(qualifier): qualifierMap.get(new byte[0]);
}
/**
* Returns the value of the first column in the Result.
* @return value of the first column
@ -675,8 +631,8 @@ public class Result implements Writable, WritableWithSize {
throw new Exception("This row doesn't have the same number of KVs: "
+ res1.toString() + " compared to " + res2.toString());
}
KeyValue[] ourKVs = res1.sorted();
KeyValue[] replicatedKVs = res2.sorted();
KeyValue[] ourKVs = res1.raw();
KeyValue[] replicatedKVs = res2.raw();
for (int i = 0; i < res1.size(); i++) {
if (!ourKVs[i].equals(replicatedKVs[i]) &&
!Bytes.equals(ourKVs[i].getValue(), replicatedKVs[i].getValue())) {

View File

@ -51,8 +51,6 @@ import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.MultiPut;
import org.apache.hadoop.hbase.client.MultiPutResponse;
import org.apache.hadoop.hbase.client.MultiAction;
import org.apache.hadoop.hbase.client.Action;
import org.apache.hadoop.hbase.client.MultiResponse;
@ -203,9 +201,6 @@ public class HbaseObjectWritable implements Writable, WritableWithSize, Configur
addToMap(Delete [].class, code++);
addToMap(MultiPut.class, code++);
addToMap(MultiPutResponse.class, code++);
addToMap(HLog.Entry.class, code++);
addToMap(HLog.Entry[].class, code++);
addToMap(HLogKey.class, code++);

View File

@ -33,8 +33,6 @@ import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.MultiAction;
import org.apache.hadoop.hbase.client.MultiPut;
import org.apache.hadoop.hbase.client.MultiPutResponse;
import org.apache.hadoop.hbase.client.MultiResponse;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -324,15 +322,6 @@ public interface HRegionInterface extends VersionedProtocol, Stoppable, Abortabl
*/
public <R> MultiResponse multi(MultiAction<R> multi) throws IOException;
/**
* Multi put for putting multiple regions worth of puts at once.
*
* @param puts the request
* @return the reply
* @throws IOException e
*/
public MultiPutResponse multiPut(MultiPut puts) throws IOException;
/**
* Bulk load an HFile into an open region
*/

View File

@ -88,8 +88,6 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.MultiAction;
import org.apache.hadoop.hbase.client.MultiPut;
import org.apache.hadoop.hbase.client.MultiPutResponse;
import org.apache.hadoop.hbase.client.MultiResponse;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -3116,25 +3114,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
return response;
}
/**
* @deprecated Use HRegionServer.multi( MultiAction action) instead
*/
@Override
public MultiPutResponse multiPut(MultiPut puts) throws IOException {
checkOpen();
MultiPutResponse resp = new MultiPutResponse();
// do each region as it's own.
for (Map.Entry<byte[], List<Put>> e : puts.puts.entrySet()) {
int result = put(e.getKey(), e.getValue());
resp.addResult(e.getKey(), result);
e.getValue().clear(); // clear some RAM
}
return resp;
}
/**
* Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}
* method using the registered protocol handlers.

View File

@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
@ -318,7 +317,7 @@ public class ThriftServer {
get.addColumn(family, qualifier);
}
Result result = table.get(get);
return ThriftUtilities.cellFromHBase(result.sorted());
return ThriftUtilities.cellFromHBase(result.raw());
} catch (IOException e) {
throw new IOError(e.getMessage());
}
@ -346,7 +345,7 @@ public class ThriftServer {
get.addColumn(family, qualifier);
get.setMaxVersions(numVersions);
Result result = table.get(get);
return ThriftUtilities.cellFromHBase(result.sorted());
return ThriftUtilities.cellFromHBase(result.raw());
} catch (IOException e) {
throw new IOError(e.getMessage());
}
@ -378,7 +377,7 @@ public class ThriftServer {
get.setTimeRange(Long.MIN_VALUE, timestamp);
get.setMaxVersions(numVersions);
Result result = table.get(get);
return ThriftUtilities.cellFromHBase(result.sorted());
return ThriftUtilities.cellFromHBase(result.raw());
} catch (IOException e) {
throw new IOError(e.getMessage());
}
@ -916,7 +915,7 @@ public class ThriftServer {
try {
HTable table = getTable(getBytes(tableName));
Result result = table.getRowOrBefore(getBytes(row), getBytes(family));
return ThriftUtilities.cellFromHBase(result.sorted());
return ThriftUtilities.cellFromHBase(result.raw());
} catch (IOException e) {
throw new IOError(e.getMessage());
}

View File

@ -135,7 +135,7 @@ public class ThriftUtilities {
TRowResult result = new TRowResult();
result.row = ByteBuffer.wrap(result_.getRow());
result.columns = new TreeMap<ByteBuffer, TCell>();
for(KeyValue kv : result_.sorted()) {
for(KeyValue kv : result_.raw()) {
result.columns.put(
ByteBuffer.wrap(KeyValue.makeColumn(kv.getFamily(),
kv.getQualifier())),

View File

@ -355,10 +355,10 @@ public class TestSerialization {
Result deResult = (Result)Writables.getWritable(rb, new Result());
assertTrue("results are not equivalent, first key mismatch",
result.sorted()[0].equals(deResult.sorted()[0]));
result.raw()[0].equals(deResult.raw()[0]));
assertTrue("results are not equivalent, second key mismatch",
result.sorted()[1].equals(deResult.sorted()[1]));
result.raw()[1].equals(deResult.raw()[1]));
// Test empty Result
Result r = new Result();
@ -394,9 +394,9 @@ public class TestSerialization {
// Call sorted() first
deResult = (Result)Writables.getWritable(rb, new Result());
assertTrue("results are not equivalent, first key mismatch",
result.sorted()[0].equals(deResult.sorted()[0]));
result.raw()[0].equals(deResult.raw()[0]));
assertTrue("results are not equivalent, second key mismatch",
result.sorted()[1].equals(deResult.sorted()[1]));
result.raw()[1].equals(deResult.raw()[1]));
// Call raw() first
deResult = (Result)Writables.getWritable(rb, new Result());
@ -443,8 +443,8 @@ public class TestSerialization {
assertTrue(results.length == deResults.length);
for(int i=0;i<results.length;i++) {
KeyValue [] keysA = results[i].sorted();
KeyValue [] keysB = deResults[i].sorted();
KeyValue [] keysA = results[i].raw();
KeyValue [] keysB = deResults[i].raw();
assertTrue(keysA.length == keysB.length);
for(int j=0;j<keysA.length;j++) {
assertTrue("Expected equivalent keys but found:\n" +

View File

@ -106,7 +106,7 @@ public class TimestampTestBase extends HBaseTestCase {
get.setMaxVersions(3);
Result result = incommon.get(get);
assertEquals(1, result.size());
long time = Bytes.toLong(result.sorted()[0].getValue());
long time = Bytes.toLong(result.raw()[0].getValue());
assertEquals(time, currentTime);
}
@ -135,7 +135,7 @@ public class TimestampTestBase extends HBaseTestCase {
get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
get.setMaxVersions(tss.length);
Result result = incommon.get(get);
KeyValue [] kvs = result.sorted();
KeyValue [] kvs = result.raw();
assertEquals(kvs.length, tss.length);
for(int i=0;i<kvs.length;i++) {
t = Bytes.toLong(kvs[i].getValue());
@ -151,7 +151,7 @@ public class TimestampTestBase extends HBaseTestCase {
get.setTimeRange(0, maxStamp);
get.setMaxVersions(kvs.length - 1);
result = incommon.get(get);
kvs = result.sorted();
kvs = result.raw();
assertEquals(kvs.length, tss.length - 1);
for(int i=1;i<kvs.length;i++) {
t = Bytes.toLong(kvs[i-1].getValue());

View File

@ -1939,15 +1939,15 @@ public class TestFromClientSide {
result = scanner.next();
assertTrue("Expected 1 key but received " + result.size(),
result.size() == 1);
assertTrue(Bytes.equals(result.sorted()[0].getRow(), ROWS[3]));
assertTrue(Bytes.equals(result.sorted()[0].getValue(), VALUES[0]));
assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[3]));
assertTrue(Bytes.equals(result.raw()[0].getValue(), VALUES[0]));
result = scanner.next();
assertTrue("Expected 2 keys but received " + result.size(),
result.size() == 2);
assertTrue(Bytes.equals(result.sorted()[0].getRow(), ROWS[4]));
assertTrue(Bytes.equals(result.sorted()[1].getRow(), ROWS[4]));
assertTrue(Bytes.equals(result.sorted()[0].getValue(), VALUES[1]));
assertTrue(Bytes.equals(result.sorted()[1].getValue(), VALUES[2]));
assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[4]));
assertTrue(Bytes.equals(result.raw()[1].getRow(), ROWS[4]));
assertTrue(Bytes.equals(result.raw()[0].getValue(), VALUES[1]));
assertTrue(Bytes.equals(result.raw()[1].getValue(), VALUES[2]));
scanner.close();
// Add test of bulk deleting.
@ -2073,7 +2073,7 @@ public class TestFromClientSide {
Get get = new Get(ROWS[numRows-1]);
Result result = ht.get(get);
assertNumKeys(result, numColsPerRow);
KeyValue [] keys = result.sorted();
KeyValue [] keys = result.raw();
for(int i=0;i<result.size();i++) {
assertKey(keys[i], ROWS[numRows-1], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
}
@ -2084,7 +2084,7 @@ public class TestFromClientSide {
int rowCount = 0;
while((result = scanner.next()) != null) {
assertNumKeys(result, numColsPerRow);
KeyValue [] kvs = result.sorted();
KeyValue [] kvs = result.raw();
for(int i=0;i<numColsPerRow;i++) {
assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
}
@ -2102,7 +2102,7 @@ public class TestFromClientSide {
get = new Get(ROWS[numRows-1]);
result = ht.get(get);
assertNumKeys(result, numColsPerRow);
keys = result.sorted();
keys = result.raw();
for(int i=0;i<result.size();i++) {
assertKey(keys[i], ROWS[numRows-1], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
}
@ -2113,7 +2113,7 @@ public class TestFromClientSide {
rowCount = 0;
while((result = scanner.next()) != null) {
assertNumKeys(result, numColsPerRow);
KeyValue [] kvs = result.sorted();
KeyValue [] kvs = result.raw();
for(int i=0;i<numColsPerRow;i++) {
assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
}
@ -2935,7 +2935,7 @@ public class TestFromClientSide {
assertTrue("Expected " + idxs.length + " keys but result contains "
+ result.size(), result.size() == idxs.length);
KeyValue [] keys = result.sorted();
KeyValue [] keys = result.raw();
for(int i=0;i<keys.length;i++) {
byte [] family = families[idxs[i][0]];
@ -2965,7 +2965,7 @@ public class TestFromClientSide {
int expectedResults = end - start + 1;
assertEquals(expectedResults, result.size());
KeyValue [] keys = result.sorted();
KeyValue [] keys = result.raw();
for (int i=0; i<keys.length; i++) {
byte [] value = values[end-i];
@ -2999,7 +2999,7 @@ public class TestFromClientSide {
equals(row, result.getRow()));
assertTrue("Expected two keys but result contains " + result.size(),
result.size() == 2);
KeyValue [] kv = result.sorted();
KeyValue [] kv = result.raw();
KeyValue kvA = kv[0];
assertTrue("(A) Expected family [" + Bytes.toString(familyA) + "] " +
"Got family [" + Bytes.toString(kvA.getFamily()) + "]",
@ -3030,7 +3030,7 @@ public class TestFromClientSide {
equals(row, result.getRow()));
assertTrue("Expected a single key but result contains " + result.size(),
result.size() == 1);
KeyValue kv = result.sorted()[0];
KeyValue kv = result.raw()[0];
assertTrue("Expected family [" + Bytes.toString(family) + "] " +
"Got family [" + Bytes.toString(kv.getFamily()) + "]",
equals(family, kv.getFamily()));
@ -3050,7 +3050,7 @@ public class TestFromClientSide {
equals(row, result.getRow()));
assertTrue("Expected a single key but result contains " + result.size(),
result.size() == 1);
KeyValue kv = result.sorted()[0];
KeyValue kv = result.raw()[0];
assertTrue("Expected family [" + Bytes.toString(family) + "] " +
"Got family [" + Bytes.toString(kv.getFamily()) + "]",
equals(family, kv.getFamily()));
@ -3589,7 +3589,7 @@ public class TestFromClientSide {
scan.addColumn(CONTENTS_FAMILY, null);
ResultScanner scanner = table.getScanner(scan);
for (Result r : scanner) {
for(KeyValue key : r.sorted()) {
for(KeyValue key : r.raw()) {
System.out.println(Bytes.toString(r.getRow()) + ": " + key.toString());
}
}
@ -3769,7 +3769,7 @@ public class TestFromClientSide {
int index = 0;
Result r = null;
while ((r = s.next()) != null) {
for(KeyValue key : r.sorted()) {
for(KeyValue key : r.raw()) {
times[index++] = key.getTimestamp();
}
}
@ -3803,7 +3803,7 @@ public class TestFromClientSide {
int index = 0;
Result r = null;
while ((r = s.next()) != null) {
for(KeyValue key : r.sorted()) {
for(KeyValue key : r.raw()) {
times[index++] = key.getTimestamp();
}
}
@ -3870,7 +3870,7 @@ public class TestFromClientSide {
try {
for (Result r : s) {
put = new Put(r.getRow());
for (KeyValue kv : r.sorted()) {
for (KeyValue kv : r.raw()) {
put.add(kv);
}
b.put(put);

View File

@ -2749,7 +2749,7 @@ public class TestHRegion extends HBaseTestCase {
// TODO this was removed, now what dangit?!
// search looking for the qualifier in question?
long timestamp = 0;
for (KeyValue kv : result.sorted()) {
for (KeyValue kv : result.raw()) {
if (Bytes.equals(kv.getFamily(), families[0])
&& Bytes.equals(kv.getQualifier(), qualifiers[0])) {
timestamp = kv.getTimestamp();
@ -3074,7 +3074,7 @@ public class TestHRegion extends HBaseTestCase {
get.addColumn(family, qf);
}
Result result = newReg.get(get, null);
KeyValue [] raw = result.sorted();
KeyValue [] raw = result.raw();
assertEquals(families.length, result.size());
for(int j=0; j<families.length; j++) {
assertEquals(0, Bytes.compareTo(row, raw[j].getRow()));

View File

@ -230,7 +230,7 @@ public class TestMergeTool extends HBaseTestCase {
get.addFamily(FAMILY);
Result result = merged.get(get, null);
assertEquals(1, result.size());
byte [] bytes = result.sorted()[0].getValue();
byte [] bytes = result.raw()[0].getValue();
assertNotNull(Bytes.toStringBinary(rows[i][j]), bytes);
assertTrue(Bytes.equals(bytes, rows[i][j]));
}
@ -249,7 +249,7 @@ public class TestMergeTool extends HBaseTestCase {
Get get = new Get(rows[i][j]);
get.addFamily(FAMILY);
Result result = regions[i].get(get, null);
byte [] bytes = result.sorted()[0].getValue();
byte [] bytes = result.raw()[0].getValue();
assertNotNull(bytes);
assertTrue(Bytes.equals(bytes, rows[i][j]));
}