From 10b5f1667df97649ea7ffdf8d1fe4d2c776d4d55 Mon Sep 17 00:00:00 2001 From: rahulgidwani Date: Tue, 20 Oct 2015 15:46:58 -0700 Subject: [PATCH] HBASE-14355 Scan different TimeRange for each column family --- .../java/org/apache/hadoop/hbase/client/Get.java | 25 + .../java/org/apache/hadoop/hbase/client/Scan.java | 33 + .../apache/hadoop/hbase/protobuf/ProtobufUtil.java | 95 +- .../hbase/protobuf/generated/ClientProtos.java | 1007 +++++++++++++++++--- .../hbase/protobuf/generated/HBaseProtos.java | 779 ++++++++++++++- hbase-protocol/src/main/protobuf/Client.proto | 2 + hbase-protocol/src/main/protobuf/HBase.proto | 6 + .../hadoop/hbase/regionserver/StoreFile.java | 18 +- .../hbase/regionserver/StoreFileScanner.java | 36 +- .../hadoop/hbase/regionserver/TestStoreFile.java | 50 +- 10 files changed, 1821 insertions(+), 230 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java index c71ee0d..8e7dda2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java @@ -79,6 +79,7 @@ public class Get extends Query private boolean closestRowBefore = false; private Map> familyMap = new TreeMap>(Bytes.BYTES_COMPARATOR); + private Map cftr = new TreeMap(Bytes.BYTES_COMPARATOR); /** * Create a Get operation for the specified row. @@ -109,6 +110,7 @@ public class Get extends Query this.storeLimit = get.getMaxResultsPerColumnFamily(); this.storeOffset = get.getRowOffsetPerColumnFamily(); this.tr = get.getTimeRange(); + this.cftr = get.getColumnFamilyTimeRange(); this.checkExistenceOnly = get.isCheckExistenceOnly(); Map> fams = get.getFamilyMap(); for (Map.Entry> entry : fams.entrySet()) { @@ -205,6 +207,21 @@ public class Get extends Query } /** + * Get versions of columns only within the specified timestamp range and column family, + * [cf, minStamp, maxStamp). + * @param cf the column family to restrict + * @param minStamp minimum timestamp value, inclusive + * @param maxStamp maximum timestamp value, exclusive + * @throws IOException if invalid time range + * @return this for invocation chaining + */ + public Get setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp) + throws IOException { + cftr.put(cf, new TimeRange(minStamp, maxStamp)); + return this; + } + + /** * Get versions of columns with the specified timestamp. * @param timestamp version timestamp * @return this for invocation chaining @@ -341,6 +358,14 @@ public class Get extends Query } /** + * Method for retrieving the get's TimeRange by Column Family + * @return Map + */ + public Map getColumnFamilyTimeRange() { + return this.cftr; + } + + /** * Method for retrieving the keys in the familyMap * @return keys in the current familyMap */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java index 9d46bc7..9f30bd2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java @@ -29,6 +29,7 @@ import java.util.NavigableSet; import java.util.TreeMap; import java.util.TreeSet; +import com.google.common.collect.Maps; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HConstants; @@ -137,6 +138,7 @@ public class Scan extends Query { private TimeRange tr = new TimeRange(); private Map> familyMap = new TreeMap>(Bytes.BYTES_COMPARATOR); + private Map colFamTimeRangeMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); private Boolean loadColumnFamiliesOnDemand = null; private Boolean asyncPrefetch = null; @@ -232,6 +234,7 @@ public class Scan extends Query { small = scan.isSmall(); TimeRange ctr = scan.getTimeRange(); tr = new TimeRange(ctr.getMin(), ctr.getMax()); + colFamTimeRangeMap = scan.getColumnFamilyTimeRange(); Map> fams = scan.getFamilyMap(); for (Map.Entry> entry : fams.entrySet()) { byte [] fam = entry.getKey(); @@ -262,6 +265,7 @@ public class Scan extends Query { this.storeLimit = get.getMaxResultsPerColumnFamily(); this.storeOffset = get.getRowOffsetPerColumnFamily(); this.tr = get.getTimeRange(); + this.colFamTimeRangeMap = get.getColumnFamilyTimeRange(); this.familyMap = get.getFamilyMap(); this.getScan = true; this.asyncPrefetch = false; @@ -332,6 +336,28 @@ public class Scan extends Query { } /** + * Get versions of columns only within the specified timestamp range, + * [minStamp, maxStamp) on a per CF bases. Note, default maximum versions to return is 1. If + * your time range spans more than one version and you want all versions + * returned, up the number of versions beyond the default. + * Column Family time ranges take precedence over the global time range. + * + * @param cf the column family for which you want to restrict + * @param minStamp minimum timestamp value, inclusive + * @param maxStamp maximum timestamp value, exclusive + * @return this + * @throws IOException if invalid time range + * @see #setMaxVersions() + * @see #setMaxVersions(int) + */ + + public Scan setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp) throws IOException { + colFamTimeRangeMap.put(cf, new TimeRange(minStamp, maxStamp)); + return this; + } + + + /** * Get versions of columns with the specified timestamp. Note, default maximum * versions to return is 1. If your time range spans more than one version * and you want all versions returned, up the number of versions beyond the @@ -638,6 +664,13 @@ public class Scan extends Query { } /** + * @return Map a map of column families to time ranges + */ + public Map getColumnFamilyTimeRange() { + return this.colFamTimeRangeMap; + } + + /** * @return RowFilter */ @Override 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 21bf6aa..36bf4c9 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 @@ -38,6 +38,7 @@ import java.util.Map.Entry; import java.util.NavigableSet; import java.util.concurrent.TimeUnit; +import com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -459,17 +460,16 @@ public final class ProtobufUtil { if (proto.hasStoreOffset()) { get.setRowOffsetPerColumnFamily(proto.getStoreOffset()); } - if (proto.hasTimeRange()) { - HBaseProtos.TimeRange timeRange = proto.getTimeRange(); - long minStamp = 0; - long maxStamp = Long.MAX_VALUE; - if (timeRange.hasFrom()) { - minStamp = timeRange.getFrom(); - } - if (timeRange.hasTo()) { - maxStamp = timeRange.getTo(); + if (proto.getCfTimeRangeCount() > 0) { + for (HBaseProtos.ColumnFamilyTimeRange cftr : proto.getCfTimeRangeList()) { + TimeRange timeRange = protoToTimeRange(cftr.getTimeRange()); + get.setColumnFamilyTimeRange(cftr.getColumnFamily().toByteArray(), + timeRange.getMin(), timeRange.getMax()); } - get.setTimeRange(minStamp, maxStamp); + } + if (proto.hasTimeRange()) { + TimeRange timeRange = protoToTimeRange(proto.getTimeRange()); + get.setTimeRange(timeRange.getMin(), timeRange.getMax()); } if (proto.hasFilter()) { FilterProtos.Filter filter = proto.getFilter(); @@ -830,16 +830,8 @@ public final class ProtobufUtil { } } if (proto.hasTimeRange()) { - HBaseProtos.TimeRange timeRange = proto.getTimeRange(); - long minStamp = 0; - long maxStamp = Long.MAX_VALUE; - if (timeRange.hasFrom()) { - minStamp = timeRange.getFrom(); - } - if (timeRange.hasTo()) { - maxStamp = timeRange.getTo(); - } - increment.setTimeRange(minStamp, maxStamp); + TimeRange timeRange = protoToTimeRange(proto.getTimeRange()); + increment.setTimeRange(timeRange.getMin(), timeRange.getMax()); } increment.setDurability(toDurability(proto.getDurability())); for (NameBytesPair attribute : proto.getAttributeList()) { @@ -874,6 +866,12 @@ public final class ProtobufUtil { scanBuilder.setLoadColumnFamiliesOnDemand(loadColumnFamiliesOnDemand.booleanValue()); } scanBuilder.setMaxVersions(scan.getMaxVersions()); + for (Entry cftr : scan.getColumnFamilyTimeRange().entrySet()) { + HBaseProtos.ColumnFamilyTimeRange.Builder b = HBaseProtos.ColumnFamilyTimeRange.newBuilder(); + b.setColumnFamily(ByteString.copyFrom(cftr.getKey())); + b.setTimeRange(timeRangeToProto(cftr.getValue())); + scanBuilder.addCfTimeRange(b); + } TimeRange timeRange = scan.getTimeRange(); if (!timeRange.isAllTime()) { HBaseProtos.TimeRange.Builder timeRangeBuilder = @@ -968,17 +966,16 @@ public final class ProtobufUtil { if (proto.hasLoadColumnFamiliesOnDemand()) { scan.setLoadColumnFamiliesOnDemand(proto.getLoadColumnFamiliesOnDemand()); } - if (proto.hasTimeRange()) { - HBaseProtos.TimeRange timeRange = proto.getTimeRange(); - long minStamp = 0; - long maxStamp = Long.MAX_VALUE; - if (timeRange.hasFrom()) { - minStamp = timeRange.getFrom(); + if (proto.getCfTimeRangeCount() > 0) { + for (HBaseProtos.ColumnFamilyTimeRange cftr : proto.getCfTimeRangeList()) { + TimeRange timeRange = protoToTimeRange(cftr.getTimeRange()); + scan.setColumnFamilyTimeRange(cftr.getColumnFamily().toByteArray(), + timeRange.getMin(), timeRange.getMax()); } - if (timeRange.hasTo()) { - maxStamp = timeRange.getTo(); - } - scan.setTimeRange(minStamp, maxStamp); + } + if (proto.hasTimeRange()) { + TimeRange timeRange = protoToTimeRange(proto.getTimeRange()); + scan.setTimeRange(timeRange.getMin(), timeRange.getMax()); } if (proto.hasFilter()) { FilterProtos.Filter filter = proto.getFilter(); @@ -1037,6 +1034,12 @@ public final class ProtobufUtil { if (get.getFilter() != null) { builder.setFilter(ProtobufUtil.toFilter(get.getFilter())); } + for (Entry cftr : get.getColumnFamilyTimeRange().entrySet()) { + HBaseProtos.ColumnFamilyTimeRange.Builder b = HBaseProtos.ColumnFamilyTimeRange.newBuilder(); + b.setColumnFamily(ByteString.copyFrom(cftr.getKey())); + b.setTimeRange(timeRangeToProto(cftr.getValue())); + builder.addCfTimeRange(b); + } TimeRange timeRange = get.getTimeRange(); if (!timeRange.isAllTime()) { HBaseProtos.TimeRange.Builder timeRangeBuilder = @@ -2492,7 +2495,7 @@ public final class ProtobufUtil { cell.getFamily().toByteArray(), cell.getQualifier().toByteArray(), cell.getTimestamp(), - (byte)cell.getCellType().getNumber(), + (byte) cell.getCellType().getNumber(), cell.getValue().toByteArray()); } @@ -3157,4 +3160,34 @@ public final class ProtobufUtil { } return scList; } + + private static HBaseProtos.TimeRange.Builder timeRangeToProto(TimeRange timeRange) { + HBaseProtos.TimeRange.Builder timeRangeBuilder = + HBaseProtos.TimeRange.newBuilder(); + timeRangeBuilder.setFrom(timeRange.getMin()); + timeRangeBuilder.setTo(timeRange.getMax()); + return timeRangeBuilder; + } + + private static TimeRange protoToTimeRange(HBaseProtos.TimeRange timeRange) throws IOException { + long minStamp = 0; + long maxStamp = Long.MAX_VALUE; + if (timeRange.hasFrom()) { + minStamp = timeRange.getFrom(); + } + if (timeRange.hasTo()) { + maxStamp = timeRange.getTo(); + } + return new TimeRange(minStamp, maxStamp); + } + + private static Map convert(List cftrs) + throws IOException { + Map result = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); + for (HBaseProtos.ColumnFamilyTimeRange cftr : cftrs) { + HBaseProtos.TimeRange tr = cftr.getTimeRange(); + result.put(cftr.getColumnFamily().toByteArray(), new TimeRange(tr.getFrom(), tr.getTo())); + } + return result; + } } diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java index c4b1eec..0966c24 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java @@ -1935,6 +1935,31 @@ public final class ClientProtos { * optional .hbase.pb.Consistency consistency = 12 [default = STRONG]; */ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Consistency getConsistency(); + + // repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + java.util.List + getCfTimeRangeList(); + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange getCfTimeRange(int index); + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + int getCfTimeRangeCount(); + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + java.util.List + getCfTimeRangeOrBuilderList(); + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder getCfTimeRangeOrBuilder( + int index); } /** * Protobuf type {@code hbase.pb.Get} @@ -2077,6 +2102,14 @@ public final class ClientProtos { } break; } + case 106: { + if (!((mutable_bitField0_ & 0x00000800) == 0x00000800)) { + cfTimeRange_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000800; + } + cfTimeRange_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.PARSER, extensionRegistry)); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -2091,6 +2124,9 @@ public final class ClientProtos { if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) { attribute_ = java.util.Collections.unmodifiableList(attribute_); } + if (((mutable_bitField0_ & 0x00000800) == 0x00000800)) { + cfTimeRange_ = java.util.Collections.unmodifiableList(cfTimeRange_); + } this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); } @@ -2361,6 +2397,42 @@ public final class ClientProtos { return consistency_; } + // repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + public static final int CF_TIME_RANGE_FIELD_NUMBER = 13; + private java.util.List cfTimeRange_; + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public java.util.List getCfTimeRangeList() { + return cfTimeRange_; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public java.util.List + getCfTimeRangeOrBuilderList() { + return cfTimeRange_; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public int getCfTimeRangeCount() { + return cfTimeRange_.size(); + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange getCfTimeRange(int index) { + return cfTimeRange_.get(index); + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder getCfTimeRangeOrBuilder( + int index) { + return cfTimeRange_.get(index); + } + private void initFields() { row_ = com.google.protobuf.ByteString.EMPTY; column_ = java.util.Collections.emptyList(); @@ -2373,6 +2445,7 @@ public final class ClientProtos { storeOffset_ = 0; existenceOnly_ = false; consistency_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Consistency.STRONG; + cfTimeRange_ = java.util.Collections.emptyList(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -2401,6 +2474,12 @@ public final class ClientProtos { return false; } } + for (int i = 0; i < getCfTimeRangeCount(); i++) { + if (!getCfTimeRange(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } memoizedIsInitialized = 1; return true; } @@ -2441,6 +2520,9 @@ public final class ClientProtos { if (((bitField0_ & 0x00000100) == 0x00000100)) { output.writeEnum(12, consistency_.getNumber()); } + for (int i = 0; i < cfTimeRange_.size(); i++) { + output.writeMessage(13, cfTimeRange_.get(i)); + } getUnknownFields().writeTo(output); } @@ -2494,6 +2576,10 @@ public final class ClientProtos { size += com.google.protobuf.CodedOutputStream .computeEnumSize(12, consistency_.getNumber()); } + for (int i = 0; i < cfTimeRange_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(13, cfTimeRange_.get(i)); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -2566,6 +2652,8 @@ public final class ClientProtos { result = result && (getConsistency() == other.getConsistency()); } + result = result && getCfTimeRangeList() + .equals(other.getCfTimeRangeList()); result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -2623,6 +2711,10 @@ public final class ClientProtos { hash = (37 * hash) + CONSISTENCY_FIELD_NUMBER; hash = (53 * hash) + hashEnum(getConsistency()); } + if (getCfTimeRangeCount() > 0) { + hash = (37 * hash) + CF_TIME_RANGE_FIELD_NUMBER; + hash = (53 * hash) + getCfTimeRangeList().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -2735,6 +2827,7 @@ public final class ClientProtos { getAttributeFieldBuilder(); getFilterFieldBuilder(); getTimeRangeFieldBuilder(); + getCfTimeRangeFieldBuilder(); } } private static Builder create() { @@ -2781,6 +2874,12 @@ public final class ClientProtos { bitField0_ = (bitField0_ & ~0x00000200); consistency_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Consistency.STRONG; bitField0_ = (bitField0_ & ~0x00000400); + if (cfTimeRangeBuilder_ == null) { + cfTimeRange_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000800); + } else { + cfTimeRangeBuilder_.clear(); + } return this; } @@ -2871,6 +2970,15 @@ public final class ClientProtos { to_bitField0_ |= 0x00000100; } result.consistency_ = consistency_; + if (cfTimeRangeBuilder_ == null) { + if (((bitField0_ & 0x00000800) == 0x00000800)) { + cfTimeRange_ = java.util.Collections.unmodifiableList(cfTimeRange_); + bitField0_ = (bitField0_ & ~0x00000800); + } + result.cfTimeRange_ = cfTimeRange_; + } else { + result.cfTimeRange_ = cfTimeRangeBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -2966,6 +3074,32 @@ public final class ClientProtos { if (other.hasConsistency()) { setConsistency(other.getConsistency()); } + if (cfTimeRangeBuilder_ == null) { + if (!other.cfTimeRange_.isEmpty()) { + if (cfTimeRange_.isEmpty()) { + cfTimeRange_ = other.cfTimeRange_; + bitField0_ = (bitField0_ & ~0x00000800); + } else { + ensureCfTimeRangeIsMutable(); + cfTimeRange_.addAll(other.cfTimeRange_); + } + onChanged(); + } + } else { + if (!other.cfTimeRange_.isEmpty()) { + if (cfTimeRangeBuilder_.isEmpty()) { + cfTimeRangeBuilder_.dispose(); + cfTimeRangeBuilder_ = null; + cfTimeRange_ = other.cfTimeRange_; + bitField0_ = (bitField0_ & ~0x00000800); + cfTimeRangeBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getCfTimeRangeFieldBuilder() : null; + } else { + cfTimeRangeBuilder_.addAllMessages(other.cfTimeRange_); + } + } + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -2993,6 +3127,12 @@ public final class ClientProtos { return false; } } + for (int i = 0; i < getCfTimeRangeCount(); i++) { + if (!getCfTimeRange(i).isInitialized()) { + + return false; + } + } return true; } @@ -3986,6 +4126,246 @@ public final class ClientProtos { return this; } + // repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + private java.util.List cfTimeRange_ = + java.util.Collections.emptyList(); + private void ensureCfTimeRangeIsMutable() { + if (!((bitField0_ & 0x00000800) == 0x00000800)) { + cfTimeRange_ = new java.util.ArrayList(cfTimeRange_); + bitField0_ |= 0x00000800; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder> cfTimeRangeBuilder_; + + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public java.util.List getCfTimeRangeList() { + if (cfTimeRangeBuilder_ == null) { + return java.util.Collections.unmodifiableList(cfTimeRange_); + } else { + return cfTimeRangeBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public int getCfTimeRangeCount() { + if (cfTimeRangeBuilder_ == null) { + return cfTimeRange_.size(); + } else { + return cfTimeRangeBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange getCfTimeRange(int index) { + if (cfTimeRangeBuilder_ == null) { + return cfTimeRange_.get(index); + } else { + return cfTimeRangeBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public Builder setCfTimeRange( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange value) { + if (cfTimeRangeBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureCfTimeRangeIsMutable(); + cfTimeRange_.set(index, value); + onChanged(); + } else { + cfTimeRangeBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public Builder setCfTimeRange( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder builderForValue) { + if (cfTimeRangeBuilder_ == null) { + ensureCfTimeRangeIsMutable(); + cfTimeRange_.set(index, builderForValue.build()); + onChanged(); + } else { + cfTimeRangeBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public Builder addCfTimeRange(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange value) { + if (cfTimeRangeBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureCfTimeRangeIsMutable(); + cfTimeRange_.add(value); + onChanged(); + } else { + cfTimeRangeBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public Builder addCfTimeRange( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange value) { + if (cfTimeRangeBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureCfTimeRangeIsMutable(); + cfTimeRange_.add(index, value); + onChanged(); + } else { + cfTimeRangeBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public Builder addCfTimeRange( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder builderForValue) { + if (cfTimeRangeBuilder_ == null) { + ensureCfTimeRangeIsMutable(); + cfTimeRange_.add(builderForValue.build()); + onChanged(); + } else { + cfTimeRangeBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public Builder addCfTimeRange( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder builderForValue) { + if (cfTimeRangeBuilder_ == null) { + ensureCfTimeRangeIsMutable(); + cfTimeRange_.add(index, builderForValue.build()); + onChanged(); + } else { + cfTimeRangeBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public Builder addAllCfTimeRange( + java.lang.Iterable values) { + if (cfTimeRangeBuilder_ == null) { + ensureCfTimeRangeIsMutable(); + super.addAll(values, cfTimeRange_); + onChanged(); + } else { + cfTimeRangeBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public Builder clearCfTimeRange() { + if (cfTimeRangeBuilder_ == null) { + cfTimeRange_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000800); + onChanged(); + } else { + cfTimeRangeBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public Builder removeCfTimeRange(int index) { + if (cfTimeRangeBuilder_ == null) { + ensureCfTimeRangeIsMutable(); + cfTimeRange_.remove(index); + onChanged(); + } else { + cfTimeRangeBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder getCfTimeRangeBuilder( + int index) { + return getCfTimeRangeFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder getCfTimeRangeOrBuilder( + int index) { + if (cfTimeRangeBuilder_ == null) { + return cfTimeRange_.get(index); } else { + return cfTimeRangeBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public java.util.List + getCfTimeRangeOrBuilderList() { + if (cfTimeRangeBuilder_ != null) { + return cfTimeRangeBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(cfTimeRange_); + } + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder addCfTimeRangeBuilder() { + return getCfTimeRangeFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder addCfTimeRangeBuilder( + int index) { + return getCfTimeRangeFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 13; + */ + public java.util.List + getCfTimeRangeBuilderList() { + return getCfTimeRangeFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder> + getCfTimeRangeFieldBuilder() { + if (cfTimeRangeBuilder_ == null) { + cfTimeRangeBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder>( + cfTimeRange_, + ((bitField0_ & 0x00000800) == 0x00000800), + getParentForChildren(), + isClean()); + cfTimeRange_ = null; + } + return cfTimeRangeBuilder_; + } + // @@protoc_insertion_point(builder_scope:hbase.pb.Get) } @@ -13682,6 +14062,31 @@ public final class ClientProtos { * optional uint32 caching = 17; */ int getCaching(); + + // repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + java.util.List + getCfTimeRangeList(); + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange getCfTimeRange(int index); + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + int getCfTimeRangeCount(); + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + java.util.List + getCfTimeRangeOrBuilderList(); + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder getCfTimeRangeOrBuilder( + int index); } /** * Protobuf type {@code hbase.pb.Scan} @@ -13858,6 +14263,14 @@ public final class ClientProtos { caching_ = input.readUInt32(); break; } + case 146: { + if (!((mutable_bitField0_ & 0x00020000) == 0x00020000)) { + cfTimeRange_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00020000; + } + cfTimeRange_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.PARSER, extensionRegistry)); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -13872,6 +14285,9 @@ public final class ClientProtos { if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) { attribute_ = java.util.Collections.unmodifiableList(attribute_); } + if (((mutable_bitField0_ & 0x00020000) == 0x00020000)) { + cfTimeRange_ = java.util.Collections.unmodifiableList(cfTimeRange_); + } this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); } @@ -14236,6 +14652,42 @@ public final class ClientProtos { return caching_; } + // repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + public static final int CF_TIME_RANGE_FIELD_NUMBER = 18; + private java.util.List cfTimeRange_; + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public java.util.List getCfTimeRangeList() { + return cfTimeRange_; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public java.util.List + getCfTimeRangeOrBuilderList() { + return cfTimeRange_; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public int getCfTimeRangeCount() { + return cfTimeRange_.size(); + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange getCfTimeRange(int index) { + return cfTimeRange_.get(index); + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder getCfTimeRangeOrBuilder( + int index) { + return cfTimeRange_.get(index); + } + private void initFields() { column_ = java.util.Collections.emptyList(); attribute_ = java.util.Collections.emptyList(); @@ -14254,6 +14706,7 @@ public final class ClientProtos { reversed_ = false; consistency_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Consistency.STRONG; caching_ = 0; + cfTimeRange_ = java.util.Collections.emptyList(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -14278,6 +14731,12 @@ public final class ClientProtos { return false; } } + for (int i = 0; i < getCfTimeRangeCount(); i++) { + if (!getCfTimeRange(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } memoizedIsInitialized = 1; return true; } @@ -14336,6 +14795,9 @@ public final class ClientProtos { if (((bitField0_ & 0x00004000) == 0x00004000)) { output.writeUInt32(17, caching_); } + for (int i = 0; i < cfTimeRange_.size(); i++) { + output.writeMessage(18, cfTimeRange_.get(i)); + } getUnknownFields().writeTo(output); } @@ -14413,6 +14875,10 @@ public final class ClientProtos { size += com.google.protobuf.CodedOutputStream .computeUInt32Size(17, caching_); } + for (int i = 0; i < cfTimeRange_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(18, cfTimeRange_.get(i)); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -14515,6 +14981,8 @@ public final class ClientProtos { result = result && (getCaching() == other.getCaching()); } + result = result && getCfTimeRangeList() + .equals(other.getCfTimeRangeList()); result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -14596,6 +15064,10 @@ public final class ClientProtos { hash = (37 * hash) + CACHING_FIELD_NUMBER; hash = (53 * hash) + getCaching(); } + if (getCfTimeRangeCount() > 0) { + hash = (37 * hash) + CF_TIME_RANGE_FIELD_NUMBER; + hash = (53 * hash) + getCfTimeRangeList().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -14712,6 +15184,7 @@ public final class ClientProtos { getAttributeFieldBuilder(); getFilterFieldBuilder(); getTimeRangeFieldBuilder(); + getCfTimeRangeFieldBuilder(); } } private static Builder create() { @@ -14770,6 +15243,12 @@ public final class ClientProtos { bitField0_ = (bitField0_ & ~0x00008000); caching_ = 0; bitField0_ = (bitField0_ & ~0x00010000); + if (cfTimeRangeBuilder_ == null) { + cfTimeRange_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00020000); + } else { + cfTimeRangeBuilder_.clear(); + } return this; } @@ -14884,6 +15363,15 @@ public final class ClientProtos { to_bitField0_ |= 0x00004000; } result.caching_ = caching_; + if (cfTimeRangeBuilder_ == null) { + if (((bitField0_ & 0x00020000) == 0x00020000)) { + cfTimeRange_ = java.util.Collections.unmodifiableList(cfTimeRange_); + bitField0_ = (bitField0_ & ~0x00020000); + } + result.cfTimeRange_ = cfTimeRange_; + } else { + result.cfTimeRange_ = cfTimeRangeBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -14997,6 +15485,32 @@ public final class ClientProtos { if (other.hasCaching()) { setCaching(other.getCaching()); } + if (cfTimeRangeBuilder_ == null) { + if (!other.cfTimeRange_.isEmpty()) { + if (cfTimeRange_.isEmpty()) { + cfTimeRange_ = other.cfTimeRange_; + bitField0_ = (bitField0_ & ~0x00020000); + } else { + ensureCfTimeRangeIsMutable(); + cfTimeRange_.addAll(other.cfTimeRange_); + } + onChanged(); + } + } else { + if (!other.cfTimeRange_.isEmpty()) { + if (cfTimeRangeBuilder_.isEmpty()) { + cfTimeRangeBuilder_.dispose(); + cfTimeRangeBuilder_ = null; + cfTimeRange_ = other.cfTimeRange_; + bitField0_ = (bitField0_ & ~0x00020000); + cfTimeRangeBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getCfTimeRangeFieldBuilder() : null; + } else { + cfTimeRangeBuilder_.addAllMessages(other.cfTimeRange_); + } + } + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -15020,6 +15534,12 @@ public final class ClientProtos { return false; } } + for (int i = 0; i < getCfTimeRangeCount(); i++) { + if (!getCfTimeRange(i).isInitialized()) { + + return false; + } + } return true; } @@ -16210,6 +16730,246 @@ public final class ClientProtos { return this; } + // repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + private java.util.List cfTimeRange_ = + java.util.Collections.emptyList(); + private void ensureCfTimeRangeIsMutable() { + if (!((bitField0_ & 0x00020000) == 0x00020000)) { + cfTimeRange_ = new java.util.ArrayList(cfTimeRange_); + bitField0_ |= 0x00020000; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder> cfTimeRangeBuilder_; + + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public java.util.List getCfTimeRangeList() { + if (cfTimeRangeBuilder_ == null) { + return java.util.Collections.unmodifiableList(cfTimeRange_); + } else { + return cfTimeRangeBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public int getCfTimeRangeCount() { + if (cfTimeRangeBuilder_ == null) { + return cfTimeRange_.size(); + } else { + return cfTimeRangeBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange getCfTimeRange(int index) { + if (cfTimeRangeBuilder_ == null) { + return cfTimeRange_.get(index); + } else { + return cfTimeRangeBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public Builder setCfTimeRange( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange value) { + if (cfTimeRangeBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureCfTimeRangeIsMutable(); + cfTimeRange_.set(index, value); + onChanged(); + } else { + cfTimeRangeBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public Builder setCfTimeRange( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder builderForValue) { + if (cfTimeRangeBuilder_ == null) { + ensureCfTimeRangeIsMutable(); + cfTimeRange_.set(index, builderForValue.build()); + onChanged(); + } else { + cfTimeRangeBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public Builder addCfTimeRange(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange value) { + if (cfTimeRangeBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureCfTimeRangeIsMutable(); + cfTimeRange_.add(value); + onChanged(); + } else { + cfTimeRangeBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public Builder addCfTimeRange( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange value) { + if (cfTimeRangeBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureCfTimeRangeIsMutable(); + cfTimeRange_.add(index, value); + onChanged(); + } else { + cfTimeRangeBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public Builder addCfTimeRange( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder builderForValue) { + if (cfTimeRangeBuilder_ == null) { + ensureCfTimeRangeIsMutable(); + cfTimeRange_.add(builderForValue.build()); + onChanged(); + } else { + cfTimeRangeBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public Builder addCfTimeRange( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder builderForValue) { + if (cfTimeRangeBuilder_ == null) { + ensureCfTimeRangeIsMutable(); + cfTimeRange_.add(index, builderForValue.build()); + onChanged(); + } else { + cfTimeRangeBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public Builder addAllCfTimeRange( + java.lang.Iterable values) { + if (cfTimeRangeBuilder_ == null) { + ensureCfTimeRangeIsMutable(); + super.addAll(values, cfTimeRange_); + onChanged(); + } else { + cfTimeRangeBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public Builder clearCfTimeRange() { + if (cfTimeRangeBuilder_ == null) { + cfTimeRange_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00020000); + onChanged(); + } else { + cfTimeRangeBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public Builder removeCfTimeRange(int index) { + if (cfTimeRangeBuilder_ == null) { + ensureCfTimeRangeIsMutable(); + cfTimeRange_.remove(index); + onChanged(); + } else { + cfTimeRangeBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder getCfTimeRangeBuilder( + int index) { + return getCfTimeRangeFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder getCfTimeRangeOrBuilder( + int index) { + if (cfTimeRangeBuilder_ == null) { + return cfTimeRange_.get(index); } else { + return cfTimeRangeBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public java.util.List + getCfTimeRangeOrBuilderList() { + if (cfTimeRangeBuilder_ != null) { + return cfTimeRangeBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(cfTimeRange_); + } + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder addCfTimeRangeBuilder() { + return getCfTimeRangeFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder addCfTimeRangeBuilder( + int index) { + return getCfTimeRangeFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ColumnFamilyTimeRange cf_time_range = 18; + */ + public java.util.List + getCfTimeRangeBuilderList() { + return getCfTimeRangeFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder> + getCfTimeRangeFieldBuilder() { + if (cfTimeRangeBuilder_ == null) { + cfTimeRangeBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder>( + cfTimeRange_, + ((bitField0_ & 0x00020000) == 0x00020000), + getParentForChildren(), + isClean()); + cfTimeRange_ = null; + } + return cfTimeRangeBuilder_; + } + // @@protoc_insertion_point(builder_scope:hbase.pb.Scan) } @@ -33121,7 +33881,7 @@ public final class ClientProtos { "o\032\017MapReduce.proto\"\037\n\016Authorizations\022\r\n\005" + "label\030\001 \003(\t\"$\n\016CellVisibility\022\022\n\nexpress" + "ion\030\001 \002(\t\"+\n\006Column\022\016\n\006family\030\001 \002(\014\022\021\n\tq" + - "ualifier\030\002 \003(\014\"\336\002\n\003Get\022\013\n\003row\030\001 \002(\014\022 \n\006c" + + "ualifier\030\002 \003(\014\"\226\003\n\003Get\022\013\n\003row\030\001 \002(\014\022 \n\006c" + "olumn\030\002 \003(\0132\020.hbase.pb.Column\022*\n\tattribu" + "te\030\003 \003(\0132\027.hbase.pb.NameBytesPair\022 \n\006fil" + "ter\030\004 \001(\0132\020.hbase.pb.Filter\022\'\n\ntime_rang" + @@ -33130,125 +33890,128 @@ public final class ClientProtos { "e\022\023\n\013store_limit\030\010 \001(\r\022\024\n\014store_offset\030\t" + " \001(\r\022\035\n\016existence_only\030\n \001(\010:\005false\0222\n\013c" + "onsistency\030\014 \001(\0162\025.hbase.pb.Consistency:" + - "\006STRONG\"\203\001\n\006Result\022\034\n\004cell\030\001 \003(\0132\016.hbase" + - ".pb.Cell\022\035\n\025associated_cell_count\030\002 \001(\005\022" + - "\016\n\006exists\030\003 \001(\010\022\024\n\005stale\030\004 \001(\010:\005false\022\026\n" + - "\007partial\030\005 \001(\010:\005false\"S\n\nGetRequest\022)\n\006r" + - "egion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022\032" + - "\n\003get\030\002 \002(\0132\r.hbase.pb.Get\"/\n\013GetRespons", - "e\022 \n\006result\030\001 \001(\0132\020.hbase.pb.Result\"\222\001\n\t" + - "Condition\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002 \002(\014\022\021" + - "\n\tqualifier\030\003 \002(\014\022+\n\014compare_type\030\004 \002(\0162" + - "\025.hbase.pb.CompareType\022(\n\ncomparator\030\005 \002" + - "(\0132\024.hbase.pb.Comparator\"\364\006\n\rMutationPro" + - "to\022\013\n\003row\030\001 \001(\014\0229\n\013mutate_type\030\002 \001(\0162$.h" + - "base.pb.MutationProto.MutationType\0229\n\014co" + - "lumn_value\030\003 \003(\0132#.hbase.pb.MutationProt" + - "o.ColumnValue\022\021\n\ttimestamp\030\004 \001(\004\022*\n\tattr" + - "ibute\030\005 \003(\0132\027.hbase.pb.NameBytesPair\022C\n\n", - "durability\030\006 \001(\0162\".hbase.pb.MutationProt" + - "o.Durability:\013USE_DEFAULT\022\'\n\ntime_range\030" + - "\007 \001(\0132\023.hbase.pb.TimeRange\022\035\n\025associated" + - "_cell_count\030\010 \001(\005\022\r\n\005nonce\030\t \001(\004\032\371\001\n\013Col" + - "umnValue\022\016\n\006family\030\001 \002(\014\022K\n\017qualifier_va" + - "lue\030\002 \003(\01322.hbase.pb.MutationProto.Colum" + - "nValue.QualifierValue\032\214\001\n\016QualifierValue" + - "\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005value\030\002 \001(\014\022\021\n\tti" + - "mestamp\030\003 \001(\004\0227\n\013delete_type\030\004 \001(\0162\".hba" + - "se.pb.MutationProto.DeleteType\022\014\n\004tags\030\005", - " \001(\014\"W\n\nDurability\022\017\n\013USE_DEFAULT\020\000\022\014\n\010S" + - "KIP_WAL\020\001\022\r\n\tASYNC_WAL\020\002\022\014\n\010SYNC_WAL\020\003\022\r" + - "\n\tFSYNC_WAL\020\004\">\n\014MutationType\022\n\n\006APPEND\020" + - "\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002\022\n\n\006DELETE\020\003\"p\n" + - "\nDeleteType\022\026\n\022DELETE_ONE_VERSION\020\000\022\034\n\030D" + - "ELETE_MULTIPLE_VERSIONS\020\001\022\021\n\rDELETE_FAMI" + - "LY\020\002\022\031\n\025DELETE_FAMILY_VERSION\020\003\"\242\001\n\rMuta" + - "teRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Reg" + - "ionSpecifier\022)\n\010mutation\030\002 \002(\0132\027.hbase.p" + - "b.MutationProto\022&\n\tcondition\030\003 \001(\0132\023.hba", - "se.pb.Condition\022\023\n\013nonce_group\030\004 \001(\004\"E\n\016" + - "MutateResponse\022 \n\006result\030\001 \001(\0132\020.hbase.p" + - "b.Result\022\021\n\tprocessed\030\002 \001(\010\"\346\003\n\004Scan\022 \n\006" + - "column\030\001 \003(\0132\020.hbase.pb.Column\022*\n\tattrib" + - "ute\030\002 \003(\0132\027.hbase.pb.NameBytesPair\022\021\n\tst" + - "art_row\030\003 \001(\014\022\020\n\010stop_row\030\004 \001(\014\022 \n\006filte" + - "r\030\005 \001(\0132\020.hbase.pb.Filter\022\'\n\ntime_range\030" + - "\006 \001(\0132\023.hbase.pb.TimeRange\022\027\n\014max_versio" + - "ns\030\007 \001(\r:\0011\022\032\n\014cache_blocks\030\010 \001(\010:\004true\022" + - "\022\n\nbatch_size\030\t \001(\r\022\027\n\017max_result_size\030\n", - " \001(\004\022\023\n\013store_limit\030\013 \001(\r\022\024\n\014store_offse" + - "t\030\014 \001(\r\022&\n\036load_column_families_on_deman" + - "d\030\r \001(\010\022\r\n\005small\030\016 \001(\010\022\027\n\010reversed\030\017 \001(\010" + - ":\005false\0222\n\013consistency\030\020 \001(\0162\025.hbase.pb." + - "Consistency:\006STRONG\022\017\n\007caching\030\021 \001(\r\"\220\002\n" + - "\013ScanRequest\022)\n\006region\030\001 \001(\0132\031.hbase.pb." + - "RegionSpecifier\022\034\n\004scan\030\002 \001(\0132\016.hbase.pb" + - ".Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016number_of_r" + - "ows\030\004 \001(\r\022\025\n\rclose_scanner\030\005 \001(\010\022\025\n\rnext" + - "_call_seq\030\006 \001(\004\022\037\n\027client_handles_partia", - "ls\030\007 \001(\010\022!\n\031client_handles_heartbeats\030\010 " + - "\001(\010\022\032\n\022track_scan_metrics\030\t \001(\010\"\232\002\n\014Scan" + - "Response\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n\nsc" + - "anner_id\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003" + - "ttl\030\004 \001(\r\022!\n\007results\030\005 \003(\0132\020.hbase.pb.Re" + - "sult\022\r\n\005stale\030\006 \001(\010\022\037\n\027partial_flag_per_" + - "result\030\007 \003(\010\022\036\n\026more_results_in_region\030\010" + - " \001(\010\022\031\n\021heartbeat_message\030\t \001(\010\022+\n\014scan_" + - "metrics\030\n \001(\0132\025.hbase.pb.ScanMetrics\"\305\001\n" + - "\024BulkLoadHFileRequest\022)\n\006region\030\001 \002(\0132\031.", - "hbase.pb.RegionSpecifier\022>\n\013family_path\030" + - "\002 \003(\0132).hbase.pb.BulkLoadHFileRequest.Fa" + - "milyPath\022\026\n\016assign_seq_num\030\003 \001(\010\032*\n\nFami" + - "lyPath\022\016\n\006family\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025" + - "BulkLoadHFileResponse\022\016\n\006loaded\030\001 \002(\010\"a\n" + - "\026CoprocessorServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014" + - "service_name\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022" + - "\017\n\007request\030\004 \002(\014\"B\n\030CoprocessorServiceRe" + - "sult\022&\n\005value\030\001 \001(\0132\027.hbase.pb.NameBytes" + - "Pair\"v\n\031CoprocessorServiceRequest\022)\n\006reg", - "ion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022.\n\004" + - "call\030\002 \002(\0132 .hbase.pb.CoprocessorService" + - "Call\"o\n\032CoprocessorServiceResponse\022)\n\006re" + - "gion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022&\n" + - "\005value\030\002 \002(\0132\027.hbase.pb.NameBytesPair\"\226\001" + - "\n\006Action\022\r\n\005index\030\001 \001(\r\022)\n\010mutation\030\002 \001(" + - "\0132\027.hbase.pb.MutationProto\022\032\n\003get\030\003 \001(\0132" + - "\r.hbase.pb.Get\0226\n\014service_call\030\004 \001(\0132 .h" + - "base.pb.CoprocessorServiceCall\"k\n\014Region" + - "Action\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Region", - "Specifier\022\016\n\006atomic\030\002 \001(\010\022 \n\006action\030\003 \003(" + - "\0132\020.hbase.pb.Action\"D\n\017RegionLoadStats\022\027" + - "\n\014memstoreLoad\030\001 \001(\005:\0010\022\030\n\rheapOccupancy" + - "\030\002 \001(\005:\0010\"\332\001\n\021ResultOrException\022\r\n\005index" + - "\030\001 \001(\r\022 \n\006result\030\002 \001(\0132\020.hbase.pb.Result" + - "\022*\n\texception\030\003 \001(\0132\027.hbase.pb.NameBytes" + - "Pair\022:\n\016service_result\030\004 \001(\0132\".hbase.pb." + - "CoprocessorServiceResult\022,\n\tloadStats\030\005 " + - "\001(\0132\031.hbase.pb.RegionLoadStats\"x\n\022Region" + - "ActionResult\0226\n\021resultOrException\030\001 \003(\0132", - "\033.hbase.pb.ResultOrException\022*\n\texceptio" + - "n\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014Mult" + - "iRequest\022,\n\014regionAction\030\001 \003(\0132\026.hbase.p" + - "b.RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\tco" + - "ndition\030\003 \001(\0132\023.hbase.pb.Condition\"\\\n\rMu" + - "ltiResponse\0228\n\022regionActionResult\030\001 \003(\0132" + - "\034.hbase.pb.RegionActionResult\022\021\n\tprocess" + - "ed\030\002 \001(\010*\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n\010T" + - "IMELINE\020\0012\203\004\n\rClientService\0222\n\003Get\022\024.hba" + - "se.pb.GetRequest\032\025.hbase.pb.GetResponse\022", - ";\n\006Mutate\022\027.hbase.pb.MutateRequest\032\030.hba" + - "se.pb.MutateResponse\0225\n\004Scan\022\025.hbase.pb." + - "ScanRequest\032\026.hbase.pb.ScanResponse\022P\n\rB" + - "ulkLoadHFile\022\036.hbase.pb.BulkLoadHFileReq" + - "uest\032\037.hbase.pb.BulkLoadHFileResponse\022X\n" + - "\013ExecService\022#.hbase.pb.CoprocessorServi" + - "ceRequest\032$.hbase.pb.CoprocessorServiceR" + - "esponse\022d\n\027ExecRegionServerService\022#.hba" + - "se.pb.CoprocessorServiceRequest\032$.hbase." + - "pb.CoprocessorServiceResponse\0228\n\005Multi\022\026", - ".hbase.pb.MultiRequest\032\027.hbase.pb.MultiR" + - "esponseBB\n*org.apache.hadoop.hbase.proto" + - "buf.generatedB\014ClientProtosH\001\210\001\001\240\001\001" + "\006STRONG\0226\n\rcf_time_range\030\r \003(\0132\037.hbase.p" + + "b.ColumnFamilyTimeRange\"\203\001\n\006Result\022\034\n\004ce" + + "ll\030\001 \003(\0132\016.hbase.pb.Cell\022\035\n\025associated_c" + + "ell_count\030\002 \001(\005\022\016\n\006exists\030\003 \001(\010\022\024\n\005stale" + + "\030\004 \001(\010:\005false\022\026\n\007partial\030\005 \001(\010:\005false\"S\n" + + "\nGetRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.R", + "egionSpecifier\022\032\n\003get\030\002 \002(\0132\r.hbase.pb.G" + + "et\"/\n\013GetResponse\022 \n\006result\030\001 \001(\0132\020.hbas" + + "e.pb.Result\"\222\001\n\tCondition\022\013\n\003row\030\001 \002(\014\022\016" + + "\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022+\n\014com" + + "pare_type\030\004 \002(\0162\025.hbase.pb.CompareType\022(" + + "\n\ncomparator\030\005 \002(\0132\024.hbase.pb.Comparator" + + "\"\364\006\n\rMutationProto\022\013\n\003row\030\001 \001(\014\0229\n\013mutat" + + "e_type\030\002 \001(\0162$.hbase.pb.MutationProto.Mu" + + "tationType\0229\n\014column_value\030\003 \003(\0132#.hbase" + + ".pb.MutationProto.ColumnValue\022\021\n\ttimesta", + "mp\030\004 \001(\004\022*\n\tattribute\030\005 \003(\0132\027.hbase.pb.N" + + "ameBytesPair\022C\n\ndurability\030\006 \001(\0162\".hbase" + + ".pb.MutationProto.Durability:\013USE_DEFAUL" + + "T\022\'\n\ntime_range\030\007 \001(\0132\023.hbase.pb.TimeRan" + + "ge\022\035\n\025associated_cell_count\030\010 \001(\005\022\r\n\005non" + + "ce\030\t \001(\004\032\371\001\n\013ColumnValue\022\016\n\006family\030\001 \002(\014" + + "\022K\n\017qualifier_value\030\002 \003(\01322.hbase.pb.Mut" + + "ationProto.ColumnValue.QualifierValue\032\214\001" + + "\n\016QualifierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005v" + + "alue\030\002 \001(\014\022\021\n\ttimestamp\030\003 \001(\004\0227\n\013delete_", + "type\030\004 \001(\0162\".hbase.pb.MutationProto.Dele" + + "teType\022\014\n\004tags\030\005 \001(\014\"W\n\nDurability\022\017\n\013US" + + "E_DEFAULT\020\000\022\014\n\010SKIP_WAL\020\001\022\r\n\tASYNC_WAL\020\002" + + "\022\014\n\010SYNC_WAL\020\003\022\r\n\tFSYNC_WAL\020\004\">\n\014Mutatio" + + "nType\022\n\n\006APPEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020" + + "\002\022\n\n\006DELETE\020\003\"p\n\nDeleteType\022\026\n\022DELETE_ON" + + "E_VERSION\020\000\022\034\n\030DELETE_MULTIPLE_VERSIONS\020" + + "\001\022\021\n\rDELETE_FAMILY\020\002\022\031\n\025DELETE_FAMILY_VE" + + "RSION\020\003\"\242\001\n\rMutateRequest\022)\n\006region\030\001 \002(" + + "\0132\031.hbase.pb.RegionSpecifier\022)\n\010mutation", + "\030\002 \002(\0132\027.hbase.pb.MutationProto\022&\n\tcondi" + + "tion\030\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce" + + "_group\030\004 \001(\004\"E\n\016MutateResponse\022 \n\006result" + + "\030\001 \001(\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 " + + "\001(\010\"\236\004\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." + + "Column\022*\n\tattribute\030\002 \003(\0132\027.hbase.pb.Nam" + + "eBytesPair\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop_ro" + + "w\030\004 \001(\014\022 \n\006filter\030\005 \001(\0132\020.hbase.pb.Filte" + + "r\022\'\n\ntime_range\030\006 \001(\0132\023.hbase.pb.TimeRan" + + "ge\022\027\n\014max_versions\030\007 \001(\r:\0011\022\032\n\014cache_blo", + "cks\030\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017m" + + "ax_result_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(" + + "\r\022\024\n\014store_offset\030\014 \001(\r\022&\n\036load_column_f" + + "amilies_on_demand\030\r \001(\010\022\r\n\005small\030\016 \001(\010\022\027" + + "\n\010reversed\030\017 \001(\010:\005false\0222\n\013consistency\030\020" + + " \001(\0162\025.hbase.pb.Consistency:\006STRONG\022\017\n\007c" + + "aching\030\021 \001(\r\0226\n\rcf_time_range\030\022 \003(\0132\037.hb" + + "ase.pb.ColumnFamilyTimeRange\"\220\002\n\013ScanReq" + + "uest\022)\n\006region\030\001 \001(\0132\031.hbase.pb.RegionSp" + + "ecifier\022\034\n\004scan\030\002 \001(\0132\016.hbase.pb.Scan\022\022\n", + "\nscanner_id\030\003 \001(\004\022\026\n\016number_of_rows\030\004 \001(" + + "\r\022\025\n\rclose_scanner\030\005 \001(\010\022\025\n\rnext_call_se" + + "q\030\006 \001(\004\022\037\n\027client_handles_partials\030\007 \001(\010" + + "\022!\n\031client_handles_heartbeats\030\010 \001(\010\022\032\n\022t" + + "rack_scan_metrics\030\t \001(\010\"\232\002\n\014ScanResponse" + + "\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n\nscanner_id" + + "\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(" + + "\r\022!\n\007results\030\005 \003(\0132\020.hbase.pb.Result\022\r\n\005" + + "stale\030\006 \001(\010\022\037\n\027partial_flag_per_result\030\007" + + " \003(\010\022\036\n\026more_results_in_region\030\010 \001(\010\022\031\n\021", + "heartbeat_message\030\t \001(\010\022+\n\014scan_metrics\030" + + "\n \001(\0132\025.hbase.pb.ScanMetrics\"\305\001\n\024BulkLoa" + + "dHFileRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb" + + ".RegionSpecifier\022>\n\013family_path\030\002 \003(\0132)." + + "hbase.pb.BulkLoadHFileRequest.FamilyPath" + + "\022\026\n\016assign_seq_num\030\003 \001(\010\032*\n\nFamilyPath\022\016" + + "\n\006family\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoad" + + "HFileResponse\022\016\n\006loaded\030\001 \002(\010\"a\n\026Coproce" + + "ssorServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service_" + + "name\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007reque", + "st\030\004 \002(\014\"B\n\030CoprocessorServiceResult\022&\n\005" + + "value\030\001 \001(\0132\027.hbase.pb.NameBytesPair\"v\n\031" + + "CoprocessorServiceRequest\022)\n\006region\030\001 \002(" + + "\0132\031.hbase.pb.RegionSpecifier\022.\n\004call\030\002 \002" + + "(\0132 .hbase.pb.CoprocessorServiceCall\"o\n\032" + + "CoprocessorServiceResponse\022)\n\006region\030\001 \002" + + "(\0132\031.hbase.pb.RegionSpecifier\022&\n\005value\030\002" + + " \002(\0132\027.hbase.pb.NameBytesPair\"\226\001\n\006Action" + + "\022\r\n\005index\030\001 \001(\r\022)\n\010mutation\030\002 \001(\0132\027.hbas" + + "e.pb.MutationProto\022\032\n\003get\030\003 \001(\0132\r.hbase.", + "pb.Get\0226\n\014service_call\030\004 \001(\0132 .hbase.pb." + + "CoprocessorServiceCall\"k\n\014RegionAction\022)" + + "\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifie" + + "r\022\016\n\006atomic\030\002 \001(\010\022 \n\006action\030\003 \003(\0132\020.hbas" + + "e.pb.Action\"D\n\017RegionLoadStats\022\027\n\014memsto" + + "reLoad\030\001 \001(\005:\0010\022\030\n\rheapOccupancy\030\002 \001(\005:\001" + + "0\"\332\001\n\021ResultOrException\022\r\n\005index\030\001 \001(\r\022 " + + "\n\006result\030\002 \001(\0132\020.hbase.pb.Result\022*\n\texce" + + "ption\030\003 \001(\0132\027.hbase.pb.NameBytesPair\022:\n\016" + + "service_result\030\004 \001(\0132\".hbase.pb.Coproces", + "sorServiceResult\022,\n\tloadStats\030\005 \001(\0132\031.hb" + + "ase.pb.RegionLoadStats\"x\n\022RegionActionRe" + + "sult\0226\n\021resultOrException\030\001 \003(\0132\033.hbase." + + "pb.ResultOrException\022*\n\texception\030\002 \001(\0132" + + "\027.hbase.pb.NameBytesPair\"x\n\014MultiRequest" + + "\022,\n\014regionAction\030\001 \003(\0132\026.hbase.pb.Region" + + "Action\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\tcondition\030" + + "\003 \001(\0132\023.hbase.pb.Condition\"\\\n\rMultiRespo" + + "nse\0228\n\022regionActionResult\030\001 \003(\0132\034.hbase." + + "pb.RegionActionResult\022\021\n\tprocessed\030\002 \001(\010", + "*\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020" + + "\0012\203\004\n\rClientService\0222\n\003Get\022\024.hbase.pb.Ge" + + "tRequest\032\025.hbase.pb.GetResponse\022;\n\006Mutat" + + "e\022\027.hbase.pb.MutateRequest\032\030.hbase.pb.Mu" + + "tateResponse\0225\n\004Scan\022\025.hbase.pb.ScanRequ" + + "est\032\026.hbase.pb.ScanResponse\022P\n\rBulkLoadH" + + "File\022\036.hbase.pb.BulkLoadHFileRequest\032\037.h" + + "base.pb.BulkLoadHFileResponse\022X\n\013ExecSer" + + "vice\022#.hbase.pb.CoprocessorServiceReques" + + "t\032$.hbase.pb.CoprocessorServiceResponse\022", + "d\n\027ExecRegionServerService\022#.hbase.pb.Co" + + "processorServiceRequest\032$.hbase.pb.Copro" + + "cessorServiceResponse\0228\n\005Multi\022\026.hbase.p" + + "b.MultiRequest\032\027.hbase.pb.MultiResponseB" + + "B\n*org.apache.hadoop.hbase.protobuf.gene" + + "ratedB\014ClientProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -33278,7 +34041,7 @@ public final class ClientProtos { internal_static_hbase_pb_Get_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_Get_descriptor, - new java.lang.String[] { "Row", "Column", "Attribute", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "StoreLimit", "StoreOffset", "ExistenceOnly", "Consistency", }); + new java.lang.String[] { "Row", "Column", "Attribute", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "StoreLimit", "StoreOffset", "ExistenceOnly", "Consistency", "CfTimeRange", }); internal_static_hbase_pb_Result_descriptor = getDescriptor().getMessageTypes().get(4); internal_static_hbase_pb_Result_fieldAccessorTable = new @@ -33338,7 +34101,7 @@ public final class ClientProtos { internal_static_hbase_pb_Scan_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_Scan_descriptor, - new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", }); + new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "CfTimeRange", }); internal_static_hbase_pb_ScanRequest_descriptor = getDescriptor().getMessageTypes().get(12); internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java index 623da6c..d9d7956 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java @@ -7974,6 +7974,668 @@ public final class HBaseProtos { // @@protoc_insertion_point(class_scope:hbase.pb.TimeRange) } + public interface ColumnFamilyTimeRangeOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required bytes column_family = 1; + /** + * required bytes column_family = 1; + */ + boolean hasColumnFamily(); + /** + * required bytes column_family = 1; + */ + com.google.protobuf.ByteString getColumnFamily(); + + // required .hbase.pb.TimeRange time_range = 2; + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + boolean hasTimeRange(); + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange getTimeRange(); + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRangeOrBuilder getTimeRangeOrBuilder(); + } + /** + * Protobuf type {@code hbase.pb.ColumnFamilyTimeRange} + * + *
+   * ColumnFamily Specific TimeRange 
+   * 
+ */ + public static final class ColumnFamilyTimeRange extends + com.google.protobuf.GeneratedMessage + implements ColumnFamilyTimeRangeOrBuilder { + // Use ColumnFamilyTimeRange.newBuilder() to construct. + private ColumnFamilyTimeRange(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private ColumnFamilyTimeRange(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final ColumnFamilyTimeRange defaultInstance; + public static ColumnFamilyTimeRange getDefaultInstance() { + return defaultInstance; + } + + public ColumnFamilyTimeRange getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ColumnFamilyTimeRange( + 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 10: { + bitField0_ |= 0x00000001; + columnFamily_ = input.readBytes(); + break; + } + case 18: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.Builder subBuilder = null; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + subBuilder = timeRange_.toBuilder(); + } + timeRange_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(timeRange_); + timeRange_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + 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.HBaseProtos.internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilyTimeRange_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public ColumnFamilyTimeRange parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ColumnFamilyTimeRange(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required bytes column_family = 1; + public static final int COLUMN_FAMILY_FIELD_NUMBER = 1; + private com.google.protobuf.ByteString columnFamily_; + /** + * required bytes column_family = 1; + */ + public boolean hasColumnFamily() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bytes column_family = 1; + */ + public com.google.protobuf.ByteString getColumnFamily() { + return columnFamily_; + } + + // required .hbase.pb.TimeRange time_range = 2; + public static final int TIME_RANGE_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange timeRange_; + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + public boolean hasTimeRange() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange getTimeRange() { + return timeRange_; + } + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRangeOrBuilder getTimeRangeOrBuilder() { + return timeRange_; + } + + private void initFields() { + columnFamily_ = com.google.protobuf.ByteString.EMPTY; + timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasColumnFamily()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTimeRange()) { + 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.writeBytes(1, columnFamily_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, timeRange_); + } + 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 + .computeBytesSize(1, columnFamily_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, timeRange_); + } + 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(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange) obj; + + boolean result = true; + result = result && (hasColumnFamily() == other.hasColumnFamily()); + if (hasColumnFamily()) { + result = result && getColumnFamily() + .equals(other.getColumnFamily()); + } + result = result && (hasTimeRange() == other.hasTimeRange()); + if (hasTimeRange()) { + result = result && getTimeRange() + .equals(other.getTimeRange()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasColumnFamily()) { + hash = (37 * hash) + COLUMN_FAMILY_FIELD_NUMBER; + hash = (53 * hash) + getColumnFamily().hashCode(); + } + if (hasTimeRange()) { + hash = (37 * hash) + TIME_RANGE_FIELD_NUMBER; + hash = (53 * hash) + getTimeRange().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange 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.HBaseProtos.ColumnFamilyTimeRange parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange 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.HBaseProtos.ColumnFamilyTimeRange parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange 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.HBaseProtos.ColumnFamilyTimeRange parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange 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.HBaseProtos.ColumnFamilyTimeRange parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange 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.HBaseProtos.ColumnFamilyTimeRange 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 hbase.pb.ColumnFamilyTimeRange} + * + *
+     * ColumnFamily Specific TimeRange 
+     * 
+ */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilyTimeRange_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getTimeRangeFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + columnFamily_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + if (timeRangeBuilder_ == null) { + timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance(); + } else { + timeRangeBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange build() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.columnFamily_ = columnFamily_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (timeRangeBuilder_ == null) { + result.timeRange_ = timeRange_; + } else { + result.timeRange_ = timeRangeBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.getDefaultInstance()) return this; + if (other.hasColumnFamily()) { + setColumnFamily(other.getColumnFamily()); + } + if (other.hasTimeRange()) { + mergeTimeRange(other.getTimeRange()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasColumnFamily()) { + + return false; + } + if (!hasTimeRange()) { + + 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.HBaseProtos.ColumnFamilyTimeRange parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required bytes column_family = 1; + private com.google.protobuf.ByteString columnFamily_ = com.google.protobuf.ByteString.EMPTY; + /** + * required bytes column_family = 1; + */ + public boolean hasColumnFamily() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bytes column_family = 1; + */ + public com.google.protobuf.ByteString getColumnFamily() { + return columnFamily_; + } + /** + * required bytes column_family = 1; + */ + public Builder setColumnFamily(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + columnFamily_ = value; + onChanged(); + return this; + } + /** + * required bytes column_family = 1; + */ + public Builder clearColumnFamily() { + bitField0_ = (bitField0_ & ~0x00000001); + columnFamily_ = getDefaultInstance().getColumnFamily(); + onChanged(); + return this; + } + + // required .hbase.pb.TimeRange time_range = 2; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRangeOrBuilder> timeRangeBuilder_; + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + public boolean hasTimeRange() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange getTimeRange() { + if (timeRangeBuilder_ == null) { + return timeRange_; + } else { + return timeRangeBuilder_.getMessage(); + } + } + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + public Builder setTimeRange(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange value) { + if (timeRangeBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + timeRange_ = value; + onChanged(); + } else { + timeRangeBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + public Builder setTimeRange( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.Builder builderForValue) { + if (timeRangeBuilder_ == null) { + timeRange_ = builderForValue.build(); + onChanged(); + } else { + timeRangeBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + public Builder mergeTimeRange(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange value) { + if (timeRangeBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + timeRange_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance()) { + timeRange_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.newBuilder(timeRange_).mergeFrom(value).buildPartial(); + } else { + timeRange_ = value; + } + onChanged(); + } else { + timeRangeBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + public Builder clearTimeRange() { + if (timeRangeBuilder_ == null) { + timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance(); + onChanged(); + } else { + timeRangeBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.Builder getTimeRangeBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getTimeRangeFieldBuilder().getBuilder(); + } + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRangeOrBuilder getTimeRangeOrBuilder() { + if (timeRangeBuilder_ != null) { + return timeRangeBuilder_.getMessageOrBuilder(); + } else { + return timeRange_; + } + } + /** + * required .hbase.pb.TimeRange time_range = 2; + */ + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRangeOrBuilder> + getTimeRangeFieldBuilder() { + if (timeRangeBuilder_ == null) { + timeRangeBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRangeOrBuilder>( + timeRange_, + getParentForChildren(), + isClean()); + timeRange_ = null; + } + return timeRangeBuilder_; + } + + // @@protoc_insertion_point(builder_scope:hbase.pb.ColumnFamilyTimeRange) + } + + static { + defaultInstance = new ColumnFamilyTimeRange(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:hbase.pb.ColumnFamilyTimeRange) + } + public interface ServerNameOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -19301,6 +19963,11 @@ public final class HBaseProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_hbase_pb_TimeRange_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_hbase_pb_ColumnFamilyTimeRange_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_ServerName_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable @@ -19414,42 +20081,44 @@ public final class HBaseProtos { "ionSpecifier.RegionSpecifierType\022\r\n\005valu" + "e\030\002 \002(\014\"?\n\023RegionSpecifierType\022\017\n\013REGION" + "_NAME\020\001\022\027\n\023ENCODED_REGION_NAME\020\002\"%\n\tTime" + - "Range\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(\004\"A\n\nServe" + - "rName\022\021\n\thost_name\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022\022" + - "\n\nstart_code\030\003 \001(\004\"\033\n\013Coprocessor\022\014\n\004nam" + - "e\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004name\030\001 \002(\t\022" + - "\r\n\005value\030\002 \002(\t\",\n\rNameBytesPair\022\014\n\004name\030", - "\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesBytesPair\022\r" + - "\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\",\n\rNameInt" + - "64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002 \001(\003\"\325\001\n\023" + - "SnapshotDescription\022\014\n\004name\030\001 \002(\t\022\r\n\005tab" + - "le\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:\0010\0227\n\004ty" + - "pe\030\004 \001(\0162\".hbase.pb.SnapshotDescription." + - "Type:\005FLUSH\022\017\n\007version\030\005 \001(\005\022\r\n\005owner\030\006 " + - "\001(\t\".\n\004Type\022\014\n\010DISABLED\020\000\022\t\n\005FLUSH\020\001\022\r\n\t" + - "SKIPFLUSH\020\002\"\206\001\n\024ProcedureDescription\022\021\n\t" + - "signature\030\001 \002(\t\022\020\n\010instance\030\002 \001(\t\022\030\n\rcre", - "ation_time\030\003 \001(\003:\0010\022/\n\rconfiguration\030\004 \003" + - "(\0132\030.hbase.pb.NameStringPair\"\n\n\010EmptyMsg" + - "\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 \002(\003\"\037\n\tDoubleM" + - "sg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\rBigDecimalMsg\022" + - "\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n\004UUID\022\026\n\016least" + - "_sig_bits\030\001 \002(\004\022\025\n\rmost_sig_bits\030\002 \002(\004\"T" + - "\n\023NamespaceDescriptor\022\014\n\004name\030\001 \002(\014\022/\n\rc" + - "onfiguration\030\002 \003(\0132\030.hbase.pb.NameString" + - "Pair\"o\n\013VersionInfo\022\017\n\007version\030\001 \002(\t\022\013\n\003" + - "url\030\002 \002(\t\022\020\n\010revision\030\003 \002(\t\022\014\n\004user\030\004 \002(", - "\t\022\014\n\004date\030\005 \002(\t\022\024\n\014src_checksum\030\006 \002(\t\"Q\n" + - "\020RegionServerInfo\022\020\n\010infoPort\030\001 \001(\005\022+\n\014v" + - "ersion_info\030\002 \001(\0132\025.hbase.pb.VersionInfo" + - "*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQU" + - "AL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATE" + - "R_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010" + - "TimeUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECOND" + - "S\020\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MI" + - "NUTES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n*org.apac" + - "he.hadoop.hbase.protobuf.generatedB\013HBas", - "eProtosH\001\240\001\001" + "Range\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(\004\"W\n\025Colum" + + "nFamilyTimeRange\022\025\n\rcolumn_family\030\001 \002(\014\022" + + "\'\n\ntime_range\030\002 \002(\0132\023.hbase.pb.TimeRange" + + "\"A\n\nServerName\022\021\n\thost_name\030\001 \002(\t\022\014\n\004por" + + "t\030\002 \001(\r\022\022\n\nstart_code\030\003 \001(\004\"\033\n\013Coprocess", + "or\022\014\n\004name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004na" + + "me\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesPair" + + "\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesBy" + + "tesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\"," + + "\n\rNameInt64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002" + + " \001(\003\"\325\001\n\023SnapshotDescription\022\014\n\004name\030\001 \002" + + "(\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003" + + ":\0010\0227\n\004type\030\004 \001(\0162\".hbase.pb.SnapshotDes" + + "cription.Type:\005FLUSH\022\017\n\007version\030\005 \001(\005\022\r\n" + + "\005owner\030\006 \001(\t\".\n\004Type\022\014\n\010DISABLED\020\000\022\t\n\005FL", + "USH\020\001\022\r\n\tSKIPFLUSH\020\002\"\206\001\n\024ProcedureDescri" + + "ption\022\021\n\tsignature\030\001 \002(\t\022\020\n\010instance\030\002 \001" + + "(\t\022\030\n\rcreation_time\030\003 \001(\003:\0010\022/\n\rconfigur" + + "ation\030\004 \003(\0132\030.hbase.pb.NameStringPair\"\n\n" + + "\010EmptyMsg\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 \002(\003\"\037" + + "\n\tDoubleMsg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\rBigDe" + + "cimalMsg\022\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n\004UUID" + + "\022\026\n\016least_sig_bits\030\001 \002(\004\022\025\n\rmost_sig_bit" + + "s\030\002 \002(\004\"T\n\023NamespaceDescriptor\022\014\n\004name\030\001" + + " \002(\014\022/\n\rconfiguration\030\002 \003(\0132\030.hbase.pb.N", + "ameStringPair\"o\n\013VersionInfo\022\017\n\007version\030" + + "\001 \002(\t\022\013\n\003url\030\002 \002(\t\022\020\n\010revision\030\003 \002(\t\022\014\n\004" + + "user\030\004 \002(\t\022\014\n\004date\030\005 \002(\t\022\024\n\014src_checksum" + + "\030\006 \002(\t\"Q\n\020RegionServerInfo\022\020\n\010infoPort\030\001" + + " \001(\005\022+\n\014version_info\030\002 \001(\0132\025.hbase.pb.Ve" + + "rsionInfo*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLE" + + "SS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022" + + "\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO" + + "_OP\020\006*n\n\010TimeUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MI" + + "CROSECONDS\020\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECOND", + "S\020\004\022\013\n\007MINUTES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n" + + "*org.apache.hadoop.hbase.protobuf.genera" + + "tedB\013HBaseProtosH\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -19510,98 +20179,104 @@ public final class HBaseProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_TimeRange_descriptor, new java.lang.String[] { "From", "To", }); - internal_static_hbase_pb_ServerName_descriptor = + internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor = getDescriptor().getMessageTypes().get(9); + internal_static_hbase_pb_ColumnFamilyTimeRange_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor, + new java.lang.String[] { "ColumnFamily", "TimeRange", }); + internal_static_hbase_pb_ServerName_descriptor = + getDescriptor().getMessageTypes().get(10); internal_static_hbase_pb_ServerName_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_ServerName_descriptor, new java.lang.String[] { "HostName", "Port", "StartCode", }); internal_static_hbase_pb_Coprocessor_descriptor = - getDescriptor().getMessageTypes().get(10); + getDescriptor().getMessageTypes().get(11); internal_static_hbase_pb_Coprocessor_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_Coprocessor_descriptor, new java.lang.String[] { "Name", }); internal_static_hbase_pb_NameStringPair_descriptor = - getDescriptor().getMessageTypes().get(11); + getDescriptor().getMessageTypes().get(12); internal_static_hbase_pb_NameStringPair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_NameStringPair_descriptor, new java.lang.String[] { "Name", "Value", }); internal_static_hbase_pb_NameBytesPair_descriptor = - getDescriptor().getMessageTypes().get(12); + getDescriptor().getMessageTypes().get(13); internal_static_hbase_pb_NameBytesPair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_NameBytesPair_descriptor, new java.lang.String[] { "Name", "Value", }); internal_static_hbase_pb_BytesBytesPair_descriptor = - getDescriptor().getMessageTypes().get(13); + getDescriptor().getMessageTypes().get(14); internal_static_hbase_pb_BytesBytesPair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_BytesBytesPair_descriptor, new java.lang.String[] { "First", "Second", }); internal_static_hbase_pb_NameInt64Pair_descriptor = - getDescriptor().getMessageTypes().get(14); + getDescriptor().getMessageTypes().get(15); internal_static_hbase_pb_NameInt64Pair_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_NameInt64Pair_descriptor, new java.lang.String[] { "Name", "Value", }); internal_static_hbase_pb_SnapshotDescription_descriptor = - getDescriptor().getMessageTypes().get(15); + getDescriptor().getMessageTypes().get(16); internal_static_hbase_pb_SnapshotDescription_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_SnapshotDescription_descriptor, new java.lang.String[] { "Name", "Table", "CreationTime", "Type", "Version", "Owner", }); internal_static_hbase_pb_ProcedureDescription_descriptor = - getDescriptor().getMessageTypes().get(16); + getDescriptor().getMessageTypes().get(17); internal_static_hbase_pb_ProcedureDescription_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_ProcedureDescription_descriptor, new java.lang.String[] { "Signature", "Instance", "CreationTime", "Configuration", }); internal_static_hbase_pb_EmptyMsg_descriptor = - getDescriptor().getMessageTypes().get(17); + getDescriptor().getMessageTypes().get(18); internal_static_hbase_pb_EmptyMsg_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_EmptyMsg_descriptor, new java.lang.String[] { }); internal_static_hbase_pb_LongMsg_descriptor = - getDescriptor().getMessageTypes().get(18); + getDescriptor().getMessageTypes().get(19); internal_static_hbase_pb_LongMsg_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_LongMsg_descriptor, new java.lang.String[] { "LongMsg", }); internal_static_hbase_pb_DoubleMsg_descriptor = - getDescriptor().getMessageTypes().get(19); + getDescriptor().getMessageTypes().get(20); internal_static_hbase_pb_DoubleMsg_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_DoubleMsg_descriptor, new java.lang.String[] { "DoubleMsg", }); internal_static_hbase_pb_BigDecimalMsg_descriptor = - getDescriptor().getMessageTypes().get(20); + getDescriptor().getMessageTypes().get(21); internal_static_hbase_pb_BigDecimalMsg_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_BigDecimalMsg_descriptor, new java.lang.String[] { "BigdecimalMsg", }); internal_static_hbase_pb_UUID_descriptor = - getDescriptor().getMessageTypes().get(21); + getDescriptor().getMessageTypes().get(22); internal_static_hbase_pb_UUID_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_UUID_descriptor, new java.lang.String[] { "LeastSigBits", "MostSigBits", }); internal_static_hbase_pb_NamespaceDescriptor_descriptor = - getDescriptor().getMessageTypes().get(22); + getDescriptor().getMessageTypes().get(23); internal_static_hbase_pb_NamespaceDescriptor_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_NamespaceDescriptor_descriptor, new java.lang.String[] { "Name", "Configuration", }); internal_static_hbase_pb_VersionInfo_descriptor = - getDescriptor().getMessageTypes().get(23); + getDescriptor().getMessageTypes().get(24); internal_static_hbase_pb_VersionInfo_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_VersionInfo_descriptor, new java.lang.String[] { "Version", "Url", "Revision", "User", "Date", "SrcChecksum", }); internal_static_hbase_pb_RegionServerInfo_descriptor = - getDescriptor().getMessageTypes().get(24); + getDescriptor().getMessageTypes().get(25); internal_static_hbase_pb_RegionServerInfo_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_RegionServerInfo_descriptor, diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto index 101854d..805414f 100644 --- a/hbase-protocol/src/main/protobuf/Client.proto +++ b/hbase-protocol/src/main/protobuf/Client.proto @@ -82,6 +82,7 @@ message Get { optional bool existence_only = 10 [default = false]; optional Consistency consistency = 12 [default = STRONG]; + repeated ColumnFamilyTimeRange cf_time_range = 13; } message Result { @@ -251,6 +252,7 @@ message Scan { optional bool reversed = 15 [default = false]; optional Consistency consistency = 16 [default = STRONG]; optional uint32 caching = 17; + repeated ColumnFamilyTimeRange cf_time_range = 18; } /** diff --git a/hbase-protocol/src/main/protobuf/HBase.proto b/hbase-protocol/src/main/protobuf/HBase.proto index 2603efa..6cb42b1 100644 --- a/hbase-protocol/src/main/protobuf/HBase.proto +++ b/hbase-protocol/src/main/protobuf/HBase.proto @@ -123,6 +123,12 @@ message TimeRange { optional uint64 to = 2; } +/* ColumnFamily Specific TimeRange */ +message ColumnFamilyTimeRange { + required bytes column_family = 1; + required TimeRange time_range = 2; +} + /* Comparison operators */ enum CompareType { LESS = 0; 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 11d71cf..84e9bac 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 @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; +import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.io.hfile.BlockType; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; @@ -1163,9 +1164,14 @@ public class StoreFile { public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread, boolean isCompaction, long readPt) { - return new StoreFileScanner(this, - getScanner(cacheBlocks, pread, isCompaction), - !isCompaction, reader.hasMVCCInfo(), readPt); + Cell kv = reader.getFirstKey(); + byte[] cf = null; + if (kv != null) { + cf = new byte[kv.getFamilyLength()]; + System.arraycopy(kv.getFamilyArray(), kv.getFamilyOffset(), cf, 0, kv.getFamilyLength()); + } + return new StoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction), cf, + !isCompaction, reader.hasMVCCInfo(), readPt); } /** @@ -1208,16 +1214,16 @@ public class StoreFile { /** * Check if this storeFile may contain keys within the TimeRange that * have not expired (i.e. not older than oldestUnexpiredTS). - * @param scan the current scan + * @param timeRange the timeRange to restrict * @param oldestUnexpiredTS the oldest timestamp that is not expired, as * determined by the column family's TTL * @return false if queried keys definitely don't exist in this StoreFile */ - boolean passesTimerangeFilter(Scan scan, long oldestUnexpiredTS) { + boolean passesTimerangeFilter(TimeRange timeRange, long oldestUnexpiredTS) { if (timeRangeTracker == null) { return true; } else { - return timeRangeTracker.includesTimeRange(scan.getTimeRange()) && + return timeRangeTracker.includesTimeRange(timeRange) && timeRangeTracker.getMaximumTimestamp() >= oldestUnexpiredTS; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java index f9e1a3c..3d5997c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java @@ -27,14 +27,15 @@ import java.util.List; import java.util.SortedSet; import java.util.concurrent.atomic.AtomicLong; -import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.regionserver.StoreFile.Reader; @@ -47,6 +48,7 @@ public class StoreFileScanner implements KeyValueScanner { // the reader it comes from: private final StoreFile.Reader reader; private final HFileScanner hfs; + private final byte[] columnFamily; private Cell cur = null; private boolean realSeekDone; @@ -62,7 +64,7 @@ public class StoreFileScanner implements KeyValueScanner { private static AtomicLong seekCount; private ScanQueryMatcher matcher; - + private long readPt; /** @@ -71,9 +73,15 @@ public class StoreFileScanner implements KeyValueScanner { */ public StoreFileScanner(StoreFile.Reader reader, HFileScanner hfs, boolean useMVCC, boolean hasMVCC, long readPt) { + this(reader, hfs, null, useMVCC, hasMVCC, readPt); + } + + public StoreFileScanner(StoreFile.Reader reader, HFileScanner hfs, byte[] columnFamily, + boolean useMVCC, boolean hasMVCC, long readPt) { this.readPt = readPt; this.reader = reader; this.hfs = hfs; + this.columnFamily = columnFamily; this.enforceMVCC = useMVCC; this.hasMVCCInfo = hasMVCC; } @@ -423,8 +431,17 @@ public class StoreFileScanner implements KeyValueScanner { @Override public boolean shouldUseScanner(Scan scan, SortedSet columns, long oldestUnexpiredTS) { - return reader.passesTimerangeFilter(scan, oldestUnexpiredTS) - && reader.passesKeyRangeFilter(scan) && reader.passesBloomFilter(scan, columns); + // if the file has no entries, no need to validate or create a scanner. + if (reader.getFirstKey() == null) { + return false; + } + validateScanRestrictions(scan); + TimeRange timeRange = scan.getColumnFamilyTimeRange().get(columnFamily); + if (timeRange == null) { + timeRange = scan.getTimeRange(); + } + return reader.passesTimerangeFilter(timeRange, oldestUnexpiredTS) && reader + .passesKeyRangeFilter(scan) && reader.passesBloomFilter(scan, columns); } @Override @@ -511,4 +528,15 @@ public class StoreFileScanner implements KeyValueScanner { public void shipped() throws IOException { this.hfs.shipped(); } + + // if we have a scan that restricts per column family time ranges but our + // store file scanner does not have the HColumnDescriptor set we can not + // figure out which CF we are scanning thus we throw an exception + private void validateScanRestrictions(Scan scan) { + if (!scan.getColumnFamilyTimeRange().isEmpty() && this.columnFamily == null) { + throw new UnsupportedOperationException("Scan is trying to restrict time ranges on a per " + + "column family basis but null column family passed to the StoreFileScannerConstructor"); + } + + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java index b763a22..15998b0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java @@ -69,6 +69,9 @@ import com.google.common.base.Joiner; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + /** * Test HStoreFile */ @@ -197,6 +200,18 @@ public class TestStoreFile extends HBaseTestCase { } @Test + public void testEmptyStoreFileRestrictKeyRanges() throws Exception { + StoreFile.Reader reader = mock(StoreFile.Reader.class); + when(reader.getFirstKey()).thenReturn(null); + byte[] cf = Bytes.toBytes("ty"); + StoreFileScanner scanner = new StoreFileScanner(reader, mock(HFileScanner.class), cf, + false, false, 0); + Scan scan = new Scan(); + scan.setColumnFamilyTimeRange(cf, 0, 1); + assertFalse(scanner.shouldUseScanner(scan, null, 0)); + } + + @Test public void testHFileLink() throws IOException { final HRegionInfo hri = new HRegionInfo(TableName.valueOf("testHFileLinkTb")); // force temp data in hbase/target/test-data instead of /tmp/hbase-xxxx/ @@ -501,9 +516,8 @@ public class TestStoreFile extends HBaseTestCase { fs.delete(f, true); assertEquals("False negatives: " + falseNeg, 0, falseNeg); int maxFalsePos = (int) (2 * 2000 * err); - assertTrue("Too many false positives: " + falsePos + " (err=" + err - + ", expected no more than " + maxFalsePos + ")", - falsePos <= maxFalsePos); + assertTrue("Too many false positives: " + falsePos + " (err=" + err + ", expected no more than " + + maxFalsePos + ")", falsePos <= maxFalsePos); } private static final int BLOCKSIZE_SMALL = 8192; @@ -708,15 +722,12 @@ public class TestStoreFile extends HBaseTestCase { @Test public void testSeqIdComparator() { - assertOrdering(StoreFile.Comparators.SEQ_ID, - mockStoreFile(true, 100, 1000, -1, "/foo/123"), - mockStoreFile(true, 100, 1000, -1, "/foo/124"), - mockStoreFile(true, 99, 1000, -1, "/foo/126"), - mockStoreFile(true, 98, 2000, -1, "/foo/126"), - mockStoreFile(false, 3453, -1, 1, "/foo/1"), - mockStoreFile(false, 2, -1, 3, "/foo/2"), - mockStoreFile(false, 1000, -1, 5, "/foo/2"), - mockStoreFile(false, 76, -1, 5, "/foo/3")); + assertOrdering(StoreFile.Comparators.SEQ_ID, mockStoreFile(true, 100, 1000, -1, "/foo/123"), + mockStoreFile(true, 100, 1000, -1, "/foo/124"), + mockStoreFile(true, 99, 1000, -1, "/foo/126"), + mockStoreFile(true, 98, 2000, -1, "/foo/126"), mockStoreFile(false, 3453, -1, 1, "/foo/1"), + mockStoreFile(false, 2, -1, 3, "/foo/2"), mockStoreFile(false, 1000, -1, 5, "/foo/2"), + mockStoreFile(false, 76, -1, 5, "/foo/3")); } /** @@ -794,7 +805,7 @@ public class TestStoreFile extends HBaseTestCase { Scan scan = new Scan(); // Make up a directory hierarchy that has a regiondir ("7e0102") and familyname. - Path storedir = new Path(new Path(testDir, "7e0102"), "familyname"); + Path storedir = new Path(new Path(testDir, "7e0102"), Bytes.toString(family)); Path dir = new Path(storedir, "1234567890"); HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); // Make a store file and write data to it. @@ -804,7 +815,7 @@ public class TestStoreFile extends HBaseTestCase { .build(); List kvList = getKeyValueSet(timestamps,numRows, - family, qualifier); + qualifier, family); for (KeyValue kv : kvList) { writer.append(kv); @@ -828,12 +839,21 @@ public class TestStoreFile extends HBaseTestCase { scan.setTimeRange(8, 10); assertTrue(scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE)); - scan.setTimeRange(7, 50); + // lets make sure it still works with column family time ranges + scan.setColumnFamilyTimeRange(family, 7, 50); assertTrue(scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE)); // This test relies on the timestamp range optimization + scan = new Scan(); scan.setTimeRange(27, 50); assertTrue(!scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE)); + + // should still use the scanner because we override the family time range + scan = new Scan(); + scan.setTimeRange(27, 50); + scan.setColumnFamilyTimeRange(family, 7, 50); + assertTrue(scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE)); + } @Test -- 2.1.0