From 274fbe9a8402a8094d5d808a148dff2cea4c91df Mon Sep 17 00:00:00 2001 From: Zhiqiu Kong Date: Tue, 31 Jan 2012 11:25:20 -0800 Subject: [PATCH] [jira] [HBASE-4542] Add filter info to slow query logging Summary: Slow opertaion log does not provide enough information when a filter is present. The followings are done to add the filter info: 1) Added toString() method for filters inheriting FilterBase, this affect 22 filters and their subclasses. The info added includes the filter's name and its members. For example, for TimestampsFilter, we'll output its class name as well as the defined timestamps. 2) Added a field 'filter' in Get::toMap() and Scan::toMap() to enable the logging of filter info. Task ID: #750975 Blame Rev: Test Plan: 1. Run and passed unit-tests to make sure it does not break things 2. Run kannan's script to trigger the slow operation logging, checked for each filter to make sure the filter info was logged. To be more detailed, the output log are as following (only 'filter' filed is put here for ease of reading): "filter":"TimestampsFilter (3/3): [2, 3, 5]" "filter":"TimestampsFilter (5/6): [2, 3, 5, 7, 11]" "filter":"ColumnPrefixFilter col2" "filter":"ColumnRangeFilter [col2a, col2b]" "filter":"ColumnCountGetFilter 8" "filter":"ColumnPaginationFilter (4, 4)" "filter":"InclusiveStopFilter row" "filter":"PrefixFilter row" "filter":"PageFilter 1" "filter":"SkipFilter TimestampsFilter (1/1): [1000]" "filter":"WhileMatchFilter TimestampsFilter (3/3): [2, 3, 5]" "filter":"KeyOnlyFilter" "filter":"FirstKeyOnlyFilter" "filter":"MultipleColumnPrefixFilter (3/3): [a, b, c]" "filter":"DependentColumnFilter (family, qualifier, true, LESS, value)" "filter":"FamilyFilter (LESS, value)" "filter":"QualifierFilter (LESS, value)" "filter":"RowFilter (LESS, value)" "filter":"ValueFilter (LESS, value)" "filter":"KeyOnlyFilter" "filter":"FirstKeyOnlyFilter" "filter":"SingleColumnValueFilter (family, qualifier, EQUAL, value)" "filter":"SingleColumnValueExcludeFilter (family, qualifier, EQUAL, value)" "filter":"FilterList AND (2/2): [KeyOnlyFilter, FirstKeyOnlyFilter]" Please check ~zhiqiu/Codes/scripts/testFilter.rb for the testing script. 3. Added unit test cases to TestOperation to verify the filters' toString() method works well. Revert Plan: Tags: Reviewers: Kannan, madhuvaidya, mbautin, JIRA CC: Kannan, madhuvaidya, mbautin, zhiqiu Differential Revision: https://reviews.facebook.net/D1539 --- .../java/org/apache/hadoop/hbase/client/Get.java | 3 + .../java/org/apache/hadoop/hbase/client/Scan.java | 9 +- .../hadoop/hbase/filter/ColumnCountGetFilter.java | 7 +- .../hbase/filter/ColumnPaginationFilter.java | 8 +- .../hadoop/hbase/filter/ColumnPrefixFilter.java | 5 + .../hadoop/hbase/filter/ColumnRangeFilter.java | 8 + .../apache/hadoop/hbase/filter/CompareFilter.java | 9 + .../hadoop/hbase/filter/DependentColumnFilter.java | 10 + .../org/apache/hadoop/hbase/filter/FilterBase.java | 7 + .../org/apache/hadoop/hbase/filter/FilterList.java | 19 ++- .../hadoop/hbase/filter/InclusiveStopFilter.java | 7 +- .../hbase/filter/MultipleColumnPrefixFilter.java | 25 ++ .../org/apache/hadoop/hbase/filter/PageFilter.java | 7 +- .../apache/hadoop/hbase/filter/PrefixFilter.java | 7 +- .../hbase/filter/SingleColumnValueFilter.java | 8 + .../org/apache/hadoop/hbase/filter/SkipFilter.java | 5 + .../hadoop/hbase/filter/TimestampsFilter.java | 25 ++ .../hadoop/hbase/filter/WhileMatchFilter.java | 5 + .../apache/hadoop/hbase/client/TestOperation.java | 237 ++++++++++++++++++++ 19 files changed, 402 insertions(+), 9 deletions(-) diff --git src/main/java/org/apache/hadoop/hbase/client/Get.java src/main/java/org/apache/hadoop/hbase/client/Get.java index 93c9e89..c27b647 100644 --- src/main/java/org/apache/hadoop/hbase/client/Get.java +++ src/main/java/org/apache/hadoop/hbase/client/Get.java @@ -368,6 +368,9 @@ public class Get extends OperationWithAttributes } } map.put("totalColumns", colCount); + if (this.filter != null) { + map.put("filter", this.filter.toString()); + } return map; } diff --git src/main/java/org/apache/hadoop/hbase/client/Scan.java src/main/java/org/apache/hadoop/hbase/client/Scan.java index cc4c6e8..7a53d44 100644 --- src/main/java/org/apache/hadoop/hbase/client/Scan.java +++ src/main/java/org/apache/hadoop/hbase/client/Scan.java @@ -304,11 +304,11 @@ public class Scan extends OperationWithAttributes implements Writable { * @param batch the maximum number of values */ public void setBatch(int batch) { - if(this.hasFilter() && this.filter.hasFilterRow()) { - throw new IncompatibleFilterException( + if (this.hasFilter() && this.filter.hasFilterRow()) { + throw new IncompatibleFilterException( "Cannot set batch on a scan using a filter" + " that returns true for filter.hasFilterRow"); - } + } this.batch = batch; } @@ -529,6 +529,9 @@ public class Scan extends OperationWithAttributes implements Writable { } } map.put("totalColumns", colCount); + if (this.filter != null) { + map.put("filter", this.filter.toString()); + } return map; } diff --git src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java index 26f11aa..c6946bb 100644 --- src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java @@ -88,4 +88,9 @@ public class ColumnCountGetFilter extends FilterBase { public void write(DataOutput out) throws IOException { out.writeInt(this.limit); } -} \ No newline at end of file + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + this.limit; + } +} diff --git src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java index 85b0af7..d6b1280 100644 --- src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java @@ -106,4 +106,10 @@ public class ColumnPaginationFilter extends FilterBase out.writeInt(this.limit); out.writeInt(this.offset); } -} \ No newline at end of file + + @Override + public String toString() { + return String.format("%s (%d, %d)", this.getClass().getSimpleName(), + this.limit, this.offset); + } +} diff --git src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java index 931c7ad..cb89b7d 100644 --- src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java @@ -101,4 +101,9 @@ public class ColumnPrefixFilter extends FilterBase { kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(), prefix, 0, prefix.length); } + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.prefix); + } } diff --git src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java index df8786c..bd5c573 100644 --- src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java @@ -202,4 +202,12 @@ public class ColumnRangeFilter extends FilterBase { .getFamilyLength(), this.minColumn, 0, this.minColumn == null ? 0 : this.minColumn.length); } + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + + (this.minColumnInclusive ? "[" : "(") + Bytes.toStringBinary(this.minColumn) + + ", " + Bytes.toStringBinary(this.maxColumn) + + (this.maxColumnInclusive ? "]" : ")"); + } } diff --git src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java index 3709060..9089f4b 100644 --- src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.filter; import org.apache.hadoop.hbase.io.HbaseObjectWritable; +import org.apache.hadoop.hbase.util.Bytes; import java.io.DataInput; import java.io.DataOutput; @@ -155,4 +156,12 @@ public abstract class CompareFilter extends FilterBase { HbaseObjectWritable.writeObject(out, comparator, WritableByteArrayComparable.class, null); } + + @Override + public String toString() { + return String.format("%s (%s, %s)", + this.getClass().getSimpleName(), + this.compareOp.name(), + Bytes.toStringBinary(this.comparator.getValue())); + } } diff --git src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java index 785bd9e..c50c821 100644 --- src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java @@ -237,4 +237,14 @@ public class DependentColumnFilter extends CompareFilter { out.writeBoolean(this.dropDependentColumn); } + @Override + public String toString() { + return String.format("%s (%s, %s, %s, %s, %s)", + this.getClass().getSimpleName(), + Bytes.toStringBinary(this.columnFamily), + Bytes.toStringBinary(this.columnQualifier), + this.dropDependentColumn, + this.compareOp.name(), + Bytes.toStringBinary(this.comparator.getValue())); + } } diff --git src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java index 0d1b123..48e26c5 100644 --- src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java +++ src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java @@ -139,4 +139,11 @@ public abstract class FilterBase implements Filter { public static Filter createFilterFromArguments(ArrayList filterArguments) { throw new IllegalArgumentException("This method has not been implemented"); } + + /** + * Return filter's info for debugging and logging purpose. + */ + public String toString() { + return this.getClass().getSimpleName(); + } } diff --git src/main/java/org/apache/hadoop/hbase/filter/FilterList.java src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 216d0db..fee1a41 100644 --- src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -51,6 +51,7 @@ public class FilterList implements Filter { } private static final Configuration conf = HBaseConfiguration.create(); + private static final int MAX_LOG_FILTERS = 5; private Operator operator = Operator.MUST_PASS_ALL; private List filters = new ArrayList(); @@ -310,4 +311,20 @@ public class FilterList implements Filter { } return keyHint; } -} \ No newline at end of file + + @Override + public String toString() { + return toString(MAX_LOG_FILTERS); + } + + protected String toString(int maxFilters) { + int endIndex = this.filters.size() < maxFilters + ? this.filters.size() : maxFilters; + return String.format("%s %s (%d/%d): %s", + this.getClass().getSimpleName(), + this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR", + endIndex, + this.filters.size(), + this.filters.subList(0, endIndex).toString()); + } +} diff --git src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java index 091800c..afa31c5 100644 --- src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java @@ -89,4 +89,9 @@ public class InclusiveStopFilter extends FilterBase { public void readFields(DataInput in) throws IOException { this.stopRowKey = Bytes.readByteArray(in); } -} \ No newline at end of file + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.stopRowKey); + } +} diff --git src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java index 233b295..11fe6e3 100644 --- src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java @@ -36,6 +36,7 @@ import java.util.ArrayList; public class MultipleColumnPrefixFilter extends FilterBase { protected byte [] hint = null; protected TreeSet sortedPrefixes = createTreeSet(); + private final static int MAX_LOG_PREFIXES = 5; public MultipleColumnPrefixFilter() { super(); @@ -136,4 +137,28 @@ public class MultipleColumnPrefixFilter extends FilterBase { } }); } + + @Override + public String toString() { + return toString(MAX_LOG_PREFIXES); + } + + protected String toString(int maxPrefixes) { + StringBuilder prefixes = new StringBuilder(); + + int count = 0; + for (byte[] ba : this.sortedPrefixes) { + if (count >= maxPrefixes) { + break; + } + ++count; + prefixes.append(Bytes.toStringBinary(ba)); + if (count < this.sortedPrefixes.size() && count < maxPrefixes) { + prefixes.append(", "); + } + } + + return String.format("%s (%d/%d): [%s]", this.getClass().getSimpleName(), + count, this.sortedPrefixes.size(), prefixes.toString()); + } } diff --git src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java index 9d66c75..b00205a 100644 --- src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java @@ -88,4 +88,9 @@ public class PageFilter extends FilterBase { public void write(final DataOutput out) throws IOException { out.writeLong(pageSize); } -} \ No newline at end of file + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + this.pageSize; + } +} diff --git src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java index 9de199b..e3c3d39 100644 --- src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java @@ -84,4 +84,9 @@ public class PrefixFilter extends FilterBase { public void readFields(DataInput in) throws IOException { this.prefix = Bytes.readByteArray(in); } -} \ No newline at end of file + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.prefix); + } +} diff --git src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java index 7e4b2ae..f6e0cb7 100644 --- src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java @@ -306,4 +306,12 @@ public class SingleColumnValueFilter extends FilterBase { out.writeBoolean(filterIfMissing); out.writeBoolean(latestVersionOnly); } + + @Override + public String toString() { + return String.format("%s (%s, %s, %s, %s)", + this.getClass().getSimpleName(), Bytes.toStringBinary(this.columnFamily), + Bytes.toStringBinary(this.columnQualifier), this.compareOp.name(), + Bytes.toStringBinary(this.comparator.getValue())); + } } diff --git src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java index 8be40ee..57fa991 100644 --- src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java @@ -103,4 +103,9 @@ public class SkipFilter extends FilterBase { throw new RuntimeException("Failed deserialize.", e); } } + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + this.filter.toString(); + } } diff --git src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java index 551cbab..5e780b5 100644 --- src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java @@ -40,6 +40,7 @@ import com.google.common.base.Preconditions; public class TimestampsFilter extends FilterBase { TreeSet timestamps; + private static final int MAX_LOG_TIMESTAMPS = 5; // Used during scans to hint the scan to stop early // once the timestamps fall below the minTimeStamp. @@ -129,4 +130,28 @@ public class TimestampsFilter extends FilterBase { out.writeLong(timestamp); } } + + @Override + public String toString() { + return toString(MAX_LOG_TIMESTAMPS); + } + + protected String toString(int maxTimestamps) { + StringBuilder tsList = new StringBuilder(); + + int count = 0; + for (Long ts : this.timestamps) { + if (count >= maxTimestamps) { + break; + } + ++count; + tsList.append(ts.toString()); + if (count < this.timestamps.size() && count < maxTimestamps) { + tsList.append(", "); + } + } + + return String.format("%s (%d/%d): [%s]", this.getClass().getSimpleName(), + count, this.timestamps.size(), tsList.toString()); + } } diff --git src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java index b9fa927..242e0bd 100644 --- src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java +++ src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java @@ -104,4 +104,9 @@ public class WhileMatchFilter extends FilterBase { throw new RuntimeException("Failed deserialize.", e); } } + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + this.filter.toString(); + } } diff --git src/test/java/org/apache/hadoop/hbase/client/TestOperation.java src/test/java/org/apache/hadoop/hbase/client/TestOperation.java index 716022d..43f856f 100644 --- src/test/java/org/apache/hadoop/hbase/client/TestOperation.java +++ src/test/java/org/apache/hadoop/hbase/client/TestOperation.java @@ -26,10 +26,36 @@ import org.apache.hadoop.hbase.SmallTests; import org.junit.Test; import java.io.IOException; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.ColumnCountGetFilter; +import org.apache.hadoop.hbase.filter.ColumnPaginationFilter; +import org.apache.hadoop.hbase.filter.ColumnPrefixFilter; +import org.apache.hadoop.hbase.filter.ColumnRangeFilter; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.DependentColumnFilter; +import org.apache.hadoop.hbase.filter.FamilyFilter; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; +import org.apache.hadoop.hbase.filter.InclusiveStopFilter; +import org.apache.hadoop.hbase.filter.KeyOnlyFilter; +import org.apache.hadoop.hbase.filter.MultipleColumnPrefixFilter; +import org.apache.hadoop.hbase.filter.PageFilter; +import org.apache.hadoop.hbase.filter.PrefixFilter; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.hbase.filter.RowFilter; +import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; +import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter; +import org.apache.hadoop.hbase.filter.SkipFilter; +import org.apache.hadoop.hbase.filter.TimestampsFilter; +import org.apache.hadoop.hbase.filter.ValueFilter; +import org.apache.hadoop.hbase.filter.WhileMatchFilter; import org.apache.hadoop.hbase.util.Bytes; import org.codehaus.jackson.map.ObjectMapper; @@ -48,6 +74,217 @@ public class TestOperation { private static ObjectMapper mapper = new ObjectMapper(); + private static List TS_LIST = Arrays.asList(2L, 3L, 5L); + private static TimestampsFilter TS_FILTER = new TimestampsFilter(TS_LIST); + private static String STR_TS_FILTER = + TS_FILTER.getClass().getSimpleName() + " (3/3): [2, 3, 5]"; + + private static List L_TS_LIST = + Arrays.asList(0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L); + private static TimestampsFilter L_TS_FILTER = + new TimestampsFilter(L_TS_LIST); + private static String STR_L_TS_FILTER = + L_TS_FILTER.getClass().getSimpleName() + " (5/11): [0, 1, 2, 3, 4]"; + + private static String COL_NAME_1 = "col1"; + private static ColumnPrefixFilter COL_PRE_FILTER = + new ColumnPrefixFilter(COL_NAME_1.getBytes()); + private static String STR_COL_PRE_FILTER = + COL_PRE_FILTER.getClass().getSimpleName() + " " + COL_NAME_1; + + private static String COL_NAME_2 = "col2"; + private static ColumnRangeFilter CR_FILTER = new ColumnRangeFilter( + COL_NAME_1.getBytes(), true, COL_NAME_2.getBytes(), false); + private static String STR_CR_FILTER = CR_FILTER.getClass().getSimpleName() + + " [" + COL_NAME_1 + ", " + COL_NAME_2 + ")"; + + private static int COL_COUNT = 9; + private static ColumnCountGetFilter CCG_FILTER = + new ColumnCountGetFilter(COL_COUNT); + private static String STR_CCG_FILTER = + CCG_FILTER.getClass().getSimpleName() + " " + COL_COUNT; + + private static int LIMIT = 3; + private static int OFFSET = 4; + private static ColumnPaginationFilter CP_FILTER = + new ColumnPaginationFilter(LIMIT, OFFSET); + private static String STR_CP_FILTER = CP_FILTER.getClass().getSimpleName() + + " (" + LIMIT + ", " + OFFSET + ")"; + + private static String STOP_ROW_KEY = "stop"; + private static InclusiveStopFilter IS_FILTER = + new InclusiveStopFilter(STOP_ROW_KEY.getBytes()); + private static String STR_IS_FILTER = + IS_FILTER.getClass().getSimpleName() + " " + STOP_ROW_KEY; + + private static String PREFIX = "prefix"; + private static PrefixFilter PREFIX_FILTER = + new PrefixFilter(PREFIX.getBytes()); + private static String STR_PREFIX_FILTER = "PrefixFilter " + PREFIX; + + private static byte[][] PREFIXES = { + "0".getBytes(), "1".getBytes(), "2".getBytes()}; + private static MultipleColumnPrefixFilter MCP_FILTER = + new MultipleColumnPrefixFilter(PREFIXES); + private static String STR_MCP_FILTER = + MCP_FILTER.getClass().getSimpleName() + " (3/3): [0, 1, 2]"; + + private static byte[][] L_PREFIXES = { + "0".getBytes(), "1".getBytes(), "2".getBytes(), "3".getBytes(), + "4".getBytes(), "5".getBytes(), "6".getBytes(), "7".getBytes()}; + private static MultipleColumnPrefixFilter L_MCP_FILTER = + new MultipleColumnPrefixFilter(L_PREFIXES); + private static String STR_L_MCP_FILTER = + L_MCP_FILTER.getClass().getSimpleName() + " (5/8): [0, 1, 2, 3, 4]"; + + private static int PAGE_SIZE = 9; + private static PageFilter PAGE_FILTER = new PageFilter(PAGE_SIZE); + private static String STR_PAGE_FILTER = + PAGE_FILTER.getClass().getSimpleName() + " " + PAGE_SIZE; + + private static SkipFilter SKIP_FILTER = new SkipFilter(L_TS_FILTER); + private static String STR_SKIP_FILTER = + SKIP_FILTER.getClass().getSimpleName() + " " + STR_L_TS_FILTER; + + private static WhileMatchFilter WHILE_FILTER = + new WhileMatchFilter(L_TS_FILTER); + private static String STR_WHILE_FILTER = + WHILE_FILTER.getClass().getSimpleName() + " " + STR_L_TS_FILTER; + + private static KeyOnlyFilter KEY_ONLY_FILTER = new KeyOnlyFilter(); + private static String STR_KEY_ONLY_FILTER = + KEY_ONLY_FILTER.getClass().getSimpleName(); + + private static FirstKeyOnlyFilter FIRST_KEY_ONLY_FILTER = + new FirstKeyOnlyFilter(); + private static String STR_FIRST_KEY_ONLY_FILTER = + FIRST_KEY_ONLY_FILTER.getClass().getSimpleName(); + + private static CompareOp CMP_OP = CompareOp.EQUAL; + private static byte[] CMP_VALUE = "value".getBytes(); + private static BinaryComparator BC = new BinaryComparator(CMP_VALUE); + private static DependentColumnFilter DC_FILTER = + new DependentColumnFilter(FAMILY, QUALIFIER, true, CMP_OP, BC); + private static String STR_DC_FILTER = String.format( + "%s (%s, %s, %s, %s, %s)", DC_FILTER.getClass().getSimpleName(), + Bytes.toStringBinary(FAMILY), Bytes.toStringBinary(QUALIFIER), true, + CMP_OP.name(), Bytes.toStringBinary(BC.getValue())); + + private static FamilyFilter FAMILY_FILTER = new FamilyFilter(CMP_OP, BC); + private static String STR_FAMILY_FILTER = + FAMILY_FILTER.getClass().getSimpleName() + " (EQUAL, value)"; + + private static QualifierFilter QUALIFIER_FILTER = + new QualifierFilter(CMP_OP, BC); + private static String STR_QUALIFIER_FILTER = + QUALIFIER_FILTER.getClass().getSimpleName() + " (EQUAL, value)"; + + private static RowFilter ROW_FILTER = new RowFilter(CMP_OP, BC); + private static String STR_ROW_FILTER = + ROW_FILTER.getClass().getSimpleName() + " (EQUAL, value)"; + + private static ValueFilter VALUE_FILTER = new ValueFilter(CMP_OP, BC); + private static String STR_VALUE_FILTER = + VALUE_FILTER.getClass().getSimpleName() + " (EQUAL, value)"; + + private static SingleColumnValueFilter SCV_FILTER = + new SingleColumnValueFilter(FAMILY, QUALIFIER, CMP_OP, CMP_VALUE); + private static String STR_SCV_FILTER = String.format("%s (%s, %s, %s, %s)", + SCV_FILTER.getClass().getSimpleName(), Bytes.toStringBinary(FAMILY), + Bytes.toStringBinary(QUALIFIER), CMP_OP.name(), + Bytes.toStringBinary(CMP_VALUE)); + + private static SingleColumnValueExcludeFilter SCVE_FILTER = + new SingleColumnValueExcludeFilter(FAMILY, QUALIFIER, CMP_OP, CMP_VALUE); + private static String STR_SCVE_FILTER = String.format("%s (%s, %s, %s, %s)", + SCVE_FILTER.getClass().getSimpleName(), Bytes.toStringBinary(FAMILY), + Bytes.toStringBinary(QUALIFIER), CMP_OP.name(), + Bytes.toStringBinary(CMP_VALUE)); + + private static FilterList AND_FILTER_LIST = new FilterList( + Operator.MUST_PASS_ALL, Arrays.asList((Filter) TS_FILTER, L_TS_FILTER, + CR_FILTER)); + private static String STR_AND_FILTER_LIST = String.format( + "%s AND (3/3): [%s, %s, %s]", AND_FILTER_LIST.getClass().getSimpleName(), + STR_TS_FILTER, STR_L_TS_FILTER, STR_CR_FILTER); + + private static FilterList OR_FILTER_LIST = new FilterList( + Operator.MUST_PASS_ONE, Arrays.asList((Filter) TS_FILTER, L_TS_FILTER, + CR_FILTER)); + private static String STR_OR_FILTER_LIST = String.format( + "%s OR (3/3): [%s, %s, %s]", AND_FILTER_LIST.getClass().getSimpleName(), + STR_TS_FILTER, STR_L_TS_FILTER, STR_CR_FILTER); + + private static FilterList L_FILTER_LIST = new FilterList( + Arrays.asList((Filter) TS_FILTER, L_TS_FILTER, CR_FILTER, COL_PRE_FILTER, + CCG_FILTER, CP_FILTER, PREFIX_FILTER, PAGE_FILTER)); + private static String STR_L_FILTER_LIST = String.format( + "%s AND (5/8): [%s, %s, %s, %s, %s]", + L_FILTER_LIST.getClass().getSimpleName(), STR_TS_FILTER, STR_L_TS_FILTER, + STR_CR_FILTER, STR_COL_PRE_FILTER, STR_CCG_FILTER, STR_CP_FILTER); + + private static Filter[] FILTERS = { + TS_FILTER, // TimestampsFilter + L_TS_FILTER, // TimestampsFilter + COL_PRE_FILTER, // ColumnPrefixFilter + CP_FILTER, // ColumnPaginationFilter + CR_FILTER, // ColumnRangeFilter + CCG_FILTER, // ColumnCountGetFilter + IS_FILTER, // InclusiveStopFilter + PREFIX_FILTER, // PrefixFilter + PAGE_FILTER, // PageFilter + SKIP_FILTER, // SkipFilter + WHILE_FILTER, // WhileMatchFilter + KEY_ONLY_FILTER, // KeyOnlyFilter + FIRST_KEY_ONLY_FILTER, // FirstKeyOnlyFilter + MCP_FILTER, // MultipleColumnPrefixFilter + L_MCP_FILTER, // MultipleColumnPrefixFilter + DC_FILTER, // DependentColumnFilter + FAMILY_FILTER, // FamilyFilter + QUALIFIER_FILTER, // QualifierFilter + ROW_FILTER, // RowFilter + VALUE_FILTER, // ValueFilter + SCV_FILTER, // SingleColumnValueFilter + SCVE_FILTER, // SingleColumnValueExcludeFilter + AND_FILTER_LIST, // FilterList + OR_FILTER_LIST, // FilterList + L_FILTER_LIST, // FilterList + }; + + private static String[] FILTERS_INFO = { + STR_TS_FILTER, // TimestampsFilter + STR_L_TS_FILTER, // TimestampsFilter + STR_COL_PRE_FILTER, // ColumnPrefixFilter + STR_CP_FILTER, // ColumnPaginationFilter + STR_CR_FILTER, // ColumnRangeFilter + STR_CCG_FILTER, // ColumnCountGetFilter + STR_IS_FILTER, // InclusiveStopFilter + STR_PREFIX_FILTER, // PrefixFilter + STR_PAGE_FILTER, // PageFilter + STR_SKIP_FILTER, // SkipFilter + STR_WHILE_FILTER, // WhileMatchFilter + STR_KEY_ONLY_FILTER, // KeyOnlyFilter + STR_FIRST_KEY_ONLY_FILTER, // FirstKeyOnlyFilter + STR_MCP_FILTER, // MultipleColumnPrefixFilter + STR_L_MCP_FILTER, // MultipleColumnPrefixFilter + STR_DC_FILTER, // DependentColumnFilter + STR_FAMILY_FILTER, // FamilyFilter + STR_QUALIFIER_FILTER, // QualifierFilter + STR_ROW_FILTER, // RowFilter + STR_VALUE_FILTER, // ValueFilter + STR_SCV_FILTER, // SingleColumnValueFilter + STR_SCVE_FILTER, // SingleColumnValueExcludeFilter + STR_AND_FILTER_LIST, // FilterList + STR_OR_FILTER_LIST, // FilterList + STR_L_FILTER_LIST, // FilterList + }; + + static { + assertEquals("The sizes of static arrays do not match: " + + "[FILTERS: %d <=> FILTERS_INFO: %d]", + FILTERS.length, FILTERS_INFO.length); + } + /** * Test the client Operations' JSON encoding to ensure that produced JSON is * parseable and that the details are present and not corrupted. -- 1.7.8.4