HBASE-21750 Most of KeyValueUtil#length can be replaced by cell#getSerializedSize for better performance because the latter one has been optimized
This commit is contained in:
parent
b0131e19f4
commit
35ed5d6c39
|
@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.CompareOperator;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
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.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.io.TimeRange;
|
import org.apache.hadoop.hbase.io.TimeRange;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
@ -933,7 +932,7 @@ public class HTable implements Table {
|
||||||
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) {
|
||||||
if (KeyValueUtil.length(cell) > maxKeyValueSize) {
|
if (cell.getSerializedSize() > maxKeyValueSize) {
|
||||||
throw new IllegalArgumentException("KeyValue size too large");
|
throw new IllegalArgumentException("KeyValue size too large");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,8 @@ import java.util.Optional;
|
||||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||||
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.Tag;
|
import org.apache.hadoop.hbase.Tag;
|
||||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -147,11 +149,13 @@ public class KeyOnlyFilter extends FilterBase {
|
||||||
|
|
||||||
static class KeyOnlyCell implements Cell {
|
static class KeyOnlyCell implements Cell {
|
||||||
private Cell cell;
|
private Cell cell;
|
||||||
|
private int keyLen;
|
||||||
private boolean lenAsVal;
|
private boolean lenAsVal;
|
||||||
|
|
||||||
public KeyOnlyCell(Cell c, boolean lenAsVal) {
|
public KeyOnlyCell(Cell c, boolean lenAsVal) {
|
||||||
this.cell = c;
|
this.cell = c;
|
||||||
this.lenAsVal = lenAsVal;
|
this.lenAsVal = lenAsVal;
|
||||||
|
this.keyLen = KeyValueUtil.keyLength(c);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -245,7 +249,7 @@ public class KeyOnlyFilter extends FilterBase {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getSerializedSize() {
|
public int getSerializedSize() {
|
||||||
return cell.getSerializedSize();
|
return KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE + keyLen + getValueLength();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.filter;
|
package org.apache.hadoop.hbase.filter;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
@ -102,6 +103,8 @@ public class TestKeyOnlyFilter {
|
||||||
assertTrue(CellUtil.matchingValue(KeyOnlyKeyValue, keyOnlyCell));
|
assertTrue(CellUtil.matchingValue(KeyOnlyKeyValue, keyOnlyCell));
|
||||||
assertTrue(KeyOnlyKeyValue.getValueLength() == keyOnlyByteBufferedCell
|
assertTrue(KeyOnlyKeyValue.getValueLength() == keyOnlyByteBufferedCell
|
||||||
.getValueLength());
|
.getValueLength());
|
||||||
|
assertEquals(8 + keyLen + (lenAsVal ? 4 : 0), KeyOnlyKeyValue.getSerializedSize());
|
||||||
|
assertEquals(8 + keyLen + (lenAsVal ? 4 : 0), keyOnlyCell.getSerializedSize());
|
||||||
if (keyOnlyByteBufferedCell.getValueLength() > 0) {
|
if (keyOnlyByteBufferedCell.getValueLength() > 0) {
|
||||||
assertTrue(CellUtil.matchingValue(KeyOnlyKeyValue,
|
assertTrue(CellUtil.matchingValue(KeyOnlyKeyValue,
|
||||||
keyOnlyByteBufferedCell));
|
keyOnlyByteBufferedCell));
|
||||||
|
|
|
@ -268,7 +268,7 @@ public class ByteBufferKeyValue extends ByteBufferExtendedCell {
|
||||||
if (this.buf.hasArray()) {
|
if (this.buf.hasArray()) {
|
||||||
return ClassSize.align(FIXED_OVERHEAD + length);
|
return ClassSize.align(FIXED_OVERHEAD + length);
|
||||||
}
|
}
|
||||||
return ClassSize.align(FIXED_OVERHEAD) + KeyValueUtil.length(this);
|
return ClassSize.align(FIXED_OVERHEAD) + this.getSerializedSize();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -48,16 +48,6 @@ public class KeyValueUtil {
|
||||||
|
|
||||||
/**************** length *********************/
|
/**************** length *********************/
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns number of bytes this cell would have been used if serialized as in {@link KeyValue}
|
|
||||||
* @param cell
|
|
||||||
* @return the length
|
|
||||||
*/
|
|
||||||
public static int length(final Cell cell) {
|
|
||||||
return length(cell.getRowLength(), cell.getFamilyLength(), cell.getQualifierLength(),
|
|
||||||
cell.getValueLength(), cell.getTagsLength(), true);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static int length(short rlen, byte flen, int qlen, int vlen, int tlen, boolean withTags) {
|
public static int length(short rlen, byte flen, int qlen, int vlen, int tlen, boolean withTags) {
|
||||||
if (withTags) {
|
if (withTags) {
|
||||||
return (int) (KeyValue.getKeyValueDataStructureSize(rlen, flen, qlen, vlen, tlen));
|
return (int) (KeyValue.getKeyValueDataStructureSize(rlen, flen, qlen, vlen, tlen));
|
||||||
|
@ -134,7 +124,7 @@ public class KeyValueUtil {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static byte[] copyToNewByteArray(final Cell cell) {
|
public static byte[] copyToNewByteArray(final Cell cell) {
|
||||||
int v1Length = length(cell);
|
int v1Length = cell.getSerializedSize();
|
||||||
byte[] backingBytes = new byte[v1Length];
|
byte[] backingBytes = new byte[v1Length];
|
||||||
appendToByteArray(cell, backingBytes, 0, true);
|
appendToByteArray(cell, backingBytes, 0, true);
|
||||||
return backingBytes;
|
return backingBytes;
|
||||||
|
|
|
@ -622,7 +622,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
||||||
// new row
|
// new row
|
||||||
collectRow();
|
collectRow();
|
||||||
}
|
}
|
||||||
curRowBytes += KeyValueUtil.length(cell);
|
curRowBytes += cell.getSerializedSize();
|
||||||
curRowKeyLength = KeyValueUtil.keyLength(cell);
|
curRowKeyLength = KeyValueUtil.keyLength(cell);
|
||||||
curRowCols++;
|
curRowCols++;
|
||||||
prevCell = cell;
|
prevCell = cell;
|
||||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.CellUtil;
|
import org.apache.hadoop.hbase.CellUtil;
|
||||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.CellSink;
|
import org.apache.hadoop.hbase.regionserver.CellSink;
|
||||||
import org.apache.hadoop.hbase.regionserver.HMobStore;
|
import org.apache.hadoop.hbase.regionserver.HMobStore;
|
||||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||||
|
@ -281,7 +280,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
||||||
cellsCountCompactedToMob++;
|
cellsCountCompactedToMob++;
|
||||||
cellsSizeCompactedToMob += c.getValueLength();
|
cellsSizeCompactedToMob += c.getValueLength();
|
||||||
}
|
}
|
||||||
int len = KeyValueUtil.length(c);
|
int len = c.getSerializedSize();
|
||||||
++progress.currentCompactedKVs;
|
++progress.currentCompactedKVs;
|
||||||
progress.totalCompactedSize += len;
|
progress.totalCompactedSize += len;
|
||||||
bytesWrittenProgressForShippedCall += len;
|
bytesWrittenProgressForShippedCall += len;
|
||||||
|
|
|
@ -29,7 +29,6 @@ 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.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||||
import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher;
|
import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher;
|
||||||
import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
|
import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
|
||||||
|
@ -212,9 +211,8 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
|
||||||
this.mobStore.getRefCellTags());
|
this.mobStore.getRefCellTags());
|
||||||
writer.append(reference);
|
writer.append(reference);
|
||||||
}
|
}
|
||||||
int len = KeyValueUtil.length(c);
|
|
||||||
if (control) {
|
if (control) {
|
||||||
throughputController.control(flushName, len);
|
throughputController.control(flushName, c.getSerializedSize());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
cells.clear();
|
cells.clear();
|
||||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
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.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.yetus.audience.InterfaceStability;
|
import org.apache.yetus.audience.InterfaceStability;
|
||||||
|
@ -292,7 +291,7 @@ public class QuotaUtil extends QuotaTableUtil {
|
||||||
long size = 0;
|
long size = 0;
|
||||||
for (Map.Entry<byte[], List<Cell>> entry : mutation.getFamilyCellMap().entrySet()) {
|
for (Map.Entry<byte[], List<Cell>> entry : mutation.getFamilyCellMap().entrySet()) {
|
||||||
for (Cell cell : entry.getValue()) {
|
for (Cell cell : entry.getValue()) {
|
||||||
size += KeyValueUtil.length(cell);
|
size += cell.getSerializedSize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return size;
|
return size;
|
||||||
|
@ -301,7 +300,7 @@ public class QuotaUtil extends QuotaTableUtil {
|
||||||
public static long calculateResultSize(final Result result) {
|
public static long calculateResultSize(final Result result) {
|
||||||
long size = 0;
|
long size = 0;
|
||||||
for (Cell cell : result.rawCells()) {
|
for (Cell cell : result.rawCells()) {
|
||||||
size += KeyValueUtil.length(cell);
|
size += cell.getSerializedSize();
|
||||||
}
|
}
|
||||||
return size;
|
return size;
|
||||||
}
|
}
|
||||||
|
@ -310,7 +309,7 @@ public class QuotaUtil extends QuotaTableUtil {
|
||||||
long size = 0;
|
long size = 0;
|
||||||
for (Result result: results) {
|
for (Result result: results) {
|
||||||
for (Cell cell : result.rawCells()) {
|
for (Cell cell : result.rawCells()) {
|
||||||
size += KeyValueUtil.length(cell);
|
size += cell.getSerializedSize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return size;
|
return size;
|
||||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.CellComparator;
|
||||||
import org.apache.hadoop.hbase.CellUtil;
|
import org.apache.hadoop.hbase.CellUtil;
|
||||||
import org.apache.hadoop.hbase.ExtendedCell;
|
import org.apache.hadoop.hbase.ExtendedCell;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||||
import org.apache.hadoop.hbase.util.ClassSize;
|
import org.apache.hadoop.hbase.util.ClassSize;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
@ -260,7 +259,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
|
||||||
|
|
||||||
offset = ByteBufferUtils.putInt(idxBuffer, offset, dataChunkID); // write data chunk id
|
offset = ByteBufferUtils.putInt(idxBuffer, offset, dataChunkID); // write data chunk id
|
||||||
offset = ByteBufferUtils.putInt(idxBuffer, offset, cell.getOffset()); // offset
|
offset = ByteBufferUtils.putInt(idxBuffer, offset, cell.getOffset()); // offset
|
||||||
offset = ByteBufferUtils.putInt(idxBuffer, offset, KeyValueUtil.length(cell)); // length
|
offset = ByteBufferUtils.putInt(idxBuffer, offset, cell.getSerializedSize()); // length
|
||||||
offset = ByteBufferUtils.putLong(idxBuffer, offset, cell.getSequenceId()); // seqId
|
offset = ByteBufferUtils.putLong(idxBuffer, offset, cell.getSequenceId()); // seqId
|
||||||
|
|
||||||
return offset;
|
return offset;
|
||||||
|
|
|
@ -92,7 +92,6 @@ import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
|
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
|
||||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||||
|
@ -8491,7 +8490,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
int listSize = cells.size();
|
int listSize = cells.size();
|
||||||
for (int i=0; i < listSize; i++) {
|
for (int i=0; i < listSize; i++) {
|
||||||
Cell cell = cells.get(i);
|
Cell cell = cells.get(i);
|
||||||
mutationSize += KeyValueUtil.length(cell);
|
mutationSize += cell.getSerializedSize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -22,13 +22,11 @@ import java.io.IOException;
|
||||||
import java.io.InterruptedIOException;
|
import java.io.InterruptedIOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.OptionalInt;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
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.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||||
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
|
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
|
||||||
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
|
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
|
||||||
|
@ -119,7 +117,8 @@ abstract class StoreFlusher {
|
||||||
boolean hasMore;
|
boolean hasMore;
|
||||||
String flushName = ThroughputControlUtil.getNameForThrottling(store, "flush");
|
String flushName = ThroughputControlUtil.getNameForThrottling(store, "flush");
|
||||||
// no control on system table (such as meta, namespace, etc) flush
|
// no control on system table (such as meta, namespace, etc) flush
|
||||||
boolean control = throughputController != null && !store.getRegionInfo().getTable().isSystemTable();
|
boolean control =
|
||||||
|
throughputController != null && !store.getRegionInfo().getTable().isSystemTable();
|
||||||
if (control) {
|
if (control) {
|
||||||
throughputController.start(flushName);
|
throughputController.start(flushName);
|
||||||
}
|
}
|
||||||
|
@ -132,17 +131,16 @@ abstract class StoreFlusher {
|
||||||
// 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.
|
||||||
sink.append(c);
|
sink.append(c);
|
||||||
int len = KeyValueUtil.length(c);
|
|
||||||
if (control) {
|
if (control) {
|
||||||
throughputController.control(flushName, len);
|
throughputController.control(flushName, c.getSerializedSize());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
kvs.clear();
|
kvs.clear();
|
||||||
}
|
}
|
||||||
} while (hasMore);
|
} while (hasMore);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
throw new InterruptedIOException("Interrupted while control throughput of flushing "
|
throw new InterruptedIOException(
|
||||||
+ flushName);
|
"Interrupted while control throughput of flushing " + flushName);
|
||||||
} finally {
|
} finally {
|
||||||
if (control) {
|
if (control) {
|
||||||
throughputController.finish(flushName);
|
throughputController.finish(flushName);
|
||||||
|
|
|
@ -34,7 +34,6 @@ 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.PrivateCellUtil;
|
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
|
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
|
||||||
|
@ -401,7 +400,7 @@ public abstract class Compactor<T extends CellSink> {
|
||||||
lastCleanCellSeqId = 0;
|
lastCleanCellSeqId = 0;
|
||||||
}
|
}
|
||||||
writer.append(c);
|
writer.append(c);
|
||||||
int len = KeyValueUtil.length(c);
|
int len = c.getSerializedSize();
|
||||||
++progress.currentCompactedKVs;
|
++progress.currentCompactedKVs;
|
||||||
progress.totalCompactedSize += len;
|
progress.totalCompactedSize += len;
|
||||||
bytesWrittenProgressForShippedCall += len;
|
bytesWrittenProgressForShippedCall += len;
|
||||||
|
|
|
@ -295,7 +295,7 @@ public class TestCellFlatSet {
|
||||||
|
|
||||||
for (Cell kv: cellArray) {
|
for (Cell kv: cellArray) {
|
||||||
// do we have enough space to write the cell data on the data chunk?
|
// do we have enough space to write the cell data on the data chunk?
|
||||||
if (dataOffset + KeyValueUtil.length(kv) > chunkCreator.getChunkSize()) {
|
if (dataOffset + kv.getSerializedSize() > chunkCreator.getChunkSize()) {
|
||||||
// allocate more data chunks if needed
|
// allocate more data chunks if needed
|
||||||
dataChunk = chunkCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP);
|
dataChunk = chunkCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP);
|
||||||
dataBuffer = dataChunk.getData();
|
dataBuffer = dataChunk.getData();
|
||||||
|
@ -314,7 +314,7 @@ public class TestCellFlatSet {
|
||||||
}
|
}
|
||||||
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataChunk.getId()); // write data chunk id
|
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataChunk.getId()); // write data chunk id
|
||||||
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataStartOfset); // offset
|
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataStartOfset); // offset
|
||||||
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, KeyValueUtil.length(kv)); // length
|
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, kv.getSerializedSize()); // length
|
||||||
idxOffset = ByteBufferUtils.putLong(idxBuffer, idxOffset, kv.getSequenceId()); // seqId
|
idxOffset = ByteBufferUtils.putLong(idxBuffer, idxOffset, kv.getSequenceId()); // seqId
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -357,7 +357,7 @@ public class TestCellFlatSet {
|
||||||
// write data chunk id
|
// write data chunk id
|
||||||
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataJumboChunk.getId());
|
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataJumboChunk.getId());
|
||||||
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataStartOfset); // offset
|
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, dataStartOfset); // offset
|
||||||
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, KeyValueUtil.length(kv)); // length
|
idxOffset = ByteBufferUtils.putInt(idxBuffer, idxOffset, kv.getSerializedSize()); // length
|
||||||
idxOffset = ByteBufferUtils.putLong(idxBuffer, idxOffset, kv.getSequenceId()); // seqId
|
idxOffset = ByteBufferUtils.putLong(idxBuffer, idxOffset, kv.getSequenceId()); // seqId
|
||||||
|
|
||||||
// Jumbo chunks are working only with one cell per chunk, thus always allocate a new jumbo
|
// Jumbo chunks are working only with one cell per chunk, thus always allocate a new jumbo
|
||||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
|
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||||
|
@ -636,10 +635,8 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
|
||||||
|
|
||||||
// test 1 bucket
|
// test 1 bucket
|
||||||
int totalCellsLen = addRowsByKeys(memstore, keys1);
|
int totalCellsLen = addRowsByKeys(memstore, keys1);
|
||||||
long oneCellOnCSLMHeapSize =
|
long oneCellOnCSLMHeapSize = ClassSize.align(
|
||||||
ClassSize.align(
|
ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + KeyValue.FIXED_OVERHEAD + kv.getSerializedSize());
|
||||||
ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + KeyValue.FIXED_OVERHEAD + KeyValueUtil
|
|
||||||
.length(kv));
|
|
||||||
|
|
||||||
long totalHeapSize = numOfCells * oneCellOnCSLMHeapSize + MutableSegment.DEEP_OVERHEAD;
|
long totalHeapSize = numOfCells * oneCellOnCSLMHeapSize + MutableSegment.DEEP_OVERHEAD;
|
||||||
assertEquals(totalCellsLen, regionServicesForStores.getMemStoreSize());
|
assertEquals(totalCellsLen, regionServicesForStores.getMemStoreSize());
|
||||||
|
@ -648,7 +645,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
|
||||||
((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and flatten
|
((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and flatten
|
||||||
assertEquals(0, memstore.getSnapshot().getCellsCount());
|
assertEquals(0, memstore.getSnapshot().getCellsCount());
|
||||||
long oneCellOnCCMHeapSize =
|
long oneCellOnCCMHeapSize =
|
||||||
ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(KeyValueUtil.length(kv));
|
ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(kv.getSerializedSize());
|
||||||
totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM
|
totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM
|
||||||
+ numOfCells * oneCellOnCCMHeapSize;
|
+ numOfCells * oneCellOnCCMHeapSize;
|
||||||
|
|
||||||
|
@ -721,7 +718,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
|
||||||
// One cell is duplicated, but it shouldn't be compacted because we are in BASIC mode.
|
// One cell is duplicated, but it shouldn't be compacted because we are in BASIC mode.
|
||||||
// totalCellsLen should remain the same
|
// totalCellsLen should remain the same
|
||||||
long oneCellOnCCMHeapSize =
|
long oneCellOnCCMHeapSize =
|
||||||
ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(KeyValueUtil.length(kv));
|
ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(kv.getSerializedSize());
|
||||||
totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM
|
totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM
|
||||||
+ numOfCells * oneCellOnCCMHeapSize;
|
+ numOfCells * oneCellOnCCMHeapSize;
|
||||||
|
|
||||||
|
@ -796,7 +793,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
|
||||||
// One cell is duplicated, but it shouldn't be compacted because we are in BASIC mode.
|
// One cell is duplicated, but it shouldn't be compacted because we are in BASIC mode.
|
||||||
// totalCellsLen should remain the same
|
// totalCellsLen should remain the same
|
||||||
long oneCellOnCCMHeapSize =
|
long oneCellOnCCMHeapSize =
|
||||||
(long) ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(KeyValueUtil.length(kv));
|
(long) ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(kv.getSerializedSize());
|
||||||
totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM
|
totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM
|
||||||
+ numOfCells * oneCellOnCCMHeapSize;
|
+ numOfCells * oneCellOnCCMHeapSize;
|
||||||
|
|
||||||
|
@ -876,7 +873,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
|
||||||
KeyValue kv = new KeyValue(Bytes.toBytes("A"), Bytes.toBytes("testfamily"),
|
KeyValue kv = new KeyValue(Bytes.toBytes("A"), Bytes.toBytes("testfamily"),
|
||||||
Bytes.toBytes("testqualifier"), System.currentTimeMillis(), val);
|
Bytes.toBytes("testqualifier"), System.currentTimeMillis(), val);
|
||||||
long oneCellOnCCMHeapSize =
|
long oneCellOnCCMHeapSize =
|
||||||
(long) ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(KeyValueUtil.length(kv));
|
(long) ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(kv.getSerializedSize());
|
||||||
long oneCellOnCSLMHeapSize =
|
long oneCellOnCSLMHeapSize =
|
||||||
ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize());
|
ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize());
|
||||||
long totalHeapSize = MutableSegment.DEEP_OVERHEAD;
|
long totalHeapSize = MutableSegment.DEEP_OVERHEAD;
|
||||||
|
@ -932,7 +929,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
|
||||||
new KeyValue(row, Bytes.toBytes("testfamily"), Bytes.toBytes("testqualifier"),
|
new KeyValue(row, Bytes.toBytes("testfamily"), Bytes.toBytes("testqualifier"),
|
||||||
System.currentTimeMillis(), val);
|
System.currentTimeMillis(), val);
|
||||||
return ClassSize.align(
|
return ClassSize.align(
|
||||||
ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + KeyValue.FIXED_OVERHEAD + KeyValueUtil.length(kv));
|
ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + KeyValue.FIXED_OVERHEAD + kv.getSerializedSize());
|
||||||
}
|
}
|
||||||
|
|
||||||
private long cellAfterFlushSize() {
|
private long cellAfterFlushSize() {
|
||||||
|
@ -945,8 +942,8 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
|
||||||
|
|
||||||
return toCellChunkMap ?
|
return toCellChunkMap ?
|
||||||
ClassSize.align(
|
ClassSize.align(
|
||||||
ClassSize.CELL_CHUNK_MAP_ENTRY + KeyValueUtil.length(kv)) :
|
ClassSize.CELL_CHUNK_MAP_ENTRY + kv.getSerializedSize()) :
|
||||||
ClassSize.align(
|
ClassSize.align(
|
||||||
ClassSize.CELL_ARRAY_MAP_ENTRY + KeyValue.FIXED_OVERHEAD + KeyValueUtil.length(kv));
|
ClassSize.CELL_ARRAY_MAP_ENTRY + KeyValue.FIXED_OVERHEAD + kv.getSerializedSize());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -86,7 +86,6 @@ import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||||
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.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||||
import org.apache.hadoop.hbase.MultithreadedTestUtil;
|
import org.apache.hadoop.hbase.MultithreadedTestUtil;
|
||||||
import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
|
import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
|
||||||
|
@ -2365,11 +2364,11 @@ public class TestHRegion {
|
||||||
|
|
||||||
Cell originalCell = CellUtil.createCell(row, COLUMN_FAMILY_BYTES, qual1,
|
Cell originalCell = CellUtil.createCell(row, COLUMN_FAMILY_BYTES, qual1,
|
||||||
System.currentTimeMillis(), KeyValue.Type.Put.getCode(), value1);
|
System.currentTimeMillis(), KeyValue.Type.Put.getCode(), value1);
|
||||||
final long originalSize = KeyValueUtil.length(originalCell);
|
final long originalSize = originalCell.getSerializedSize();
|
||||||
|
|
||||||
Cell addCell = CellUtil.createCell(row, COLUMN_FAMILY_BYTES, qual1,
|
Cell addCell = CellUtil.createCell(row, COLUMN_FAMILY_BYTES, qual1,
|
||||||
System.currentTimeMillis(), KeyValue.Type.Put.getCode(), Bytes.toBytes("xxxxxxxxxx"));
|
System.currentTimeMillis(), KeyValue.Type.Put.getCode(), Bytes.toBytes("xxxxxxxxxx"));
|
||||||
final long addSize = KeyValueUtil.length(addCell);
|
final long addSize = addCell.getSerializedSize();
|
||||||
|
|
||||||
LOG.info("originalSize:" + originalSize
|
LOG.info("originalSize:" + originalSize
|
||||||
+ ", addSize:" + addSize);
|
+ ", addSize:" + addSize);
|
||||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
|
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
|
||||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||||
|
@ -92,7 +91,7 @@ public class TestMemStoreChunkPool {
|
||||||
for (int i = 0; i < 100; i++) {
|
for (int i = 0; i < 100; i++) {
|
||||||
int valSize = rand.nextInt(1000);
|
int valSize = rand.nextInt(1000);
|
||||||
KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
|
KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
|
||||||
int size = KeyValueUtil.length(kv);
|
int size = kv.getSerializedSize();
|
||||||
ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
|
ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
|
||||||
if (newKv.getBuffer() != lastBuffer) {
|
if (newKv.getBuffer() != lastBuffer) {
|
||||||
expectedOff = 4;
|
expectedOff = 4;
|
||||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.MultithreadedTestUtil;
|
import org.apache.hadoop.hbase.MultithreadedTestUtil;
|
||||||
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
|
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
|
||||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||||
|
@ -94,7 +93,7 @@ public class TestMemStoreLAB {
|
||||||
for (int i = 0; i < 100000; i++) {
|
for (int i = 0; i < 100000; i++) {
|
||||||
int valSize = rand.nextInt(3);
|
int valSize = rand.nextInt(3);
|
||||||
KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
|
KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
|
||||||
int size = KeyValueUtil.length(kv);
|
int size = kv.getSerializedSize();
|
||||||
ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
|
ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
|
||||||
if (newKv.getBuffer() != lastBuffer) {
|
if (newKv.getBuffer() != lastBuffer) {
|
||||||
// since we add the chunkID at the 0th offset of the chunk and the
|
// since we add the chunkID at the 0th offset of the chunk and the
|
||||||
|
@ -145,7 +144,7 @@ public class TestMemStoreLAB {
|
||||||
public void doAnAction() throws Exception {
|
public void doAnAction() throws Exception {
|
||||||
int valSize = r.nextInt(3);
|
int valSize = r.nextInt(3);
|
||||||
KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
|
KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
|
||||||
int size = KeyValueUtil.length(kv);
|
int size = kv.getSerializedSize();
|
||||||
ByteBufferKeyValue newCell = (ByteBufferKeyValue) mslab.copyCellInto(kv);
|
ByteBufferKeyValue newCell = (ByteBufferKeyValue) mslab.copyCellInto(kv);
|
||||||
totalAllocated.addAndGet(size);
|
totalAllocated.addAndGet(size);
|
||||||
allocsByThisThread.add(new AllocRecord(newCell.getBuffer(), newCell.getOffset(), size));
|
allocsByThisThread.add(new AllocRecord(newCell.getBuffer(), newCell.getOffset(), size));
|
||||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -80,7 +79,7 @@ public class TestMemstoreLABWithoutPool {
|
||||||
for (int i = 0; i < 100000; i++) {
|
for (int i = 0; i < 100000; i++) {
|
||||||
int valSize = rand.nextInt(1000);
|
int valSize = rand.nextInt(1000);
|
||||||
KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
|
KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
|
||||||
int size = KeyValueUtil.length(kv);
|
int size = kv.getSerializedSize();
|
||||||
ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
|
ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
|
||||||
if (newKv.getBuffer() != lastBuffer) {
|
if (newKv.getBuffer() != lastBuffer) {
|
||||||
// since we add the chunkID at the 0th offset of the chunk and the
|
// since we add the chunkID at the 0th offset of the chunk and the
|
||||||
|
|
Loading…
Reference in New Issue