HBASE-12112 Avoid KeyValueUtil#ensureKeyValue some more simple cases.
This commit is contained in:
parent
231bc98761
commit
4fac4c1ba6
|
@ -27,11 +27,11 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||
|
@ -452,9 +452,9 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
if (values != null && values.length > 0) {
|
||||
for (Result rs : values) {
|
||||
cache.add(rs);
|
||||
for (Cell kv : rs.rawCells()) {
|
||||
// TODO make method in Cell or CellUtil
|
||||
remainingResultSize -= KeyValueUtil.ensureKeyValue(kv).heapSize();
|
||||
// We don't make Iterator here
|
||||
for (Cell cell : rs.rawCells()) {
|
||||
remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
countdown--;
|
||||
this.lastResult = rs;
|
||||
|
|
|
@ -23,11 +23,10 @@ package org.apache.hadoop.hbase.client;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -160,8 +159,9 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
|
|||
continue;
|
||||
}
|
||||
cache.add(rs);
|
||||
for (Cell kv : rs.rawCells()) {
|
||||
remainingResultSize -= KeyValueUtil.ensureKeyValue(kv).heapSize();
|
||||
// We don't make Iterator here
|
||||
for (Cell cell : rs.rawCells()) {
|
||||
remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
countdown--;
|
||||
this.lastResult = rs;
|
||||
|
|
|
@ -27,8 +27,8 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
|
@ -225,8 +225,9 @@ public class ClientSmallScanner extends ClientScanner {
|
|||
continue;
|
||||
}
|
||||
cache.add(rs);
|
||||
for (Cell kv : rs.rawCells()) {
|
||||
remainingResultSize -= KeyValueUtil.ensureKeyValue(kv).heapSize();
|
||||
// We don't make Iterator here
|
||||
for (Cell cell : rs.rawCells()) {
|
||||
remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
countdown--;
|
||||
this.lastResult = rs;
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
import org.apache.hadoop.hbase.security.access.Permission;
|
||||
import org.apache.hadoop.hbase.security.visibility.CellVisibility;
|
||||
|
@ -240,9 +239,8 @@ public class Increment extends Mutation implements Comparable<Row> {
|
|||
} else {
|
||||
moreThanOneB = true;
|
||||
}
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
||||
sb.append(Bytes.toStringBinary(kv.getKey()) + "+=" +
|
||||
Bytes.toLong(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
|
||||
sb.append(CellUtil.getCellKey(cell) + "+=" +
|
||||
Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
|
||||
}
|
||||
sb.append("}");
|
||||
}
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.CellScanner;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
|
@ -191,13 +190,8 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
|
|||
if (--maxCols <= 0 ) {
|
||||
continue;
|
||||
}
|
||||
// KeyValue v1 expectation. Cast for now until we go all Cell all the time.
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
||||
Map<String, Object> kvMap = kv.toStringMap();
|
||||
// row and family information are already available in the bigger map
|
||||
kvMap.remove("row");
|
||||
kvMap.remove("family");
|
||||
qualifierDetails.add(kvMap);
|
||||
Map<String, Object> cellMap = cellToStringMap(cell);
|
||||
qualifierDetails.add(cellMap);
|
||||
}
|
||||
}
|
||||
map.put("totalColumns", colCount);
|
||||
|
@ -208,6 +202,23 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
|
|||
return map;
|
||||
}
|
||||
|
||||
private static Map<String, Object> cellToStringMap(Cell c) {
|
||||
Map<String, Object> stringMap = new HashMap<String, Object>();
|
||||
stringMap.put("qualifier", Bytes.toStringBinary(c.getQualifierArray(), c.getQualifierOffset(),
|
||||
c.getQualifierLength()));
|
||||
stringMap.put("timestamp", c.getTimestamp());
|
||||
stringMap.put("vlen", c.getValueLength());
|
||||
List<Tag> tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
|
||||
if (tags != null) {
|
||||
List<String> tagsString = new ArrayList<String>();
|
||||
for (Tag t : tags) {
|
||||
tagsString.add((t.getType()) + ":" + Bytes.toStringBinary(t.getValue()));
|
||||
}
|
||||
stringMap.put("tag", tagsString);
|
||||
}
|
||||
return stringMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the durability for this mutation
|
||||
* @param d
|
||||
|
@ -367,8 +378,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
|
|||
size * ClassSize.REFERENCE);
|
||||
|
||||
for(Cell cell : entry.getValue()) {
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
||||
heapsize += kv.heapSize();
|
||||
heapsize += CellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
}
|
||||
heapsize += getAttributeSize();
|
||||
|
|
|
@ -743,7 +743,7 @@ public class Result implements CellScannable, CellScanner {
|
|||
public static long getTotalSizeOfCells(Result result) {
|
||||
long size = 0;
|
||||
for (Cell c : result.rawCells()) {
|
||||
size += KeyValueUtil.ensureKeyValue(c).heapSize();
|
||||
size += CellUtil.estimatedHeapSizeOf(c);
|
||||
}
|
||||
return size;
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.KeyValue;
|
|||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
@ -49,15 +50,24 @@ public class KeyOnlyFilter extends FilterBase {
|
|||
public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; }
|
||||
|
||||
@Override
|
||||
public Cell transformCell(Cell kv) {
|
||||
// TODO Move to KeyValueUtil
|
||||
public Cell transformCell(Cell cell) {
|
||||
return createKeyOnlyCell(cell);
|
||||
}
|
||||
|
||||
// TODO make matching Column a cell method or CellUtil method.
|
||||
// Even if we want to make use of KeyValue.KeyOnlyKeyValue we need to convert
|
||||
// the cell to KV so that we can make use of kv.getKey() to form the key part
|
||||
KeyValue v = KeyValueUtil.ensureKeyValue(kv);
|
||||
|
||||
return v.createKeyOnly(this.lenAsVal);
|
||||
private Cell createKeyOnlyCell(Cell c) {
|
||||
// KV format: <keylen:4><valuelen:4><key:keylen><value:valuelen>
|
||||
// Rebuild as: <keylen:4><0:4><key:keylen>
|
||||
int dataLen = lenAsVal ? Bytes.SIZEOF_INT : 0;
|
||||
int keyOffset = (2 * Bytes.SIZEOF_INT);
|
||||
int keyLen = KeyValueUtil.keyLength(c);
|
||||
byte[] newBuffer = new byte[keyLen + keyOffset + dataLen];
|
||||
Bytes.putInt(newBuffer, 0, keyLen);
|
||||
Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, dataLen);
|
||||
KeyValueUtil.appendKeyTo(c, newBuffer, keyOffset);
|
||||
if (lenAsVal) {
|
||||
Bytes.putInt(newBuffer, newBuffer.length - dataLen, c.getValueLength());
|
||||
}
|
||||
return new KeyValue(newBuffer);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,69 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* This wraps the key portion of a Cell. Key includes rowkey, family, qualifier, timestamp and type
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class CellKey {
|
||||
|
||||
private byte[] rowArray;
|
||||
private int rowOffset;
|
||||
private int rowLength;
|
||||
private byte[] familyArray;
|
||||
private int familyOffset;
|
||||
private int familyLength;
|
||||
private byte[] qualifierArray;
|
||||
private int qualifierOffset;
|
||||
private int qualifierLength;
|
||||
private long ts;
|
||||
private byte type;
|
||||
|
||||
public CellKey(byte[] rowArray, int rowOffset, int rowLength, byte[] familyArray,
|
||||
int familyOffset, int familyLength, byte[] qualifierArray, int qualifierOffset,
|
||||
int qualifierLength, long ts, byte type) {
|
||||
this.rowArray = rowArray;
|
||||
this.rowOffset = rowOffset;
|
||||
this.rowLength = rowLength;
|
||||
this.familyArray = familyArray;
|
||||
this.familyOffset = familyOffset;
|
||||
this.familyLength = familyLength;
|
||||
this.qualifierArray = qualifierArray;
|
||||
this.qualifierOffset = qualifierOffset;
|
||||
this.qualifierLength = qualifierLength;
|
||||
this.ts = ts;
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
String row = Bytes.toStringBinary(rowArray, rowOffset, rowLength);
|
||||
String family = (familyLength == 0) ? "" : Bytes.toStringBinary(familyArray, familyOffset,
|
||||
familyLength);
|
||||
String qualifier = (qualifierLength == 0) ? "" : Bytes.toStringBinary(qualifierArray,
|
||||
qualifierOffset, qualifierLength);
|
||||
return row + "/" + family +
|
||||
(family != null && family.length() > 0 ? ":" : "") + qualifier
|
||||
+ "/" + KeyValue.humanReadableTimestamp(ts) + "/" + Type.codeToType(type);
|
||||
}
|
||||
}
|
|
@ -667,6 +667,17 @@ public final class CellUtil {
|
|||
out.writeByte(cell.getTypeByte());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param cell
|
||||
* @return Key portion of the Cell including rk, cf, qualifier, ts and type.
|
||||
*/
|
||||
public static CellKey getCellKey(Cell cell){
|
||||
return new CellKey(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
|
||||
cell.getTimestamp(), cell.getTypeByte());
|
||||
}
|
||||
|
||||
/**
|
||||
* Write rowkey excluding the common part.
|
||||
* @param cell
|
||||
|
|
|
@ -32,12 +32,14 @@ import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
|||
import org.apache.hadoop.hbase.KeyValue.SamePrefixComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
import org.apache.hadoop.hbase.io.TagCompressionContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockType;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.util.LRUDictionary;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
/**
|
||||
|
@ -326,7 +328,10 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
// there. So this has to be an instance of SettableSequenceId. SeekerState need not be
|
||||
// SettableSequenceId as we never return that to top layers. When we have to, we make
|
||||
// ClonedSeekerState from it.
|
||||
protected static class ClonedSeekerState implements Cell, SettableSequenceId {
|
||||
protected static class ClonedSeekerState implements Cell, HeapSize, SettableSequenceId {
|
||||
private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
|
||||
+ (4 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (7 * Bytes.SIZEOF_INT)
|
||||
+ (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE));
|
||||
private byte[] keyOnlyBuffer;
|
||||
private ByteBuffer currentBuffer;
|
||||
private short rowLength;
|
||||
|
@ -507,6 +512,12 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
public void setSequenceId(long seqId) {
|
||||
this.seqId = seqId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long heapSize() {
|
||||
return FIXED_OVERHEAD + rowLength + familyLength + qualifierLength + valueLength + tagsLength
|
||||
+ KeyValue.TIMESTAMP_TYPE_SIZE;
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract static class
|
||||
|
|
|
@ -45,13 +45,15 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
|
||||
|
@ -307,11 +309,12 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
|
||||
private void scanKeysValues(Path file, KeyValueStatsCollector fileStats,
|
||||
HFileScanner scanner, byte[] row) throws IOException {
|
||||
KeyValue pkv = null;
|
||||
Cell pCell = null;
|
||||
do {
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(scanner.getKeyValue());
|
||||
Cell cell = scanner.getKeyValue();
|
||||
if (row != null && row.length != 0) {
|
||||
int result = Bytes.compareTo(kv.getRow(), row);
|
||||
int result = CellComparator.compareRows(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength(), row, 0, row.length);
|
||||
if (result > 0) {
|
||||
break;
|
||||
} else if (result < 0) {
|
||||
|
@ -320,48 +323,51 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
}
|
||||
// collect stats
|
||||
if (printStats) {
|
||||
fileStats.collect(kv);
|
||||
fileStats.collect(cell);
|
||||
}
|
||||
// dump key value
|
||||
if (printKey) {
|
||||
System.out.print("K: " + kv);
|
||||
System.out.print("K: " + cell);
|
||||
if (printValue) {
|
||||
System.out.print(" V: " + Bytes.toStringBinary(kv.getValue()));
|
||||
System.out.print(" V: "
|
||||
+ Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(),
|
||||
cell.getValueLength()));
|
||||
int i = 0;
|
||||
List<Tag> tags = kv.getTags();
|
||||
List<Tag> tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
|
||||
cell.getTagsLength());
|
||||
for (Tag tag : tags) {
|
||||
System.out
|
||||
.print(String.format(" T[%d]: %s", i++, Bytes.toStringBinary(tag.getValue())));
|
||||
System.out.print(String.format(" T[%d]: %s", i++,
|
||||
Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength())));
|
||||
}
|
||||
}
|
||||
System.out.println();
|
||||
}
|
||||
// check if rows are in order
|
||||
if (checkRow && pkv != null) {
|
||||
if (Bytes.compareTo(pkv.getRow(), kv.getRow()) > 0) {
|
||||
if (checkRow && pCell != null) {
|
||||
if (CellComparator.compareRows(pCell, cell) > 0) {
|
||||
System.err.println("WARNING, previous row is greater then"
|
||||
+ " current row\n\tfilename -> " + file + "\n\tprevious -> "
|
||||
+ Bytes.toStringBinary(pkv.getKey()) + "\n\tcurrent -> "
|
||||
+ Bytes.toStringBinary(kv.getKey()));
|
||||
+ CellUtil.getCellKey(pCell) + "\n\tcurrent -> "
|
||||
+ CellUtil.getCellKey(cell));
|
||||
}
|
||||
}
|
||||
// check if families are consistent
|
||||
if (checkFamily) {
|
||||
String fam = Bytes.toString(kv.getFamily());
|
||||
String fam = Bytes.toString(cell.getFamilyArray(), cell.getFamilyOffset(),
|
||||
cell.getFamilyLength());
|
||||
if (!file.toString().contains(fam)) {
|
||||
System.err.println("WARNING, filename does not match kv family,"
|
||||
+ "\n\tfilename -> " + file + "\n\tkeyvalue -> "
|
||||
+ Bytes.toStringBinary(kv.getKey()));
|
||||
+ CellUtil.getCellKey(cell));
|
||||
}
|
||||
if (pkv != null
|
||||
&& !Bytes.equals(pkv.getFamily(), kv.getFamily())) {
|
||||
if (pCell != null && CellComparator.compareFamilies(pCell, cell) != 0) {
|
||||
System.err.println("WARNING, previous kv has different family"
|
||||
+ " compared to current key\n\tfilename -> " + file
|
||||
+ "\n\tprevious -> " + Bytes.toStringBinary(pkv.getKey())
|
||||
+ "\n\tcurrent -> " + Bytes.toStringBinary(kv.getKey()));
|
||||
+ "\n\tprevious -> " + CellUtil.getCellKey(pCell)
|
||||
+ "\n\tcurrent -> " + CellUtil.getCellKey(cell));
|
||||
}
|
||||
}
|
||||
pkv = kv;
|
||||
pCell = cell;
|
||||
++count;
|
||||
} while (scanner.next());
|
||||
}
|
||||
|
@ -451,21 +457,21 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
|
||||
byte[] biggestRow = null;
|
||||
|
||||
private KeyValue prevKV = null;
|
||||
private Cell prevCell = null;
|
||||
private long maxRowBytes = 0;
|
||||
private long curRowKeyLength;
|
||||
|
||||
public void collect(KeyValue kv) {
|
||||
valLen.update(kv.getValueLength());
|
||||
if (prevKV != null &&
|
||||
KeyValue.COMPARATOR.compareRows(prevKV, kv) != 0) {
|
||||
public void collect(Cell cell) {
|
||||
valLen.update(cell.getValueLength());
|
||||
if (prevCell != null &&
|
||||
KeyValue.COMPARATOR.compareRows(prevCell, cell) != 0) {
|
||||
// new row
|
||||
collectRow();
|
||||
}
|
||||
curRowBytes += kv.getLength();
|
||||
curRowKeyLength = kv.getKeyLength();
|
||||
curRowBytes += KeyValueUtil.length(cell);
|
||||
curRowKeyLength = KeyValueUtil.keyLength(cell);
|
||||
curRowCols++;
|
||||
prevKV = kv;
|
||||
prevCell = cell;
|
||||
}
|
||||
|
||||
private void collectRow() {
|
||||
|
@ -473,8 +479,8 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
rowSizeCols.update(curRowCols);
|
||||
keyLen.update(curRowKeyLength);
|
||||
|
||||
if (curRowBytes > maxRowBytes && prevKV != null) {
|
||||
biggestRow = prevKV.getRow();
|
||||
if (curRowBytes > maxRowBytes && prevCell != null) {
|
||||
biggestRow = prevCell.getRow();
|
||||
maxRowBytes = curRowBytes;
|
||||
}
|
||||
|
||||
|
@ -490,7 +496,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
if (prevKV == null)
|
||||
if (prevCell == null)
|
||||
return "no data available for statistics";
|
||||
|
||||
// Dump the metrics to the output stream
|
||||
|
|
|
@ -29,10 +29,10 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
|
||||
|
@ -1021,10 +1021,9 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
if (comp == 0) {
|
||||
if (seekBefore) {
|
||||
if (lastKeyValueSize < 0) {
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(key);
|
||||
throw new IllegalStateException("blockSeek with seekBefore "
|
||||
+ "at the first key of the block: key="
|
||||
+ Bytes.toStringBinary(kv.getKey(), kv.getKeyOffset(), kv.getKeyLength())
|
||||
+ CellUtil.getCellKey(key)
|
||||
+ ", blockOffset=" + block.getOffset() + ", onDiskSize="
|
||||
+ block.getOnDiskSizeWithHeader());
|
||||
}
|
||||
|
|
|
@ -27,9 +27,9 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
import org.apache.hadoop.hbase.io.crypto.Cipher;
|
||||
|
@ -295,10 +295,9 @@ public class HFileReaderV3 extends HFileReaderV2 {
|
|||
if (comp == 0) {
|
||||
if (seekBefore) {
|
||||
if (lastKeyValueSize < 0) {
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(key);
|
||||
throw new IllegalStateException("blockSeek with seekBefore "
|
||||
+ "at the first key of the block: key="
|
||||
+ Bytes.toStringBinary(kv.getKey(), kv.getKeyOffset(), kv.getKeyLength())
|
||||
+ CellUtil.getCellKey(key)
|
||||
+ ", blockOffset=" + block.getOffset() + ", onDiskSize="
|
||||
+ block.getOnDiskSizeWithHeader());
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
|
||||
|
@ -104,7 +105,7 @@ class DefaultStoreFileManager implements StoreFileManager {
|
|||
|
||||
@Override
|
||||
public Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
|
||||
Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final KeyValue candidate) {
|
||||
Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final Cell candidate) {
|
||||
// Default store has nothing useful to do here.
|
||||
// TODO: move this comment when implementing Level:
|
||||
// Level store can trim the list by range, removing all the files which cannot have
|
||||
|
|
|
@ -2042,7 +2042,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
try {
|
||||
Store store = getStore(family);
|
||||
// get the closest key. (HStore.getRowKeyAtOrBefore can return null)
|
||||
KeyValue key = store.getRowKeyAtOrBefore(row);
|
||||
Cell key = store.getRowKeyAtOrBefore(row);
|
||||
Result result = null;
|
||||
if (key != null) {
|
||||
Get get = new Get(CellUtil.cloneRow(key));
|
||||
|
|
|
@ -49,13 +49,13 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompoundConfiguration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
|
@ -709,30 +709,30 @@ public class HStore implements Store {
|
|||
if (verifyBulkLoads) {
|
||||
long verificationStartTime = EnvironmentEdgeManager.currentTime();
|
||||
LOG.info("Full verification started for bulk load hfile: " + srcPath.toString());
|
||||
Cell prevKV = null;
|
||||
Cell prevCell = null;
|
||||
HFileScanner scanner = reader.getScanner(false, false, false);
|
||||
scanner.seekTo();
|
||||
do {
|
||||
Cell kv = scanner.getKeyValue();
|
||||
if (prevKV != null) {
|
||||
if (Bytes.compareTo(prevKV.getRowArray(), prevKV.getRowOffset(),
|
||||
prevKV.getRowLength(), kv.getRowArray(), kv.getRowOffset(),
|
||||
kv.getRowLength()) > 0) {
|
||||
Cell cell = scanner.getKeyValue();
|
||||
if (prevCell != null) {
|
||||
if (CellComparator.compareRows(prevCell, cell) > 0) {
|
||||
throw new InvalidHFileException("Previous row is greater than"
|
||||
+ " current row: path=" + srcPath + " previous="
|
||||
+ Bytes.toStringBinary(KeyValueUtil.ensureKeyValue(prevKV).getKey()) + " current="
|
||||
+ Bytes.toStringBinary(KeyValueUtil.ensureKeyValue(kv).getKey()));
|
||||
+ CellUtil.getCellKey(prevCell) + " current="
|
||||
+ CellUtil.getCellKey(cell));
|
||||
}
|
||||
if (Bytes.compareTo(prevKV.getFamilyArray(), prevKV.getFamilyOffset(),
|
||||
prevKV.getFamilyLength(), kv.getFamilyArray(), kv.getFamilyOffset(),
|
||||
kv.getFamilyLength()) != 0) {
|
||||
if (CellComparator.compareFamilies(prevCell, cell) != 0) {
|
||||
throw new InvalidHFileException("Previous key had different"
|
||||
+ " family compared to current key: path=" + srcPath
|
||||
+ " previous=" + Bytes.toStringBinary(prevKV.getFamily())
|
||||
+ " current=" + Bytes.toStringBinary(kv.getFamily()));
|
||||
+ " previous="
|
||||
+ Bytes.toStringBinary(prevCell.getFamilyArray(), prevCell.getFamilyOffset(),
|
||||
prevCell.getFamilyLength())
|
||||
+ " current="
|
||||
+ Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
|
||||
cell.getFamilyLength()));
|
||||
}
|
||||
}
|
||||
prevKV = kv;
|
||||
prevCell = cell;
|
||||
} while (scanner.next());
|
||||
LOG.info("Full verification complete for bulk load hfile: " + srcPath.toString()
|
||||
+ " took " + (EnvironmentEdgeManager.currentTime() - verificationStartTime)
|
||||
|
@ -1673,7 +1673,7 @@ public class HStore implements Store {
|
|||
}
|
||||
|
||||
@Override
|
||||
public KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException {
|
||||
public Cell getRowKeyAtOrBefore(final byte[] row) throws IOException {
|
||||
// If minVersions is set, we will not ignore expired KVs.
|
||||
// As we're only looking for the latest matches, that should be OK.
|
||||
// With minVersions > 0 we guarantee that any KV that has any version
|
||||
|
@ -1698,17 +1698,17 @@ public class HStore implements Store {
|
|||
StoreFile sf = sfIterator.next();
|
||||
sfIterator.remove(); // Remove sf from iterator.
|
||||
boolean haveNewCandidate = rowAtOrBeforeFromStoreFile(sf, state);
|
||||
KeyValue keyv = KeyValueUtil.ensureKeyValue(state.getCandidate());
|
||||
Cell candidate = state.getCandidate();
|
||||
// we have an optimization here which stops the search if we find exact match.
|
||||
if (keyv != null && CellUtil.matchingRow(keyv, row)) {
|
||||
return KeyValueUtil.ensureKeyValue(state.getCandidate());
|
||||
if (candidate != null && CellUtil.matchingRow(candidate, row)) {
|
||||
return candidate;
|
||||
}
|
||||
if (haveNewCandidate) {
|
||||
sfIterator = this.storeEngine.getStoreFileManager().updateCandidateFilesForRowKeyBefore(
|
||||
sfIterator, state.getTargetKey(), KeyValueUtil.ensureKeyValue(state.getCandidate()));
|
||||
sfIterator, state.getTargetKey(), candidate);
|
||||
}
|
||||
}
|
||||
return KeyValueUtil.ensureKeyValue(state.getCandidate());
|
||||
return state.getCandidate();
|
||||
} finally {
|
||||
this.lock.readLock().unlock();
|
||||
}
|
||||
|
|
|
@ -48,8 +48,6 @@ import org.apache.hadoop.hbase.HBaseIOException;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -2046,7 +2044,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
lease = regionServer.leases.removeLease(scannerName);
|
||||
List<Result> results = new ArrayList<Result>(rows);
|
||||
long currentScanResultSize = 0;
|
||||
long totalKvSize = 0;
|
||||
long totalCellSize = 0;
|
||||
|
||||
boolean done = false;
|
||||
// Call coprocessor. Get region info from scanner.
|
||||
|
@ -2056,9 +2054,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
if (!results.isEmpty()) {
|
||||
for (Result r : results) {
|
||||
for (Cell cell : r.rawCells()) {
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
||||
currentScanResultSize += kv.heapSize();
|
||||
totalKvSize += kv.getLength();
|
||||
currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
|
||||
totalCellSize += CellUtil.estimatedLengthOf(cell);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2088,9 +2085,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
boolean moreRows = scanner.nextRaw(values);
|
||||
if (!values.isEmpty()) {
|
||||
for (Cell cell : values) {
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
||||
currentScanResultSize += kv.heapSize();
|
||||
totalKvSize += kv.getLength();
|
||||
currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
|
||||
totalCellSize += CellUtil.estimatedLengthOf(cell);
|
||||
}
|
||||
results.add(Result.create(values, null, stale));
|
||||
i++;
|
||||
|
@ -2102,7 +2098,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
}
|
||||
}
|
||||
region.readRequestsCount.add(i);
|
||||
region.getMetrics().updateScanNext(totalKvSize);
|
||||
region.getMetrics().updateScanNext(totalCellSize);
|
||||
} finally {
|
||||
region.closeRegionOperation();
|
||||
}
|
||||
|
|
|
@ -148,10 +148,10 @@ public interface Store extends HeapSize, StoreConfigInformation {
|
|||
* see deletes before we come across cells we are to delete. Presumption is that the
|
||||
* memstore#kvset is processed before memstore#snapshot and so on.
|
||||
* @param row The row key of the targeted row.
|
||||
* @return Found keyvalue or null if none found.
|
||||
* @return Found Cell or null if none found.
|
||||
* @throws IOException
|
||||
*/
|
||||
KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException;
|
||||
Cell getRowKeyAtOrBefore(final byte[] row) throws IOException;
|
||||
|
||||
FileSystem getFileSystem();
|
||||
|
||||
|
|
|
@ -24,8 +24,8 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
|
||||
import com.google.common.collect.ImmutableCollection;
|
||||
|
||||
|
@ -112,7 +112,7 @@ public interface StoreFileManager {
|
|||
* @return The list to replace candidateFiles.
|
||||
*/
|
||||
Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
|
||||
Iterator<StoreFile> candidateFiles, KeyValue targetKey, KeyValue candidate
|
||||
Iterator<StoreFile> candidateFiles, KeyValue targetKey, Cell candidate
|
||||
);
|
||||
|
||||
|
||||
|
|
|
@ -25,16 +25,15 @@ import java.util.Collection;
|
|||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
|
@ -45,8 +44,6 @@ import org.apache.hadoop.util.StringUtils;
|
|||
|
||||
import com.google.common.collect.ImmutableCollection;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
|
||||
/**
|
||||
* Stripe implementation of StoreFileManager.
|
||||
|
@ -179,11 +176,11 @@ public class StripeStoreFileManager
|
|||
}
|
||||
|
||||
/** See {@link StoreFileManager#getCandidateFilesForRowKeyBefore(KeyValue)} and
|
||||
* {@link StoreFileManager#updateCandidateFilesForRowKeyBefore(Iterator, KeyValue, KeyValue)}
|
||||
* {@link StoreFileManager#updateCandidateFilesForRowKeyBefore(Iterator, KeyValue, Cell)}
|
||||
* for details on this methods. */
|
||||
@Override
|
||||
public Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
|
||||
Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final KeyValue candidate) {
|
||||
Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final Cell candidate) {
|
||||
KeyBeforeConcatenatedLists.Iterator original =
|
||||
(KeyBeforeConcatenatedLists.Iterator)candidateFiles;
|
||||
assert original != null;
|
||||
|
|
Loading…
Reference in New Issue