diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 768da76..8ae0695 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.io.hfile.CellStatsTracker;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
@@ -86,6 +87,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
+import org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
@@ -2964,4 +2966,18 @@ public final class ProtobufUtil {
.setScope(toProtoQuotaScope(scope))
.build();
}
+
+ public static CellStatsTracker ToCellStatsTracker(CellTrackerProtos.CellStatsTracker proto) throws IOException {
+ return new CellStatsTracker(proto.getColCount(), proto.getRowCount(), proto.getAccColSize(),
+ proto.getAccRowSize(), proto.getMaxColSize(), proto.getMaxRowSize());
+ }
+
+ public static CellTrackerProtos.CellStatsTracker toCellStatsTracker(CellStatsTracker tracker) throws IOException {
+ CellTrackerProtos.CellStatsTracker.Builder builder = CellTrackerProtos.CellStatsTracker.newBuilder();
+ builder.setAccColSize(tracker.getTotalColSize()).setAccRowSize(tracker.getTotalRowSize())
+ .setColCount(tracker.getColCount()).setRowCount(tracker.getRowCount())
+ .setMaxColSize(tracker.getMaxColSize()).setMaxRowSize(tracker.getMaxRowSize());
+ return builder.build();
+ }
+
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/CellStatsTracker.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/CellStatsTracker.java
new file mode 100644
index 0000000..e69706f
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/CellStatsTracker.java
@@ -0,0 +1,94 @@
+package org.apache.hadoop.hbase.io.hfile;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+
+public class CellStatsTracker {
+ private Cell row = null;
+ private Cell col = null;
+
+ private long colCount;
+ private long rowCount;
+ private long accColSize;
+ private long accRowSize;
+
+ private long curColSize = 0;
+ private long curRowSize = 0;
+
+ private long maxColSize;
+ private long maxRowSize;
+
+ private int FIXED_CELL_LENGTH = 16; // is this right?
+
+ public CellStatsTracker() {}
+
+ public CellStatsTracker(long colCount, long rowCount, long accColSize, long accRowSize, long maxColSize, long maxRowSize) {
+ this.colCount = colCount;
+ this.rowCount = rowCount;
+ this.accColSize = accColSize;
+ this.accRowSize = accRowSize;
+ this.maxColSize = maxColSize;
+ this.maxRowSize = maxRowSize;
+ }
+
+ public void combine(CellStatsTracker other) {
+ this.colCount += other.colCount;
+ this.rowCount += other.rowCount;
+ this.accColSize += other.accColSize;
+ this.accRowSize += other.accRowSize;
+ this.maxColSize = Math.max(maxColSize, other.maxColSize);
+ this.maxRowSize = Math.max(maxRowSize, other.maxRowSize);
+ }
+
+ public void track(Cell cell) {
+ int length = cell.getRowLength() + cell.getFamilyLength() + cell.getQualifierLength() + cell.getValueLength() + FIXED_CELL_LENGTH;
+ if (row == null || !CellUtil.matchingRow(cell, row)) {
+ rowCount++;
+
+ accRowSize += curRowSize;
+ maxRowSize = Math.max(maxRowSize, curRowSize);
+ curRowSize = 0;
+
+ accColSize += curColSize;
+ maxColSize = Math.max(maxColSize, curColSize);
+ curColSize = 0;
+
+ row = cell;
+ col = cell;
+ } else if (col == null || !CellUtil.matchingFamily(cell, col) || !CellUtil.matchingColumn(cell, col)) {
+ colCount++;
+
+ accColSize += curColSize;
+ maxColSize = Math.max(maxColSize, curColSize);
+ curColSize = 0;
+
+ col = cell;
+ }
+ curRowSize += length;
+ curColSize += length;
+ }
+
+ public long getMaxColSize() {
+ return maxColSize;
+ }
+
+ public long getMaxRowSize() {
+ return maxRowSize;
+ }
+
+ public long getTotalColSize() {
+ return accColSize;
+ }
+
+ public long getTotalRowSize() {
+ return accRowSize;
+ }
+
+ public long getRowCount() {
+ return rowCount;
+ }
+
+ public long getColCount() {
+ return colCount;
+ }
+}
diff --git a/hbase-protocol/pom.xml b/hbase-protocol/pom.xml
index 297a7af..507b1b7 100644
--- a/hbase-protocol/pom.xml
+++ b/hbase-protocol/pom.xml
@@ -158,6 +158,7 @@
Aggregate.proto
Authentication.proto
Cell.proto
+ CellTracker.proto
Client.proto
ClusterId.proto
ClusterStatus.proto
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/CellTrackerProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/CellTrackerProtos.java
new file mode 100644
index 0000000..fa5db25
--- /dev/null
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/CellTrackerProtos.java
@@ -0,0 +1,902 @@
+// Generated by the protocol buffer compiler. DO NOT EDIT!
+// source: CellTracker.proto
+
+package org.apache.hadoop.hbase.protobuf.generated;
+
+public final class CellTrackerProtos {
+ private CellTrackerProtos() {}
+ public static void registerAllExtensions(
+ com.google.protobuf.ExtensionRegistry registry) {
+ }
+ public interface CellStatsTrackerOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // required int64 row_count = 1;
+ /**
+ * required int64 row_count = 1;
+ */
+ boolean hasRowCount();
+ /**
+ * required int64 row_count = 1;
+ */
+ long getRowCount();
+
+ // required int64 col_count = 2;
+ /**
+ * required int64 col_count = 2;
+ */
+ boolean hasColCount();
+ /**
+ * required int64 col_count = 2;
+ */
+ long getColCount();
+
+ // required int64 acc_col_size = 3;
+ /**
+ * required int64 acc_col_size = 3;
+ */
+ boolean hasAccColSize();
+ /**
+ * required int64 acc_col_size = 3;
+ */
+ long getAccColSize();
+
+ // required int64 acc_row_size = 4;
+ /**
+ * required int64 acc_row_size = 4;
+ */
+ boolean hasAccRowSize();
+ /**
+ * required int64 acc_row_size = 4;
+ */
+ long getAccRowSize();
+
+ // required int64 max_col_size = 5;
+ /**
+ * required int64 max_col_size = 5;
+ */
+ boolean hasMaxColSize();
+ /**
+ * required int64 max_col_size = 5;
+ */
+ long getMaxColSize();
+
+ // required int64 max_row_size = 6;
+ /**
+ * required int64 max_row_size = 6;
+ */
+ boolean hasMaxRowSize();
+ /**
+ * required int64 max_row_size = 6;
+ */
+ long getMaxRowSize();
+ }
+ /**
+ * Protobuf type {@code CellStatsTracker}
+ */
+ public static final class CellStatsTracker extends
+ com.google.protobuf.GeneratedMessage
+ implements CellStatsTrackerOrBuilder {
+ // Use CellStatsTracker.newBuilder() to construct.
+ private CellStatsTracker(com.google.protobuf.GeneratedMessage.Builder> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private CellStatsTracker(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final CellStatsTracker defaultInstance;
+ public static CellStatsTracker getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public CellStatsTracker getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private CellStatsTracker(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 8: {
+ bitField0_ |= 0x00000001;
+ rowCount_ = input.readInt64();
+ break;
+ }
+ case 16: {
+ bitField0_ |= 0x00000002;
+ colCount_ = input.readInt64();
+ break;
+ }
+ case 24: {
+ bitField0_ |= 0x00000004;
+ accColSize_ = input.readInt64();
+ break;
+ }
+ case 32: {
+ bitField0_ |= 0x00000008;
+ accRowSize_ = input.readInt64();
+ break;
+ }
+ case 40: {
+ bitField0_ |= 0x00000010;
+ maxColSize_ = input.readInt64();
+ break;
+ }
+ case 48: {
+ bitField0_ |= 0x00000020;
+ maxRowSize_ = input.readInt64();
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.internal_static_CellStatsTracker_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.internal_static_CellStatsTracker_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker.class, org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser PARSER =
+ new com.google.protobuf.AbstractParser() {
+ public CellStatsTracker parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new CellStatsTracker(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // required int64 row_count = 1;
+ public static final int ROW_COUNT_FIELD_NUMBER = 1;
+ private long rowCount_;
+ /**
+ * required int64 row_count = 1;
+ */
+ public boolean hasRowCount() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * required int64 row_count = 1;
+ */
+ public long getRowCount() {
+ return rowCount_;
+ }
+
+ // required int64 col_count = 2;
+ public static final int COL_COUNT_FIELD_NUMBER = 2;
+ private long colCount_;
+ /**
+ * required int64 col_count = 2;
+ */
+ public boolean hasColCount() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * required int64 col_count = 2;
+ */
+ public long getColCount() {
+ return colCount_;
+ }
+
+ // required int64 acc_col_size = 3;
+ public static final int ACC_COL_SIZE_FIELD_NUMBER = 3;
+ private long accColSize_;
+ /**
+ * required int64 acc_col_size = 3;
+ */
+ public boolean hasAccColSize() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * required int64 acc_col_size = 3;
+ */
+ public long getAccColSize() {
+ return accColSize_;
+ }
+
+ // required int64 acc_row_size = 4;
+ public static final int ACC_ROW_SIZE_FIELD_NUMBER = 4;
+ private long accRowSize_;
+ /**
+ * required int64 acc_row_size = 4;
+ */
+ public boolean hasAccRowSize() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * required int64 acc_row_size = 4;
+ */
+ public long getAccRowSize() {
+ return accRowSize_;
+ }
+
+ // required int64 max_col_size = 5;
+ public static final int MAX_COL_SIZE_FIELD_NUMBER = 5;
+ private long maxColSize_;
+ /**
+ * required int64 max_col_size = 5;
+ */
+ public boolean hasMaxColSize() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ /**
+ * required int64 max_col_size = 5;
+ */
+ public long getMaxColSize() {
+ return maxColSize_;
+ }
+
+ // required int64 max_row_size = 6;
+ public static final int MAX_ROW_SIZE_FIELD_NUMBER = 6;
+ private long maxRowSize_;
+ /**
+ * required int64 max_row_size = 6;
+ */
+ public boolean hasMaxRowSize() {
+ return ((bitField0_ & 0x00000020) == 0x00000020);
+ }
+ /**
+ * required int64 max_row_size = 6;
+ */
+ public long getMaxRowSize() {
+ return maxRowSize_;
+ }
+
+ private void initFields() {
+ rowCount_ = 0L;
+ colCount_ = 0L;
+ accColSize_ = 0L;
+ accRowSize_ = 0L;
+ maxColSize_ = 0L;
+ maxRowSize_ = 0L;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasRowCount()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasColCount()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasAccColSize()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasAccRowSize()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasMaxColSize()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasMaxRowSize()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeInt64(1, rowCount_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeInt64(2, colCount_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeInt64(3, accColSize_);
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ output.writeInt64(4, accRowSize_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ output.writeInt64(5, maxColSize_);
+ }
+ if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ output.writeInt64(6, maxRowSize_);
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(1, rowCount_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(2, colCount_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(3, accColSize_);
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(4, accRowSize_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(5, maxColSize_);
+ }
+ if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(6, maxRowSize_);
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code CellStatsTracker}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder
+ implements org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTrackerOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.internal_static_CellStatsTracker_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.internal_static_CellStatsTracker_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker.class, org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker.Builder.class);
+ }
+
+ // Construct using org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ rowCount_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ colCount_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ accColSize_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ accRowSize_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000008);
+ maxColSize_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000010);
+ maxRowSize_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000020);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.internal_static_CellStatsTracker_descriptor;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker.getDefaultInstance();
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker build() {
+ org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker buildPartial() {
+ org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker result = new org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.rowCount_ = rowCount_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.colCount_ = colCount_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.accColSize_ = accColSize_;
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000008;
+ }
+ result.accRowSize_ = accRowSize_;
+ if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+ to_bitField0_ |= 0x00000010;
+ }
+ result.maxColSize_ = maxColSize_;
+ if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+ to_bitField0_ |= 0x00000020;
+ }
+ result.maxRowSize_ = maxRowSize_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker) {
+ return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker other) {
+ if (other == org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker.getDefaultInstance()) return this;
+ if (other.hasRowCount()) {
+ setRowCount(other.getRowCount());
+ }
+ if (other.hasColCount()) {
+ setColCount(other.getColCount());
+ }
+ if (other.hasAccColSize()) {
+ setAccColSize(other.getAccColSize());
+ }
+ if (other.hasAccRowSize()) {
+ setAccRowSize(other.getAccRowSize());
+ }
+ if (other.hasMaxColSize()) {
+ setMaxColSize(other.getMaxColSize());
+ }
+ if (other.hasMaxRowSize()) {
+ setMaxRowSize(other.getMaxRowSize());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasRowCount()) {
+
+ return false;
+ }
+ if (!hasColCount()) {
+
+ return false;
+ }
+ if (!hasAccColSize()) {
+
+ return false;
+ }
+ if (!hasAccRowSize()) {
+
+ return false;
+ }
+ if (!hasMaxColSize()) {
+
+ return false;
+ }
+ if (!hasMaxRowSize()) {
+
+ return false;
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.CellTrackerProtos.CellStatsTracker) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // required int64 row_count = 1;
+ private long rowCount_ ;
+ /**
+ * required int64 row_count = 1;
+ */
+ public boolean hasRowCount() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * required int64 row_count = 1;
+ */
+ public long getRowCount() {
+ return rowCount_;
+ }
+ /**
+ * required int64 row_count = 1;
+ */
+ public Builder setRowCount(long value) {
+ bitField0_ |= 0x00000001;
+ rowCount_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * required int64 row_count = 1;
+ */
+ public Builder clearRowCount() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ rowCount_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // required int64 col_count = 2;
+ private long colCount_ ;
+ /**
+ * required int64 col_count = 2;
+ */
+ public boolean hasColCount() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * required int64 col_count = 2;
+ */
+ public long getColCount() {
+ return colCount_;
+ }
+ /**
+ * required int64 col_count = 2;
+ */
+ public Builder setColCount(long value) {
+ bitField0_ |= 0x00000002;
+ colCount_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * required int64 col_count = 2;
+ */
+ public Builder clearColCount() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ colCount_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // required int64 acc_col_size = 3;
+ private long accColSize_ ;
+ /**
+ * required int64 acc_col_size = 3;
+ */
+ public boolean hasAccColSize() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * required int64 acc_col_size = 3;
+ */
+ public long getAccColSize() {
+ return accColSize_;
+ }
+ /**
+ * required int64 acc_col_size = 3;
+ */
+ public Builder setAccColSize(long value) {
+ bitField0_ |= 0x00000004;
+ accColSize_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * required int64 acc_col_size = 3;
+ */
+ public Builder clearAccColSize() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ accColSize_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // required int64 acc_row_size = 4;
+ private long accRowSize_ ;
+ /**
+ * required int64 acc_row_size = 4;
+ */
+ public boolean hasAccRowSize() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * required int64 acc_row_size = 4;
+ */
+ public long getAccRowSize() {
+ return accRowSize_;
+ }
+ /**
+ * required int64 acc_row_size = 4;
+ */
+ public Builder setAccRowSize(long value) {
+ bitField0_ |= 0x00000008;
+ accRowSize_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * required int64 acc_row_size = 4;
+ */
+ public Builder clearAccRowSize() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ accRowSize_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // required int64 max_col_size = 5;
+ private long maxColSize_ ;
+ /**
+ * required int64 max_col_size = 5;
+ */
+ public boolean hasMaxColSize() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ /**
+ * required int64 max_col_size = 5;
+ */
+ public long getMaxColSize() {
+ return maxColSize_;
+ }
+ /**
+ * required int64 max_col_size = 5;
+ */
+ public Builder setMaxColSize(long value) {
+ bitField0_ |= 0x00000010;
+ maxColSize_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * required int64 max_col_size = 5;
+ */
+ public Builder clearMaxColSize() {
+ bitField0_ = (bitField0_ & ~0x00000010);
+ maxColSize_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // required int64 max_row_size = 6;
+ private long maxRowSize_ ;
+ /**
+ * required int64 max_row_size = 6;
+ */
+ public boolean hasMaxRowSize() {
+ return ((bitField0_ & 0x00000020) == 0x00000020);
+ }
+ /**
+ * required int64 max_row_size = 6;
+ */
+ public long getMaxRowSize() {
+ return maxRowSize_;
+ }
+ /**
+ * required int64 max_row_size = 6;
+ */
+ public Builder setMaxRowSize(long value) {
+ bitField0_ |= 0x00000020;
+ maxRowSize_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * required int64 max_row_size = 6;
+ */
+ public Builder clearMaxRowSize() {
+ bitField0_ = (bitField0_ & ~0x00000020);
+ maxRowSize_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:CellStatsTracker)
+ }
+
+ static {
+ defaultInstance = new CellStatsTracker(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:CellStatsTracker)
+ }
+
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_CellStatsTracker_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_CellStatsTracker_fieldAccessorTable;
+
+ public static com.google.protobuf.Descriptors.FileDescriptor
+ getDescriptor() {
+ return descriptor;
+ }
+ private static com.google.protobuf.Descriptors.FileDescriptor
+ descriptor;
+ static {
+ java.lang.String[] descriptorData = {
+ "\n\021CellTracker.proto\"\220\001\n\020CellStatsTracker" +
+ "\022\021\n\trow_count\030\001 \002(\003\022\021\n\tcol_count\030\002 \002(\003\022\024" +
+ "\n\014acc_col_size\030\003 \002(\003\022\024\n\014acc_row_size\030\004 \002" +
+ "(\003\022\024\n\014max_col_size\030\005 \002(\003\022\024\n\014max_row_size" +
+ "\030\006 \002(\003B?\n*org.apache.hadoop.hbase.protob" +
+ "uf.generatedB\021CellTrackerProtos"
+ };
+ com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+ new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+ public com.google.protobuf.ExtensionRegistry assignDescriptors(
+ com.google.protobuf.Descriptors.FileDescriptor root) {
+ descriptor = root;
+ internal_static_CellStatsTracker_descriptor =
+ getDescriptor().getMessageTypes().get(0);
+ internal_static_CellStatsTracker_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_CellStatsTracker_descriptor,
+ new java.lang.String[] { "RowCount", "ColCount", "AccColSize", "AccRowSize", "MaxColSize", "MaxRowSize", });
+ return null;
+ }
+ };
+ com.google.protobuf.Descriptors.FileDescriptor
+ .internalBuildGeneratedFileFrom(descriptorData,
+ new com.google.protobuf.Descriptors.FileDescriptor[] {
+ }, assigner);
+ }
+
+ // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/hbase-protocol/src/main/protobuf/CellTracker.proto b/hbase-protocol/src/main/protobuf/CellTracker.proto
new file mode 100644
index 0000000..5ce98bf
--- /dev/null
+++ b/hbase-protocol/src/main/protobuf/CellTracker.proto
@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "CellTrackerProtos";
+
+ message CellStatsTracker {
+ required int64 row_count = 1;
+ required int64 col_count = 2;
+ required int64 acc_col_size = 3;
+ required int64 acc_row_size = 4;
+ required int64 max_col_size = 5;
+ required int64 max_row_size = 6;
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index 3e139a5..f99e244 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -48,10 +48,12 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.CellStatsTracker;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
import org.apache.hadoop.hbase.util.BloomFilter;
import org.apache.hadoop.hbase.util.BloomFilterFactory;
@@ -109,6 +111,9 @@ public class StoreFile {
/** Key for Timerange information in metadata*/
public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
+ /** Key for Cell Stats information in metadata*/
+ public static final byte[] CELLSTATS_KEY = Bytes.toBytes("CELLSTATS");
+
/** Key for timestamp of earliest-put in metadata*/
public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
@@ -708,6 +713,7 @@ public class StoreFile {
protected int bytesPerChecksum;
TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
+ CellStatsTracker cellStatsTracker = new CellStatsTracker();
/* isTimeRangeTrackerSet keeps track if the timeRange has already been set
* When flushing a memstore, we set TimeRange and use this variable to
* indicate that it doesn't need to be calculated again while
@@ -795,6 +801,7 @@ public class StoreFile {
public void appendTrackedTimestampsToMetadata() throws IOException {
appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+ appendFileInfo(CELLSTATS_KEY, ProtobufUtil.toCellStatsTracker(cellStatsTracker).toByteArray());
}
/**
@@ -820,6 +827,7 @@ public class StoreFile {
if (!isTimeRangeTrackerSet) {
timeRangeTracker.includeTimestamp(cell);
}
+ cellStatsTracker.track(cell);
}
private void appendGeneralBloomfilter(final Cell cell) throws IOException {