HBASE-12047 Avoid usage of KeyValueUtil#ensureKeyValue in simple cases.

This commit is contained in:
anoopsjohn 2014-09-23 16:42:01 +05:30
parent 411991cf1d
commit 791a03b40f
15 changed files with 124 additions and 151 deletions

View File

@ -26,8 +26,8 @@ import java.util.UUID;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.security.visibility.CellVisibility; import org.apache.hadoop.hbase.security.visibility.CellVisibility;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -121,14 +121,13 @@ public class Append extends Mutation {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public Append add(final Cell cell) { public Append add(final Cell cell) {
// Presume it is KeyValue for now. // Presume it is KeyValue for now.
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); byte [] family = CellUtil.cloneFamily(cell);
byte [] family = kv.getFamily();
List<Cell> list = this.familyMap.get(family); List<Cell> list = this.familyMap.get(family);
if (list == null) { if (list == null) {
list = new ArrayList<Cell>(); list = new ArrayList<Cell>();
} }
// find where the new entry should be placed in the List // find where the new entry should be placed in the List
list.add(kv); list.add(cell);
this.familyMap.put(family, list); this.familyMap.put(family, list);
return this; return this;
} }

View File

@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
@ -1449,9 +1448,7 @@ public class HTable implements HTableInterface, RegionLocator {
if (maxKeyValueSize > 0) { if (maxKeyValueSize > 0) {
for (List<Cell> list : put.getFamilyCellMap().values()) { for (List<Cell> list : put.getFamilyCellMap().values()) {
for (Cell cell : list) { for (Cell cell : list) {
// KeyValue v1 expectation. Cast for now. if (KeyValueUtil.length(cell) > maxKeyValueSize) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
if (kv.getLength() > maxKeyValueSize) {
throw new IllegalArgumentException("KeyValue size too large"); throw new IllegalArgumentException("KeyValue size too large");
} }
} }

View File

@ -28,6 +28,7 @@ import java.util.UUID;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.io.TimeRange;
@ -97,17 +98,16 @@ public class Increment extends Mutation implements Comparable<Row> {
*/ */
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public Increment add(Cell cell) throws IOException{ public Increment add(Cell cell) throws IOException{
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); byte [] family = CellUtil.cloneFamily(cell);
byte [] family = kv.getFamily();
List<Cell> list = getCellList(family); List<Cell> list = getCellList(family);
//Checking that the row of the kv is the same as the put //Checking that the row of the kv is the same as the put
int res = Bytes.compareTo(this.row, 0, row.length, int res = Bytes.compareTo(this.row, 0, row.length,
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength()); cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
if (res != 0) { if (res != 0) {
throw new WrongRowIOException("The row in " + kv.toString() + throw new WrongRowIOException("The row in " + cell +
" doesn't match the original one " + Bytes.toStringBinary(this.row)); " doesn't match the original one " + Bytes.toStringBinary(this.row));
} }
list.add(kv); list.add(cell);
familyMap.put(family, list); familyMap.put(family, list);
return this; return this;
} }
@ -133,7 +133,7 @@ public class Increment extends Mutation implements Comparable<Row> {
List<Cell> list = getCellList(family); List<Cell> list = getCellList(family);
KeyValue kv = createPutKeyValue(family, qualifier, ts, Bytes.toBytes(amount)); KeyValue kv = createPutKeyValue(family, qualifier, ts, Bytes.toBytes(amount));
list.add(kv); list.add(kv);
familyMap.put(kv.getFamily(), list); familyMap.put(CellUtil.cloneFamily(kv), list);
return this; return this;
} }
@ -197,9 +197,8 @@ public class Increment extends Mutation implements Comparable<Row> {
for (Map.Entry<byte [], List<Cell>> entry: map.entrySet()) { for (Map.Entry<byte [], List<Cell>> entry: map.entrySet()) {
NavigableMap<byte [], Long> longs = new TreeMap<byte [], Long>(Bytes.BYTES_COMPARATOR); NavigableMap<byte [], Long> longs = new TreeMap<byte [], Long>(Bytes.BYTES_COMPARATOR);
for (Cell cell: entry.getValue()) { for (Cell cell: entry.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); longs.put(CellUtil.cloneQualifier(cell),
longs.put(kv.getQualifier(), Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
Bytes.toLong(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
} }
results.put(entry.getKey(), longs); results.put(entry.getKey(), longs);
} }

View File

@ -28,11 +28,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
@ -292,9 +292,8 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
} }
long resultSize = 0; long resultSize = 0;
for (Result rr : rrs) { for (Result rr : rrs) {
for (Cell kv : rr.rawCells()) { for (Cell cell : rr.rawCells()) {
// TODO add getLength to Cell/use CellUtil#estimatedSizeOf resultSize += CellUtil.estimatedLengthOf(cell);
resultSize += KeyValueUtil.ensureKeyValue(kv).getLength();
} }
} }
this.scanMetrics.countOfBytesInResults.addAndGet(resultSize); this.scanMetrics.countOfBytesInResults.addAndGet(resultSize);

View File

@ -119,7 +119,7 @@ public class FuzzyRowFilter extends FilterBase {
// TODO: is there a better way than throw exception? (stop the scanner?) // TODO: is there a better way than throw exception? (stop the scanner?)
throw new IllegalStateException("No next row key that satisfies fuzzy exists when" throw new IllegalStateException("No next row key that satisfies fuzzy exists when"
+ " getNextKeyHint() is invoked." + " Filter: " + this.toString() + " currentKV: " + " getNextKeyHint() is invoked." + " Filter: " + this.toString() + " currentKV: "
+ KeyValueUtil.ensureKeyValue(currentCell).toString()); + currentCell);
} }
return KeyValueUtil.createFirstOnRow(nextRowKey); return KeyValueUtil.createFirstOnRow(nextRowKey);

View File

@ -673,7 +673,7 @@ public final class ProtobufUtil {
delete = delete =
new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp); new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
} }
delete.addDeleteMarker(KeyValueUtil.ensureKeyValue(cell)); delete.addDeleteMarker(cell);
} }
} else { } else {
delete = new Delete(row, timestamp); delete = new Delete(row, timestamp);
@ -737,7 +737,7 @@ public final class ProtobufUtil {
if (append == null) { if (append == null) {
append = new Append(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); append = new Append(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
} }
append.add(KeyValueUtil.ensureKeyValue(cell)); append.add(cell);
} }
} else { } else {
append = new Append(row); append = new Append(row);
@ -816,7 +816,7 @@ public final class ProtobufUtil {
if (increment == null) { if (increment == null) {
increment = new Increment(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); increment = new Increment(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
} }
increment.add(KeyValueUtil.ensureKeyValue(cell)); increment.add(cell);
} }
} else { } else {
increment = new Increment(row); increment = new Increment(row);
@ -1126,14 +1126,13 @@ public final class ProtobufUtil {
List<Cell> values = family.getValue(); List<Cell> values = family.getValue();
if (values != null && values.size() > 0) { if (values != null && values.size() > 0) {
for (Cell cell: values) { for (Cell cell: values) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
valueBuilder.setQualifier(ByteStringer.wrap( valueBuilder.setQualifier(ByteStringer.wrap(
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength())); cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()));
valueBuilder.setValue(ByteStringer.wrap( valueBuilder.setValue(ByteStringer.wrap(
kv.getValueArray(), kv.getValueOffset(), kv.getValueLength())); cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
if (kv.getTagsLength() > 0) { if (cell.getTagsLength() > 0) {
valueBuilder.setTags(ByteStringer.wrap(kv.getTagsArray(), valueBuilder.setTags(ByteStringer.wrap(cell.getTagsArray(),
kv.getTagsOffset(), kv.getTagsLength())); cell.getTagsOffset(), cell.getTagsLength()));
} }
columnBuilder.addQualifierValue(valueBuilder.build()); columnBuilder.addQualifierValue(valueBuilder.build());
} }
@ -1189,18 +1188,17 @@ public final class ProtobufUtil {
columnBuilder.clear(); columnBuilder.clear();
columnBuilder.setFamily(ByteStringer.wrap(family.getKey())); columnBuilder.setFamily(ByteStringer.wrap(family.getKey()));
for (Cell cell: family.getValue()) { for (Cell cell: family.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
valueBuilder.setQualifier(ByteStringer.wrap( valueBuilder.setQualifier(ByteStringer.wrap(
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength())); cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()));
valueBuilder.setValue(ByteStringer.wrap( valueBuilder.setValue(ByteStringer.wrap(
kv.getValueArray(), kv.getValueOffset(), kv.getValueLength())); cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
valueBuilder.setTimestamp(kv.getTimestamp()); valueBuilder.setTimestamp(cell.getTimestamp());
if(cell.getTagsLength() > 0) { if(cell.getTagsLength() > 0) {
valueBuilder.setTags(ByteStringer.wrap(kv.getTagsArray(), kv.getTagsOffset(), valueBuilder.setTags(ByteStringer.wrap(cell.getTagsArray(), cell.getTagsOffset(),
kv.getTagsLength())); cell.getTagsLength()));
} }
if (type == MutationType.DELETE || (type == MutationType.PUT && CellUtil.isDelete(kv))) { if (type == MutationType.DELETE || (type == MutationType.PUT && CellUtil.isDelete(cell))) {
KeyValue.Type keyValueType = KeyValue.Type.codeToType(kv.getType()); KeyValue.Type keyValueType = KeyValue.Type.codeToType(cell.getTypeByte());
valueBuilder.setDeleteType(toDeleteType(keyValueType)); valueBuilder.setDeleteType(toDeleteType(keyValueType));
} }
columnBuilder.addQualifierValue(valueBuilder.build()); columnBuilder.addQualifierValue(valueBuilder.build());

View File

@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SettableSequenceId; import org.apache.hadoop.hbase.SettableSequenceId;
@ -475,34 +476,31 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
@Override @Override
@Deprecated @Deprecated
public byte[] getValue() { public byte[] getValue() {
throw new UnsupportedOperationException("getValue() not supported"); return CellUtil.cloneValue(this);
} }
@Override @Override
@Deprecated @Deprecated
public byte[] getFamily() { public byte[] getFamily() {
throw new UnsupportedOperationException("getFamily() not supported"); return CellUtil.cloneFamily(this);
} }
@Override @Override
@Deprecated @Deprecated
public byte[] getQualifier() { public byte[] getQualifier() {
throw new UnsupportedOperationException("getQualifier() not supported"); return CellUtil.cloneQualifier(this);
} }
@Override @Override
@Deprecated @Deprecated
public byte[] getRow() { public byte[] getRow() {
throw new UnsupportedOperationException("getRow() not supported"); return CellUtil.cloneRow(this);
} }
@Override @Override
public String toString() { public String toString() {
KeyValue kv = KeyValueUtil.copyToNewKeyValue(this); return KeyValue.keyToString(this.keyOnlyBuffer, 0, KeyValueUtil.keyLength(this)) + "/vlen="
if (kv == null) { + getValueLength() + "/seqid=" + seqId;
return "null";
}
return kv.toString();
} }
@Override @Override

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValueUtil;
@ -84,9 +85,8 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
Result result = Result.create(values); Result result = Result.create(values);
if (this.scanMetrics != null) { if (this.scanMetrics != null) {
long resultSize = 0; long resultSize = 0;
for (Cell kv : values) { for (Cell cell : values) {
// TODO add getLength to Cell/use CellUtil#estimatedSizeOf resultSize += CellUtil.estimatedLengthOf(cell);
resultSize += KeyValueUtil.ensureKeyValue(kv).getLength();
} }
this.scanMetrics.countOfBytesInResults.addAndGet(resultSize); this.scanMetrics.countOfBytesInResults.addAndGet(resultSize);
} }

View File

@ -145,10 +145,8 @@ import org.apache.hadoop.hbase.util.HashedBytes;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.io.MultipleIOException; import org.apache.hadoop.io.MultipleIOException;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
@ -3102,7 +3100,7 @@ public class HRegion implements HeapSize { // , Writable{
CellUtil.setSequenceId(cell, mvccNum); CellUtil.setSequenceId(cell, mvccNum);
Pair<Long, Cell> ret = store.add(cell); Pair<Long, Cell> ret = store.add(cell);
size += ret.getFirst(); size += ret.getFirst();
memstoreCells.add(KeyValueUtil.ensureKeyValue(ret.getSecond())); memstoreCells.add(ret.getSecond());
} }
} }
@ -3118,7 +3116,7 @@ public class HRegion implements HeapSize { // , Writable{
int kvsRolledback = 0; int kvsRolledback = 0;
for (Cell cell : memstoreCells) { for (Cell cell : memstoreCells) {
byte[] family = cell.getFamily(); byte[] family = CellUtil.cloneFamily(cell);
Store store = getStore(family); Store store = getStore(family);
store.rollback(cell); store.rollback(cell);
kvsRolledback++; kvsRolledback++;
@ -3170,8 +3168,8 @@ public class HRegion implements HeapSize { // , Writable{
for (List<Cell> kvs : familyMap.values()) { for (List<Cell> kvs : familyMap.values()) {
for (Cell cell : kvs) { for (Cell cell : kvs) {
// see if the user-side TS is out of range. latest = server-side // see if the user-side TS is out of range. latest = server-side
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); long ts = cell.getTimestamp();
if (!kv.isLatestTimestamp() && kv.getTimestamp() > maxTs) { if (ts != HConstants.LATEST_TIMESTAMP && ts > maxTs) {
throw new FailedSanityCheckException("Timestamp for KV out of range " throw new FailedSanityCheckException("Timestamp for KV out of range "
+ cell + " (too.new=" + timestampSlop + ")"); + cell + " (too.new=" + timestampSlop + ")");
} }
@ -4935,8 +4933,8 @@ public class HRegion implements HeapSize { // , Writable{
// do after lock // do after lock
if (this.metricsRegion != null) { if (this.metricsRegion != null) {
long totalSize = 0l; long totalSize = 0l;
for (Cell kv : results) { for (Cell cell : results) {
totalSize += KeyValueUtil.ensureKeyValue(kv).getLength(); totalSize += CellUtil.estimatedLengthOf(cell);
} }
this.metricsRegion.updateGet(totalSize); this.metricsRegion.updateGet(totalSize);
} }
@ -5123,8 +5121,8 @@ public class HRegion implements HeapSize { // , Writable{
processor.getRowsToLock().iterator().next()) + "..."); processor.getRowsToLock().iterator().next()) + "...");
for (Mutation m : mutations) { for (Mutation m : mutations) {
for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) { for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cellScanner.current()); Cell cell = cellScanner.current();
getStore(kv).rollback(kv); getStore(cell).rollback(cell);
} }
} }
} }
@ -5283,60 +5281,61 @@ public class HRegion implements HeapSize { // , Writable{
// Would be nice if KeyValue had scatter/gather logic // Would be nice if KeyValue had scatter/gather logic
int idx = 0; int idx = 0;
for (Cell cell : family.getValue()) { for (Cell cell : family.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); Cell newCell;
KeyValue newKV; Cell oldCell = null;
KeyValue oldKv = null;
if (idx < results.size() if (idx < results.size()
&& CellUtil.matchingQualifier(results.get(idx),kv)) { && CellUtil.matchingQualifier(results.get(idx), cell)) {
oldKv = KeyValueUtil.ensureKeyValue(results.get(idx)); oldCell = results.get(idx);
// allocate an empty kv once // allocate an empty kv once
newKV = new KeyValue(row.length, kv.getFamilyLength(), newCell = new KeyValue(row.length, cell.getFamilyLength(),
kv.getQualifierLength(), now, KeyValue.Type.Put, cell.getQualifierLength(), now, KeyValue.Type.Put,
oldKv.getValueLength() + kv.getValueLength(), oldCell.getValueLength() + cell.getValueLength(),
oldKv.getTagsLength() + kv.getTagsLength()); oldCell.getTagsLength() + cell.getTagsLength());
// copy in the value // copy in the value
System.arraycopy(oldKv.getValueArray(), oldKv.getValueOffset(), System.arraycopy(oldCell.getValueArray(), oldCell.getValueOffset(),
newKV.getValueArray(), newKV.getValueOffset(), newCell.getValueArray(), newCell.getValueOffset(),
oldKv.getValueLength()); oldCell.getValueLength());
System.arraycopy(kv.getValueArray(), kv.getValueOffset(), System.arraycopy(cell.getValueArray(), cell.getValueOffset(),
newKV.getValueArray(), newCell.getValueArray(),
newKV.getValueOffset() + oldKv.getValueLength(), newCell.getValueOffset() + oldCell.getValueLength(),
kv.getValueLength()); cell.getValueLength());
// copy in the tags // copy in the tags
System.arraycopy(oldKv.getTagsArray(), oldKv.getTagsOffset(), newKV.getTagsArray(), System.arraycopy(oldCell.getTagsArray(), oldCell.getTagsOffset(),
newKV.getTagsOffset(), oldKv.getTagsLength()); newCell.getTagsArray(), newCell.getTagsOffset(), oldCell.getTagsLength());
System.arraycopy(kv.getTagsArray(), kv.getTagsOffset(), newKV.getTagsArray(), System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), newCell.getTagsArray(),
newKV.getTagsOffset() + oldKv.getTagsLength(), kv.getTagsLength()); newCell.getTagsOffset() + oldCell.getTagsLength(), cell.getTagsLength());
// copy in row, family, and qualifier // copy in row, family, and qualifier
System.arraycopy(kv.getRowArray(), kv.getRowOffset(), System.arraycopy(cell.getRowArray(), cell.getRowOffset(),
newKV.getRowArray(), newKV.getRowOffset(), kv.getRowLength()); newCell.getRowArray(), newCell.getRowOffset(), cell.getRowLength());
System.arraycopy(kv.getFamilyArray(), kv.getFamilyOffset(), System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(),
newKV.getFamilyArray(), newKV.getFamilyOffset(), newCell.getFamilyArray(), newCell.getFamilyOffset(),
kv.getFamilyLength()); cell.getFamilyLength());
System.arraycopy(kv.getQualifierArray(), kv.getQualifierOffset(), System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(),
newKV.getQualifierArray(), newKV.getQualifierOffset(), newCell.getQualifierArray(), newCell.getQualifierOffset(),
kv.getQualifierLength()); cell.getQualifierLength());
idx++; idx++;
} else { } else {
newKV = kv;
// Append's KeyValue.Type==Put and ts==HConstants.LATEST_TIMESTAMP, // Append's KeyValue.Type==Put and ts==HConstants.LATEST_TIMESTAMP,
// so only need to update the timestamp to 'now' // so only need to update the timestamp to 'now'
// TODO get rid of KeyValueUtil.ensureKeyValue
KeyValue newKV = KeyValueUtil.ensureKeyValue(cell);
newKV.updateLatestStamp(Bytes.toBytes(now)); newKV.updateLatestStamp(Bytes.toBytes(now));
newCell = newKV;
} }
newKV.setSequenceId(mvccNum); CellUtil.setSequenceId(newCell, mvccNum);
// Give coprocessors a chance to update the new cell // Give coprocessors a chance to update the new cell
if (coprocessorHost != null) { if (coprocessorHost != null) {
newKV = KeyValueUtil.ensureKeyValue(coprocessorHost.postMutationBeforeWAL( newCell = coprocessorHost.postMutationBeforeWAL(RegionObserver.MutationType.APPEND,
RegionObserver.MutationType.APPEND, append, oldKv, newKV)); append, oldCell, newCell);
} }
kvs.add(newKV); kvs.add(newCell);
// Append update to WAL // Append update to WAL
if (writeToWAL) { if (writeToWAL) {
if (walEdits == null) { if (walEdits == null) {
walEdits = new WALEdit(); walEdits = new WALEdit();
} }
walEdits.add(newKV); walEdits.add(newCell);
} }
} }
@ -5354,8 +5353,7 @@ public class HRegion implements HeapSize { // , Writable{
} else { } else {
// otherwise keep older versions around // otherwise keep older versions around
for (Cell cell: entry.getValue()) { for (Cell cell: entry.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); Pair<Long, Cell> ret = store.add(cell);
Pair<Long, Cell> ret = store.add(kv);
size += ret.getFirst(); size += ret.getFirst();
memstoreCells.add(ret.getSecond()); memstoreCells.add(ret.getSecond());
doRollBackMemstore = true; doRollBackMemstore = true;
@ -5520,7 +5518,7 @@ public class HRegion implements HeapSize { // , Writable{
byte[] val = Bytes.toBytes(amount); byte[] val = Bytes.toBytes(amount);
int oldCellTagsLen = (c == null) ? 0 : c.getTagsLength(); int oldCellTagsLen = (c == null) ? 0 : c.getTagsLength();
int incCellTagsLen = kv.getTagsLength(); int incCellTagsLen = kv.getTagsLength();
KeyValue newKV = new KeyValue(row.length, family.getKey().length, q.length, now, Cell newKV = new KeyValue(row.length, family.getKey().length, q.length, now,
KeyValue.Type.Put, val.length, oldCellTagsLen + incCellTagsLen); KeyValue.Type.Put, val.length, oldCellTagsLen + incCellTagsLen);
System.arraycopy(row, 0, newKV.getRowArray(), newKV.getRowOffset(), row.length); System.arraycopy(row, 0, newKV.getRowArray(), newKV.getRowOffset(), row.length);
System.arraycopy(family.getKey(), 0, newKV.getFamilyArray(), newKV.getFamilyOffset(), System.arraycopy(family.getKey(), 0, newKV.getFamilyArray(), newKV.getFamilyOffset(),
@ -5537,11 +5535,11 @@ public class HRegion implements HeapSize { // , Writable{
System.arraycopy(kv.getTagsArray(), kv.getTagsOffset(), newKV.getTagsArray(), System.arraycopy(kv.getTagsArray(), kv.getTagsOffset(), newKV.getTagsArray(),
newKV.getTagsOffset() + oldCellTagsLen, incCellTagsLen); newKV.getTagsOffset() + oldCellTagsLen, incCellTagsLen);
} }
newKV.setSequenceId(mvccNum); CellUtil.setSequenceId(newKV, mvccNum);
// Give coprocessors a chance to update the new cell // Give coprocessors a chance to update the new cell
if (coprocessorHost != null) { if (coprocessorHost != null) {
newKV = KeyValueUtil.ensureKeyValue(coprocessorHost.postMutationBeforeWAL( newKV = coprocessorHost.postMutationBeforeWAL(
RegionObserver.MutationType.INCREMENT, increment, c, newKV)); RegionObserver.MutationType.INCREMENT, increment, c, newKV);
} }
allKVs.add(newKV); allKVs.add(newKV);
@ -5575,8 +5573,7 @@ public class HRegion implements HeapSize { // , Writable{
} else { } else {
// otherwise keep older versions around // otherwise keep older versions around
for (Cell cell : entry.getValue()) { for (Cell cell : entry.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); Pair<Long, Cell> ret = store.add(cell);
Pair<Long, Cell> ret = store.add(kv);
size += ret.getFirst(); size += ret.getFirst();
memstoreCells.add(ret.getSecond()); memstoreCells.add(ret.getSecond());
doRollBackMemstore = true; doRollBackMemstore = true;
@ -6064,8 +6061,8 @@ public class HRegion implements HeapSize { // , Writable{
long mutationSize = 0; long mutationSize = 0;
for (List<Cell> cells: familyMap.values()) { for (List<Cell> cells: familyMap.values()) {
for (Cell cell : cells) { for (Cell cell : cells) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); // TODO we need include tags length also here.
mutationSize += kv.getKeyLength() + kv.getValueLength(); mutationSize += KeyValueUtil.keyLength(cell) + cell.getValueLength();
} }
} }

View File

@ -28,8 +28,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.compactions.Compactor; import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
@ -120,8 +118,7 @@ abstract class StoreFlusher {
// If we know that this KV is going to be included always, then let us // If we know that this KV is going to be included always, then let us
// set its memstoreTS to 0. This will help us save space when writing to // set its memstoreTS to 0. This will help us save space when writing to
// disk. // disk.
KeyValue kv = KeyValueUtil.ensureKeyValue(c); sink.append(c);
sink.append(kv);
} }
kvs.clear(); kvs.clear();
} }

View File

@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
@ -149,9 +148,9 @@ public class ReplicationSink {
addToHashMultiMap(rowMap, table, clusterIds, m); addToHashMultiMap(rowMap, table, clusterIds, m);
} }
if (CellUtil.isDelete(cell)) { if (CellUtil.isDelete(cell)) {
((Delete)m).addDeleteMarker(KeyValueUtil.ensureKeyValue(cell)); ((Delete)m).addDeleteMarker(cell);
} else { } else {
((Put)m).add(KeyValueUtil.ensureKeyValue(cell)); ((Put)m).add(cell);
} }
previousCell = cell; previousCell = cell;
} }

View File

@ -35,11 +35,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
@ -198,10 +198,9 @@ public class RemoteHTable implements HTableInterface {
long ts = put.getTimeStamp(); long ts = put.getTimeStamp();
for (List<Cell> cells: put.getFamilyCellMap().values()) { for (List<Cell> cells: put.getFamilyCellMap().values()) {
for (Cell cell: cells) { for (Cell cell: cells) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); row.addCell(new CellModel(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell),
row.addCell(new CellModel(kv.getFamily(), kv.getQualifier(), ts != HConstants.LATEST_TIMESTAMP ? ts : cell.getTimestamp(),
ts != HConstants.LATEST_TIMESTAMP ? ts : kv.getTimestamp(), CellUtil.cloneValue(cell)));
kv.getValue()));
} }
} }
CellSetModel model = new CellSetModel(); CellSetModel model = new CellSetModel();
@ -442,8 +441,7 @@ public class RemoteHTable implements HTableInterface {
for (Map.Entry<byte[], List<Cell>> e: map.entrySet()) { for (Map.Entry<byte[], List<Cell>> e: map.entrySet()) {
RowModel row = new RowModel(e.getKey()); RowModel row = new RowModel(e.getKey());
for (Cell cell: e.getValue()) { for (Cell cell: e.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); row.addCell(new CellModel(cell));
row.addCell(new CellModel(kv));
} }
model.addRow(row); model.addRow(row);
} }

View File

@ -30,6 +30,7 @@ import javax.xml.bind.annotation.XmlValue;
import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler; import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
@ -99,10 +100,11 @@ public class CellModel implements ProtobufMessageHandler, Serializable {
/** /**
* Constructor from KeyValue * Constructor from KeyValue
* @param kv * @param cell
*/ */
public CellModel(KeyValue kv) { public CellModel(org.apache.hadoop.hbase.Cell cell) {
this(kv.getFamily(), kv.getQualifier(), kv.getTimestamp(), kv.getValue()); this(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), cell.getTimestamp(), CellUtil
.cloneValue(cell));
} }
/** /**

View File

@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
@ -68,7 +67,6 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
import org.apache.hadoop.hbase.coprocessor.CoprocessorService; import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
import org.apache.hadoop.hbase.coprocessor.EndpointObserver; import org.apache.hadoop.hbase.coprocessor.EndpointObserver;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
@ -232,11 +230,10 @@ public class AccessController extends BaseMasterAndRegionObserver
for (Map.Entry<byte[], List<Cell>> f : familyMap.entrySet()) { for (Map.Entry<byte[], List<Cell>> f : familyMap.entrySet()) {
List<Cell> cells = f.getValue(); List<Cell> cells = f.getValue();
for (Cell cell: cells) { for (Cell cell: cells) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); if (Bytes.equals(cell.getFamilyArray(), cell.getFamilyOffset(),
if (Bytes.equals(kv.getFamilyArray(), kv.getFamilyOffset(), cell.getFamilyLength(), AccessControlLists.ACL_LIST_FAMILY, 0,
kv.getFamilyLength(), AccessControlLists.ACL_LIST_FAMILY, 0,
AccessControlLists.ACL_LIST_FAMILY.length)) { AccessControlLists.ACL_LIST_FAMILY.length)) {
entries.add(kv.getRow()); entries.add(CellUtil.cloneRow(cell));
} }
} }
} }
@ -771,15 +768,12 @@ public class AccessController extends BaseMasterAndRegionObserver
tags.add(tagIterator.next()); tags.add(tagIterator.next());
} }
} }
// Ensure KeyValue so we can do a scatter gather copy. This is only a win if the
// incoming cell type is actually KeyValue.
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
newCells.add( newCells.add(
new KeyValue(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(), cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
kv.getTimestamp(), KeyValue.Type.codeToType(kv.getTypeByte()), cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
kv.getValueArray(), kv.getValueOffset(), kv.getValueLength(), cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
tags)); tags));
} }
// This is supposed to be safe, won't CME // This is supposed to be safe, won't CME
@ -1769,15 +1763,14 @@ public class AccessController extends BaseMasterAndRegionObserver
// We need to create another KV, unfortunately, because the current new KV // We need to create another KV, unfortunately, because the current new KV
// has no space for tags // has no space for tags
KeyValue newKv = KeyValueUtil.ensureKeyValue(newCell); KeyValue rewriteKv = new KeyValue(newCell.getRowArray(), newCell.getRowOffset(),
KeyValue rewriteKv = new KeyValue(newKv.getRowArray(), newKv.getRowOffset(), newKv.getRowLength(), newCell.getRowLength(), newCell.getFamilyArray(), newCell.getFamilyOffset(),
newKv.getFamilyArray(), newKv.getFamilyOffset(), newKv.getFamilyLength(), newCell.getFamilyLength(), newCell.getQualifierArray(), newCell.getQualifierOffset(),
newKv.getQualifierArray(), newKv.getQualifierOffset(), newKv.getQualifierLength(), newCell.getQualifierLength(), newCell.getTimestamp(), KeyValue.Type.codeToType(newCell
newKv.getTimestamp(), KeyValue.Type.codeToType(newKv.getTypeByte()), .getTypeByte()), newCell.getValueArray(), newCell.getValueOffset(),
newKv.getValueArray(), newKv.getValueOffset(), newKv.getValueLength(), newCell.getValueLength(), tags);
tags);
// Preserve mvcc data // Preserve mvcc data
rewriteKv.setSequenceId(newKv.getMvccVersion()); rewriteKv.setSequenceId(newCell.getSequenceId());
return rewriteKv; return rewriteKv;
} }

View File

@ -30,8 +30,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
@ -333,10 +331,9 @@ public class ThriftUtilities {
in.getFamilyCellMap().entrySet()) { in.getFamilyCellMap().entrySet()) {
TColumn column = new TColumn(ByteBuffer.wrap(familyEntry.getKey())); TColumn column = new TColumn(ByteBuffer.wrap(familyEntry.getKey()));
for (org.apache.hadoop.hbase.Cell cell: familyEntry.getValue()) { for (org.apache.hadoop.hbase.Cell cell: familyEntry.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); byte[] family = CellUtil.cloneFamily(cell);
byte[] family = kv.getFamily(); byte[] qualifier = CellUtil.cloneQualifier(cell);
byte[] qualifier = kv.getQualifier(); long timestamp = cell.getTimestamp();
long timestamp = kv.getTimestamp();
if (family != null) { if (family != null) {
column.setFamily(family); column.setFamily(family);
} }
@ -344,7 +341,7 @@ public class ThriftUtilities {
column.setQualifier(qualifier); column.setQualifier(qualifier);
} }
if (timestamp != HConstants.LATEST_TIMESTAMP) { if (timestamp != HConstants.LATEST_TIMESTAMP) {
column.setTimestamp(kv.getTimestamp()); column.setTimestamp(timestamp);
} }
} }
columns.add(column); columns.add(column);