[
https://issues.apache.org/jira/browse/HBASE-4542?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Phabricator updated HBASE-4542:
-------------------------------
Attachment: D1263.2.patch
zhiqiu requested code review of "[jira] [HBase-4542] [89-fb] Add filter info to
slow query logging".
Reviewers: Kannan, madhuvaidya, mbautin, JIRA
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.
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:
REVISION DETAIL
https://reviews.facebook.net/D1263
AFFECTED FILES
src/main/java/org/apache/hadoop/hbase/client/Get.java
src/main/java/org/apache/hadoop/hbase/client/Scan.java
src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
src/test/java/org/apache/hadoop/hbase/client/TestOperation.java
> add filter info to slow query logging
> -------------------------------------
>
> Key: HBASE-4542
> URL: https://issues.apache.org/jira/browse/HBASE-4542
> Project: HBase
> Issue Type: Improvement
> Reporter: Kannan Muthukkaruppan
> Assignee: Madhuwanti Vaidya
> Attachments: D1263.2.patch
>
>
> Slow query log doesn't report filters in effect.
> For example:
> {code}
> (operationTooSlow): \
> {"processingtimems":3468,"client":"10.138.43.206:40035","timeRange":
> [0,9223372036854775807],\
> "starttimems":1317772005821,"responsesize":42411, \
> "class":"HRegionServer","table":"myTable","families":{"CF1":"ALL"]},\
> "row":"6c3b8efa132f0219b7621ed1e5c8c70b","queuetimems":0,\
> "method":"get","totalColumns":1,"maxVersions":1,"storeLimit":-1}
> {code}
> the above would suggest that all columns of myTable:CF1 are being requested
> for the given row. But in reality there could be filters in effect (such as
> ColumnPrefixFilter, ColumnRangeFilter, TimestampsFilter() etc.). We should
> enhance the slow query log to capture & report this information.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators:
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira