HBASE-11026-Provide option to filter out all rows in PerformanceEvaluation tool (Ram)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1589447 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
891b9f6a20
commit
43b9435dcb
|
@ -15543,6 +15543,344 @@ public final class FilterProtos {
|
||||||
// @@protoc_insertion_point(class_scope:WhileMatchFilter)
|
// @@protoc_insertion_point(class_scope:WhileMatchFilter)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public interface FilterAllFilterOrBuilder
|
||||||
|
extends com.google.protobuf.MessageOrBuilder {
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* Protobuf type {@code FilterAllFilter}
|
||||||
|
*/
|
||||||
|
public static final class FilterAllFilter extends
|
||||||
|
com.google.protobuf.GeneratedMessage
|
||||||
|
implements FilterAllFilterOrBuilder {
|
||||||
|
// Use FilterAllFilter.newBuilder() to construct.
|
||||||
|
private FilterAllFilter(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
|
||||||
|
super(builder);
|
||||||
|
this.unknownFields = builder.getUnknownFields();
|
||||||
|
}
|
||||||
|
private FilterAllFilter(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
|
||||||
|
|
||||||
|
private static final FilterAllFilter defaultInstance;
|
||||||
|
public static FilterAllFilter getDefaultInstance() {
|
||||||
|
return defaultInstance;
|
||||||
|
}
|
||||||
|
|
||||||
|
public FilterAllFilter getDefaultInstanceForType() {
|
||||||
|
return defaultInstance;
|
||||||
|
}
|
||||||
|
|
||||||
|
private final com.google.protobuf.UnknownFieldSet unknownFields;
|
||||||
|
@java.lang.Override
|
||||||
|
public final com.google.protobuf.UnknownFieldSet
|
||||||
|
getUnknownFields() {
|
||||||
|
return this.unknownFields;
|
||||||
|
}
|
||||||
|
private FilterAllFilter(
|
||||||
|
com.google.protobuf.CodedInputStream input,
|
||||||
|
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||||
|
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||||
|
initFields();
|
||||||
|
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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} 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.FilterProtos.internal_static_FilterAllFilter_descriptor;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||||
|
internalGetFieldAccessorTable() {
|
||||||
|
return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FilterAllFilter_fieldAccessorTable
|
||||||
|
.ensureFieldAccessorsInitialized(
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter.Builder.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static com.google.protobuf.Parser<FilterAllFilter> PARSER =
|
||||||
|
new com.google.protobuf.AbstractParser<FilterAllFilter>() {
|
||||||
|
public FilterAllFilter parsePartialFrom(
|
||||||
|
com.google.protobuf.CodedInputStream input,
|
||||||
|
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||||
|
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||||
|
return new FilterAllFilter(input, extensionRegistry);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
@java.lang.Override
|
||||||
|
public com.google.protobuf.Parser<FilterAllFilter> getParserForType() {
|
||||||
|
return PARSER;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void initFields() {
|
||||||
|
}
|
||||||
|
private byte memoizedIsInitialized = -1;
|
||||||
|
public final boolean isInitialized() {
|
||||||
|
byte isInitialized = memoizedIsInitialized;
|
||||||
|
if (isInitialized != -1) return isInitialized == 1;
|
||||||
|
|
||||||
|
memoizedIsInitialized = 1;
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void writeTo(com.google.protobuf.CodedOutputStream output)
|
||||||
|
throws java.io.IOException {
|
||||||
|
getSerializedSize();
|
||||||
|
getUnknownFields().writeTo(output);
|
||||||
|
}
|
||||||
|
|
||||||
|
private int memoizedSerializedSize = -1;
|
||||||
|
public int getSerializedSize() {
|
||||||
|
int size = memoizedSerializedSize;
|
||||||
|
if (size != -1) return size;
|
||||||
|
|
||||||
|
size = 0;
|
||||||
|
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.FilterProtos.FilterAllFilter)) {
|
||||||
|
return super.equals(obj);
|
||||||
|
}
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter other = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter) obj;
|
||||||
|
|
||||||
|
boolean result = true;
|
||||||
|
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();
|
||||||
|
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||||
|
memoizedHashCode = hash;
|
||||||
|
return hash;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter parseFrom(
|
||||||
|
com.google.protobuf.ByteString data)
|
||||||
|
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||||
|
return PARSER.parseFrom(data);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter 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.FilterProtos.FilterAllFilter parseFrom(byte[] data)
|
||||||
|
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||||
|
return PARSER.parseFrom(data);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter 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.FilterProtos.FilterAllFilter parseFrom(java.io.InputStream input)
|
||||||
|
throws java.io.IOException {
|
||||||
|
return PARSER.parseFrom(input);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter 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.FilterProtos.FilterAllFilter parseDelimitedFrom(java.io.InputStream input)
|
||||||
|
throws java.io.IOException {
|
||||||
|
return PARSER.parseDelimitedFrom(input);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter 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.FilterProtos.FilterAllFilter parseFrom(
|
||||||
|
com.google.protobuf.CodedInputStream input)
|
||||||
|
throws java.io.IOException {
|
||||||
|
return PARSER.parseFrom(input);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter 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.FilterProtos.FilterAllFilter 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 FilterAllFilter}
|
||||||
|
*/
|
||||||
|
public static final class Builder extends
|
||||||
|
com.google.protobuf.GeneratedMessage.Builder<Builder>
|
||||||
|
implements org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilterOrBuilder {
|
||||||
|
public static final com.google.protobuf.Descriptors.Descriptor
|
||||||
|
getDescriptor() {
|
||||||
|
return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FilterAllFilter_descriptor;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||||
|
internalGetFieldAccessorTable() {
|
||||||
|
return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FilterAllFilter_fieldAccessorTable
|
||||||
|
.ensureFieldAccessorsInitialized(
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter.Builder.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Construct using org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter.newBuilder()
|
||||||
|
private Builder() {
|
||||||
|
maybeForceBuilderInitialization();
|
||||||
|
}
|
||||||
|
|
||||||
|
private Builder(
|
||||||
|
com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
||||||
|
super(parent);
|
||||||
|
maybeForceBuilderInitialization();
|
||||||
|
}
|
||||||
|
private void maybeForceBuilderInitialization() {
|
||||||
|
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
private static Builder create() {
|
||||||
|
return new Builder();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder clear() {
|
||||||
|
super.clear();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder clone() {
|
||||||
|
return create().mergeFrom(buildPartial());
|
||||||
|
}
|
||||||
|
|
||||||
|
public com.google.protobuf.Descriptors.Descriptor
|
||||||
|
getDescriptorForType() {
|
||||||
|
return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FilterAllFilter_descriptor;
|
||||||
|
}
|
||||||
|
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter getDefaultInstanceForType() {
|
||||||
|
return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter.getDefaultInstance();
|
||||||
|
}
|
||||||
|
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter build() {
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter result = buildPartial();
|
||||||
|
if (!result.isInitialized()) {
|
||||||
|
throw newUninitializedMessageException(result);
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter buildPartial() {
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter result = new org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter(this);
|
||||||
|
onBuilt();
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder mergeFrom(com.google.protobuf.Message other) {
|
||||||
|
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter) {
|
||||||
|
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter)other);
|
||||||
|
} else {
|
||||||
|
super.mergeFrom(other);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter other) {
|
||||||
|
if (other == org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter.getDefaultInstance()) return this;
|
||||||
|
this.mergeUnknownFields(other.getUnknownFields());
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public final boolean isInitialized() {
|
||||||
|
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.FilterProtos.FilterAllFilter parsedMessage = null;
|
||||||
|
try {
|
||||||
|
parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
|
||||||
|
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
|
||||||
|
parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FilterAllFilter) e.getUnfinishedMessage();
|
||||||
|
throw e;
|
||||||
|
} finally {
|
||||||
|
if (parsedMessage != null) {
|
||||||
|
mergeFrom(parsedMessage);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
// @@protoc_insertion_point(builder_scope:FilterAllFilter)
|
||||||
|
}
|
||||||
|
|
||||||
|
static {
|
||||||
|
defaultInstance = new FilterAllFilter(true);
|
||||||
|
defaultInstance.initFields();
|
||||||
|
}
|
||||||
|
|
||||||
|
// @@protoc_insertion_point(class_scope:FilterAllFilter)
|
||||||
|
}
|
||||||
|
|
||||||
private static com.google.protobuf.Descriptors.Descriptor
|
private static com.google.protobuf.Descriptors.Descriptor
|
||||||
internal_static_Filter_descriptor;
|
internal_static_Filter_descriptor;
|
||||||
private static
|
private static
|
||||||
|
@ -15678,6 +16016,11 @@ public final class FilterProtos {
|
||||||
private static
|
private static
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||||
internal_static_WhileMatchFilter_fieldAccessorTable;
|
internal_static_WhileMatchFilter_fieldAccessorTable;
|
||||||
|
private static com.google.protobuf.Descriptors.Descriptor
|
||||||
|
internal_static_FilterAllFilter_descriptor;
|
||||||
|
private static
|
||||||
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||||
|
internal_static_FilterAllFilter_fieldAccessorTable;
|
||||||
|
|
||||||
public static com.google.protobuf.Descriptors.FileDescriptor
|
public static com.google.protobuf.Descriptors.FileDescriptor
|
||||||
getDescriptor() {
|
getDescriptor() {
|
||||||
|
@ -15732,8 +16075,9 @@ public final class FilterProtos {
|
||||||
"\022\026\n\ntimestamps\030\001 \003(\003B\002\020\001\"5\n\013ValueFilter\022" +
|
"\022\026\n\ntimestamps\030\001 \003(\003B\002\020\001\"5\n\013ValueFilter\022" +
|
||||||
"&\n\016compare_filter\030\001 \002(\0132\016.CompareFilter\"" +
|
"&\n\016compare_filter\030\001 \002(\0132\016.CompareFilter\"" +
|
||||||
"+\n\020WhileMatchFilter\022\027\n\006filter\030\001 \002(\0132\007.Fi" +
|
"+\n\020WhileMatchFilter\022\027\n\006filter\030\001 \002(\0132\007.Fi" +
|
||||||
"lterBB\n*org.apache.hadoop.hbase.protobuf" +
|
"lter\"\021\n\017FilterAllFilterBB\n*org.apache.ha" +
|
||||||
".generatedB\014FilterProtosH\001\210\001\001\240\001\001"
|
"doop.hbase.protobuf.generatedB\014FilterPro" +
|
||||||
|
"tosH\001\210\001\001\240\001\001"
|
||||||
};
|
};
|
||||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||||
|
@ -15902,6 +16246,12 @@ public final class FilterProtos {
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
internal_static_WhileMatchFilter_descriptor,
|
internal_static_WhileMatchFilter_descriptor,
|
||||||
new java.lang.String[] { "Filter", });
|
new java.lang.String[] { "Filter", });
|
||||||
|
internal_static_FilterAllFilter_descriptor =
|
||||||
|
getDescriptor().getMessageTypes().get(27);
|
||||||
|
internal_static_FilterAllFilter_fieldAccessorTable = new
|
||||||
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
|
internal_static_FilterAllFilter_descriptor,
|
||||||
|
new java.lang.String[] { });
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
|
@ -154,3 +154,5 @@ message ValueFilter {
|
||||||
message WhileMatchFilter {
|
message WhileMatchFilter {
|
||||||
required Filter filter = 1;
|
required Filter filter = 1;
|
||||||
}
|
}
|
||||||
|
message FilterAllFilter {
|
||||||
|
}
|
|
@ -18,6 +18,8 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase;
|
package org.apache.hadoop.hbase;
|
||||||
|
|
||||||
|
import static org.codehaus.jackson.map.SerializationConfig.Feature.SORT_PROPERTIES_ALPHABETICALLY;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.PrintStream;
|
import java.io.PrintStream;
|
||||||
import java.lang.reflect.Constructor;
|
import java.lang.reflect.Constructor;
|
||||||
|
@ -37,10 +39,6 @@ import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
|
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
||||||
import com.yammer.metrics.core.Histogram;
|
|
||||||
import com.yammer.metrics.core.MetricsRegistry;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -60,6 +58,8 @@ import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
||||||
import org.apache.hadoop.hbase.filter.CompareFilter;
|
import org.apache.hadoop.hbase.filter.CompareFilter;
|
||||||
import org.apache.hadoop.hbase.filter.Filter;
|
import org.apache.hadoop.hbase.filter.Filter;
|
||||||
|
import org.apache.hadoop.hbase.filter.FilterAllFilter;
|
||||||
|
import org.apache.hadoop.hbase.filter.FilterList;
|
||||||
import org.apache.hadoop.hbase.filter.PageFilter;
|
import org.apache.hadoop.hbase.filter.PageFilter;
|
||||||
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
|
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
|
||||||
import org.apache.hadoop.hbase.filter.WhileMatchFilter;
|
import org.apache.hadoop.hbase.filter.WhileMatchFilter;
|
||||||
|
@ -81,7 +81,9 @@ import org.apache.hadoop.util.Tool;
|
||||||
import org.apache.hadoop.util.ToolRunner;
|
import org.apache.hadoop.util.ToolRunner;
|
||||||
import org.codehaus.jackson.map.ObjectMapper;
|
import org.codehaus.jackson.map.ObjectMapper;
|
||||||
|
|
||||||
import static org.codehaus.jackson.map.SerializationConfig.Feature.SORT_PROPERTIES_ALPHABETICALLY;
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
|
import com.yammer.metrics.core.Histogram;
|
||||||
|
import com.yammer.metrics.core.MetricsRegistry;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Script used evaluating HBase performance and scalability. Runs a HBase
|
* Script used evaluating HBase performance and scalability. Runs a HBase
|
||||||
|
@ -500,9 +502,11 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
||||||
this.presplitRegions = that.presplitRegions;
|
this.presplitRegions = that.presplitRegions;
|
||||||
this.compression = that.compression;
|
this.compression = that.compression;
|
||||||
this.blockEncoding = that.blockEncoding;
|
this.blockEncoding = that.blockEncoding;
|
||||||
|
this.filterAll = that.filterAll;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean nomapred = false;
|
public boolean nomapred = false;
|
||||||
|
public boolean filterAll = false;
|
||||||
public int startRow = 0;
|
public int startRow = 0;
|
||||||
public int perClientRunRows = ROWS_PER_GB;
|
public int perClientRunRows = ROWS_PER_GB;
|
||||||
public int numClientThreads = 1;
|
public int numClientThreads = 1;
|
||||||
|
@ -659,8 +663,13 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
||||||
@Override
|
@Override
|
||||||
void testRow(final int i) throws IOException {
|
void testRow(final int i) throws IOException {
|
||||||
Scan scan = new Scan(getRandomRow(this.rand, opts.modulo));
|
Scan scan = new Scan(getRandomRow(this.rand, opts.modulo));
|
||||||
|
FilterList list = new FilterList();
|
||||||
scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
||||||
scan.setFilter(new WhileMatchFilter(new PageFilter(120)));
|
if (opts.filterAll) {
|
||||||
|
list.addFilter(new FilterAllFilter());
|
||||||
|
}
|
||||||
|
list.addFilter(new WhileMatchFilter(new PageFilter(120)));
|
||||||
|
scan.setFilter(list);
|
||||||
ResultScanner s = this.table.getScanner(scan);
|
ResultScanner s = this.table.getScanner(scan);
|
||||||
for (Result rr; (rr = s.next()) != null;) ;
|
for (Result rr; (rr = s.next()) != null;) ;
|
||||||
s.close();
|
s.close();
|
||||||
|
@ -684,6 +693,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
||||||
void testRow(final int i) throws IOException {
|
void testRow(final int i) throws IOException {
|
||||||
Pair<byte[], byte[]> startAndStopRow = getStartAndStopRow();
|
Pair<byte[], byte[]> startAndStopRow = getStartAndStopRow();
|
||||||
Scan scan = new Scan(startAndStopRow.getFirst(), startAndStopRow.getSecond());
|
Scan scan = new Scan(startAndStopRow.getFirst(), startAndStopRow.getSecond());
|
||||||
|
if (opts.filterAll) {
|
||||||
|
scan.setFilter(new FilterAllFilter());
|
||||||
|
}
|
||||||
scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
||||||
ResultScanner s = this.table.getScanner(scan);
|
ResultScanner s = this.table.getScanner(scan);
|
||||||
int count = 0;
|
int count = 0;
|
||||||
|
@ -774,6 +786,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
||||||
void testRow(final int i) throws IOException {
|
void testRow(final int i) throws IOException {
|
||||||
Get get = new Get(getRandomRow(this.rand, opts.modulo));
|
Get get = new Get(getRandomRow(this.rand, opts.modulo));
|
||||||
get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
||||||
|
if (opts.filterAll) {
|
||||||
|
get.setFilter(new FilterAllFilter());
|
||||||
|
}
|
||||||
if (LOG.isTraceEnabled()) LOG.trace(get.toString());
|
if (LOG.isTraceEnabled()) LOG.trace(get.toString());
|
||||||
if (opts.multiGet > 0) {
|
if (opts.multiGet > 0) {
|
||||||
this.gets.add(get);
|
this.gets.add(get);
|
||||||
|
@ -853,7 +868,10 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
||||||
Scan scan = new Scan(format(opts.startRow));
|
Scan scan = new Scan(format(opts.startRow));
|
||||||
scan.setCaching(30);
|
scan.setCaching(30);
|
||||||
scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
||||||
this.testScanner = table.getScanner(scan);
|
if (opts.filterAll) {
|
||||||
|
scan.setFilter(new FilterAllFilter());
|
||||||
|
}
|
||||||
|
this.testScanner = table.getScanner(scan);
|
||||||
}
|
}
|
||||||
testScanner.next();
|
testScanner.next();
|
||||||
}
|
}
|
||||||
|
@ -869,6 +887,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
||||||
void testRow(final int i) throws IOException {
|
void testRow(final int i) throws IOException {
|
||||||
Get get = new Get(format(i));
|
Get get = new Get(format(i));
|
||||||
get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
||||||
|
if (opts.filterAll) {
|
||||||
|
get.setFilter(new FilterAllFilter());
|
||||||
|
}
|
||||||
table.get(get);
|
table.get(get);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -923,13 +944,18 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected Scan constructScan(byte[] valuePrefix) throws IOException {
|
protected Scan constructScan(byte[] valuePrefix) throws IOException {
|
||||||
|
FilterList list = new FilterList();
|
||||||
Filter filter = new SingleColumnValueFilter(
|
Filter filter = new SingleColumnValueFilter(
|
||||||
FAMILY_NAME, QUALIFIER_NAME, CompareFilter.CompareOp.EQUAL,
|
FAMILY_NAME, QUALIFIER_NAME, CompareFilter.CompareOp.EQUAL,
|
||||||
new BinaryComparator(valuePrefix)
|
new BinaryComparator(valuePrefix)
|
||||||
);
|
);
|
||||||
|
list.addFilter(filter);
|
||||||
|
if(opts.filterAll) {
|
||||||
|
list.addFilter(new FilterAllFilter());
|
||||||
|
}
|
||||||
Scan scan = new Scan();
|
Scan scan = new Scan();
|
||||||
scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
|
||||||
scan.setFilter(filter);
|
scan.setFilter(list);
|
||||||
return scan;
|
return scan;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1078,6 +1104,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
||||||
"Default: false");
|
"Default: false");
|
||||||
System.err.println(" numoftags Specify the no of tags that would be needed. " +
|
System.err.println(" numoftags Specify the no of tags that would be needed. " +
|
||||||
"This works only if usetags is true.");
|
"This works only if usetags is true.");
|
||||||
|
System.err.println(" filterAll Helps to filter out all the rows on the server side"
|
||||||
|
+ " there by not returning any thing back to the client. Helps to check the server side"
|
||||||
|
+ " performance. Uses FilterAllFilter internally. ");
|
||||||
System.err.println(" latency Set to report operation latencies. Default: False");
|
System.err.println(" latency Set to report operation latencies. Default: False");
|
||||||
System.err.println();
|
System.err.println();
|
||||||
System.err.println(" Note: -D properties will be applied to the conf used. ");
|
System.err.println(" Note: -D properties will be applied to the conf used. ");
|
||||||
|
@ -1227,6 +1256,12 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final String filterOutAll = "--filterAll";
|
||||||
|
if (cmd.startsWith(filterOutAll)) {
|
||||||
|
opts.filterAll = true;
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
final String modulo = "--modulo=";
|
final String modulo = "--modulo=";
|
||||||
if (cmd.startsWith(modulo)) {
|
if (cmd.startsWith(modulo)) {
|
||||||
opts.modulo = Integer.parseInt(cmd.substring(modulo.length()));
|
opts.modulo = Integer.parseInt(cmd.substring(modulo.length()));
|
||||||
|
|
|
@ -0,0 +1,62 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.filter;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.Cell;
|
||||||
|
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
|
||||||
|
|
||||||
|
import com.google.protobuf.InvalidProtocolBufferException;
|
||||||
|
|
||||||
|
public class FilterAllFilter extends FilterBase {
|
||||||
|
|
||||||
|
public FilterAllFilter() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ReturnCode filterKeyValue(Cell v) throws IOException {
|
||||||
|
return ReturnCode.INCLUDE;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean filterRow() throws IOException {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static FilterAllFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||||
|
FilterProtos.FilterAllFilter proto;
|
||||||
|
try {
|
||||||
|
proto = FilterProtos.FilterAllFilter.parseFrom(pbBytes);
|
||||||
|
} catch (InvalidProtocolBufferException e) {
|
||||||
|
throw new DeserializationException(e);
|
||||||
|
}
|
||||||
|
return new FilterAllFilter();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
boolean areSerializedFieldsEqual(Filter o) {
|
||||||
|
if (o == this)
|
||||||
|
return true;
|
||||||
|
if (!(o instanceof FilterAllFilter))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue