HBASE-21918 the use cases in doc should use Cell instead of KeyValue

Signed-off-by: Xu Cang <xcang@salesforce.com>
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
Zheng Wang 2019-02-20 00:13:18 +08:00 committed by stack
parent 9e273b3f3e
commit e46dc61569
3 changed files with 9 additions and 9 deletions

View File

@ -452,8 +452,8 @@ scan.setFilter(f);
scan.setBatch(10); // set this if there could be many columns returned
ResultScanner rs = t.getScanner(scan);
for (Result r = rs.next(); r != null; r = rs.next()) {
for (KeyValue kv : r.raw()) {
// each kv represents a column
for (Cell cell : result.listCells()) {
// each cell represents a column
}
}
rs.close();
@ -482,8 +482,8 @@ scan.setFilter(f);
scan.setBatch(10); // set this if there could be many columns returned
ResultScanner rs = t.getScanner(scan);
for (Result r = rs.next(); r != null; r = rs.next()) {
for (KeyValue kv : r.raw()) {
// each kv represents a column
for (Cell cell : result.listCells()) {
// each cell represents a column
}
}
rs.close();
@ -518,8 +518,8 @@ scan.setFilter(f);
scan.setBatch(10); // set this if there could be many columns returned
ResultScanner rs = t.getScanner(scan);
for (Result r = rs.next(); r != null; r = rs.next()) {
for (KeyValue kv : r.raw()) {
// each kv represents a column
for (Cell cell : result.listCells()) {
// each cell represents a column
}
}
rs.close();

View File

@ -425,7 +425,7 @@ Get get = new Get(Bytes.toBytes("row1"));
get.setMaxVersions(3); // will return last 3 versions of row
Result r = table.get(get);
byte[] b = r.getValue(CF, ATTR); // returns current version of value
List<KeyValue> kv = r.getColumn(CF, ATTR); // returns all versions of this column
List<Cell> cells = r.getColumnCells(CF, ATTR); // returns all versions of this column
----
==== Put

View File

@ -417,8 +417,8 @@ public static class MyMapper extends TableMapper<ImmutableBytesWritable, Put> {
private static Put resultToPut(ImmutableBytesWritable key, Result result) throws IOException {
Put put = new Put(key.get());
for (KeyValue kv : result.raw()) {
put.add(kv);
for (Cell cell : result.listCells()) {
put.add(cell);
}
return put;
}