You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@impala.apache.org by "Quanlong Huang (Code Review)" <ge...@cloudera.org> on 2022/01/11 10:03:29 UTC

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Quanlong Huang has uploaded this change for review. ( http://gerrit.cloudera.org:8080/18141


Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't leverage the bloom filters in
ORC files. Because only EQUALS and IN-list predicates can leverage them
to skip unrelated ORC RowGroups, and we can't convert runtime bloom
filters or min-max filters into such predicates.

This patch adds runtime IN-list filters for small build side (e.g. #rows
<= 1024) of a broadcast join. Currently the IN-list filters will only
apply to ORC tables and be pushed down to the ORC reader(i.e. ORC lib).
Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set). So we disable it at row level.

Example query that will benefit from this patch:
  use tpch_orc_def;
  select count(*) from lineitem_bf join (
    select * from partsupp, part
    where ps_partkey = p_partkey and p_size = 15
      and p_type like '%BRASS' and ps_availqty < 10) v
  on l_partkey = ps_partkey and l_suppkey = ps_suppkey;

The inline-view populates a runtime IN-list filter of 4 items. Note that
we need to re-generate the lineitem table with bloom filters enabled
(e.g. setting orc.bloom.filter.columns to
"l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity" in
tblproperties), so the pushed down IN-list filter can have a better
filter rate.

TODO: fix tests due to plan changes.

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/util/CMakeLists.txt
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/PlanNodes.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M tests/query_test/test_runtime_filters.py
30 files changed, 750 insertions(+), 120 deletions(-)



  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/1
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newchange
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 1
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 4:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10094/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 4
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Comment-Date: Fri, 04 Feb 2022 00:02:42 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 20:

(11 comments)

http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py
File tests/catalog_service/test_partition_catalog_updates.py:

http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py@25
PS20, Line 25: from CatalogService import CatalogService
flake8: F401 'CatalogService.CatalogService' imported but unused


http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py@26
PS20, Line 26: from CatalogService.CatalogService import TGetFunctionsRequest
flake8: F401 'CatalogService.CatalogService.TGetFunctionsRequest' imported but unused


http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py@27
PS20, Line 27: from ErrorCodes.ttypes import TErrorCode
flake8: F401 'ErrorCodes.ttypes.TErrorCode' imported but unused


http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py@28
PS20, Line 28: from thrift.protocol import TBinaryProtocol
flake8: F401 'thrift.protocol.TBinaryProtocol' imported but unused


http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py@31
PS20, Line 31: from tests.common.impala_cluster import ImpalaCluster
flake8: F401 'tests.common.impala_cluster.ImpalaCluster' imported but unused


http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py@32
PS20, Line 32: from tests.common.impala_test_suite import ImpalaTestSuite
flake8: F401 'tests.common.impala_test_suite.ImpalaTestSuite' imported but unused


http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py@33
PS20, Line 33: from tests.common.skip import SkipIfDockerizedCluster
flake8: F401 'tests.common.skip.SkipIfDockerizedCluster' imported but unused


http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py@34
PS20, Line 34: from tests.common.test_dimensions import create_single_exec_option_dimension
flake8: F401 'tests.common.test_dimensions.create_single_exec_option_dimension' imported but unused


http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py@35
PS20, Line 35: from tests.util.filesystem_utils import WAREHOUSE
flake8: F401 'tests.util.filesystem_utils.WAREHOUSE' imported but unused


http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py@36
PS20, Line 36: from tests.util.thrift_util import create_transport
flake8: F401 'tests.util.thrift_util.create_transport' imported but unused


http://gerrit.cloudera.org:8080/#/c/18141/20/tests/catalog_service/test_partition_catalog_updates.py@88
PS20, Line 88: 
flake8: W391 blank line at end of file



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 20
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sun, 27 Feb 2022 00:23:02 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 18:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10226/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 18
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Fri, 25 Feb 2022 01:00:17 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Qifan Chen (Code Review)" <ge...@cloudera.org>.
Qifan Chen has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 23: Code-Review+2


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 23
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Wed, 02 Mar 2022 19:39:03 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 2:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10085/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 2
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Comment-Date: Wed, 02 Feb 2022 09:16:06 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#18).

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated for the build side of a broadcast join. They will only be
applied on ORC tables and be pushed down to the ORC reader(i.e. ORC
lib). To avoid exploding the IN-list, if # of distinct values of the
build side exceeds a threshold (default to 1024), we set the filter to
ALWAYS_TRUE and clear its entry. The threshold can be configured by a
new query option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set) and the lack of codegen. So we disable it at row
level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-11140: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-11141: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-11142: Consider IN-list filters in partitioned joins.

Tests:
 - Test IN-list filter on string, date and all integer types
 - Test IN-list filter with NULL
 - Test IN-list filter on complex exprs targets

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
A testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
M tests/query_test/test_runtime_filters.py
44 files changed, 1,428 insertions(+), 214 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/18
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 18
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 16:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10219/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 16
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Thu, 24 Feb 2022 03:25:30 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#16).

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated for the build side of a broadcast join. They will only be
applied on ORC tables and be pushed down to the ORC reader(i.e. ORC
lib). To avoid exploding the IN-list, if # of distinct values of the
build side exceeds a threshold (default to 1024), we set the filter to
ALWAYS_TRUE and clear its entry. The threshold can be configured by a
new query option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set) and the lack of codegen. So we disable it at row
level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-11140: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-11141: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-11142: Consider IN-list filters in partitioned joins.

Tests:
 - Test IN-list filter on string, date and all integer types
 - Test IN-list filter with NULL
 - Test IN-list filter on complex exprs targets

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
A testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
M tests/query_test/test_runtime_filters.py
44 files changed, 1,427 insertions(+), 214 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/16
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 16
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 11:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10189/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 11
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Mon, 21 Feb 2022 03:16:06 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Qifan Chen (Code Review)" <ge...@cloudera.org>.
Qifan Chen has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 14:

(10 comments)

Great. Thanks a lot for the rework.

http://gerrit.cloudera.org:8080/#/c/18141/14//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/18141/14//COMMIT_MSG@20
PS14, Line 20: Currently they
             : are generated only for small build side (e.g. #rows <= 1024) of a
             : broadcast join.
nit. They are generated for the build side of a broadcast join.

Suggest not to mention small build side until a few sentences later.


http://gerrit.cloudera.org:8080/#/c/18141/14//COMMIT_MSG@24
PS14, Line 24: #rows of the build side exceeds the threshold (1024)
nit. # of distinct values of the build side exceeds a threshold (default to 1024).


http://gerrit.cloudera.org:8080/#/c/18141/14//COMMIT_MSG@39
PS14, Line 39: IN-list filters are disabled by default
It seems a formal performance test against TPCDs can help decide the default setting. Maybe we do this once Impala 11140, 11141 and 11142 are resolved.


http://gerrit.cloudera.org:8080/#/c/18141/14/be/src/runtime/runtime-filter-bank.cc
File be/src/runtime/runtime-filter-bank.cc:

http://gerrit.cloudera.org:8080/#/c/18141/14/be/src/runtime/runtime-filter-bank.cc@380
PS14, Line 380:    uint32_t entry_limit = InListFilter::DEFAULT_ENTRY_LIMIT;
              :     if (query_state_->query_options().__isset.runtime_in_list_filter_entry_limit) {
              :       entry_limit = query_state_->query_options().runtime_in_list_filter_entry_limit;
              :     }
              :     in_list_filter = InListFilter::Create(params.in_list_filter(),
              :         fs->consumed_filter->type(), entry_limit, &obj_pool_);
              :     fs->in_list_filters.push_back(in_list_filter);
Duplicated code with a portion of implementation of RuntimeFilterBank::AllocateScratchInListFilte().


http://gerrit.cloudera.org:8080/#/c/18141/14/be/src/runtime/runtime-filter-bank.cc@444
PS14, Line 444: :DEFAULT_ENTRY_LIMIT;
I wonder if we need this, since runtime_in_list_filter_entry_limit is default to 1024 and covers the non-negative domain.


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc
File be/src/util/in-list-filter-ir.cc:

http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc@55
PS13, Line 55: if (UNLIKELY(s->ptr == nullptr)) {
             :         contains_null_ = true
> The default constructor of StringValue creates a null 'ptr'. I think we'd b
Done


http://gerrit.cloudera.org:8080/#/c/18141/14/be/src/util/in-list-filter-ir.cc
File be/src/util/in-list-filter-ir.cc:

http://gerrit.cloudera.org:8080/#/c/18141/14/be/src/util/in-list-filter-ir.cc@61
PS14, Line 61: return
should set always_true_ to true here.


http://gerrit.cloudera.org:8080/#/c/18141/14/common/thrift/ImpalaService.thrift
File common/thrift/ImpalaService.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/14/common/thrift/ImpalaService.thrift@725
PS14, Line 725:   RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT = 142;
nit. missing comment.


http://gerrit.cloudera.org:8080/#/c/18141/14/common/thrift/Query.thrift
File common/thrift/Query.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/14/common/thrift/Query.thrift@578
PS14, Line 578:   143: optional i32 runtime_in_list_filter_entry_limit = 1024;
nit. missing comment.


http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
File testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test:

http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test@127
PS13, Line 127: 
> Sure. The ORC date_tbl is corrupted and need to wait for https://gerrit.clo
Okay.



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 14
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Wed, 23 Feb 2022 16:10:22 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 23:

Build started: https://jenkins.impala.io/job/gerrit-verify-dryrun/7894/ DRY_RUN=false


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 23
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Wed, 02 Mar 2022 19:39:38 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 7:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/18141/7/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/7/be/src/exec/hdfs-orc-scanner.cc@160
PS7, Line 160:       ADD_COUNTER(scan_node_->runtime_profile(), "NumPushedDownRuntimeFilters", TUnit::UNIT);
line too long (93 > 90)


http://gerrit.cloudera.org:8080/#/c/18141/7/tests/query_test/test_runtime_filters.py
File tests/query_test/test_runtime_filters.py:

http://gerrit.cloudera.org:8080/#/c/18141/7/tests/query_test/test_runtime_filters.py@70
PS7, Line 70: [
flake8: E131 continuation line unaligned for hanging indent



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 7
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Mon, 14 Feb 2022 23:18:00 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#6).

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated only for small build side (e.g. #rows <= 1024) of a
broadcast join. They will only be applied on ORC tables and be pushed
down to the ORC reader(i.e. ORC lib). To avoid exploding the IN-list,
if #rows of the build side exceeds the threshold (1024), we set the
filter to ALWAYS_TRUE. The threshold can be configured by a new query
option, runtime_in_list_filter_entry_limit.

Example query that will benefit from this patch:
  use tpch_orc_def;
  select count(*) from lineitem_bf join (
    select * from partsupp, part
    where ps_partkey = p_partkey and p_size = 15
      and p_type like '%BRASS' and ps_availqty < 10) v
  on l_partkey = ps_partkey and l_suppkey = ps_suppkey;

The inline-view populates a runtime IN-list filter with 4 items. Note that
we need to re-generate the lineitem table with bloom filter indexes enabled
(e.g. setting orc.bloom.filter.columns to
"l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity" in
tblproperties before inserting the data), so the runtime IN-list filter
can have a better filter rate.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set). So we disable it at row level.

TODO: fix tests due to plan changes.

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M tests/query_test/test_runtime_filters.py
36 files changed, 857 insertions(+), 160 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/6
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 6
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Quanlong Huang has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 5:

Thanks for your feedback, Qifan! Addressed the comments. I'm still updating/adding tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 5
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 08 Feb 2022 02:30:42 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 10:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/18141/10/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/10/be/src/exec/hdfs-orc-scanner.cc@318
PS10, Line 318:       ADD_COUNTER(scan_node_->runtime_profile(), "NumPushedDownRuntimeFilters", TUnit::UNIT);
line too long (93 > 90)


http://gerrit.cloudera.org:8080/#/c/18141/10/tests/query_test/test_runtime_filters.py
File tests/query_test/test_runtime_filters.py:

http://gerrit.cloudera.org:8080/#/c/18141/10/tests/query_test/test_runtime_filters.py@70
PS10, Line 70: [
flake8: E131 continuation line unaligned for hanging indent



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 10
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Fri, 18 Feb 2022 11:53:21 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Quanlong Huang has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 18:

(10 comments)

Thanks! Addressed the comments.

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.h
File be/src/exec/hdfs-orc-scanner.h:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.h@427
PS17, Line 427: Decides
> nit. Decides
Done


http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.h@433
PS17, Line 433: Pushab
> nit. IsPushableInListFilter.
Done


http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.cc@1486
PS17, Line 1486: if (in_list_filter->ContainsNull()) {
               :       // Add a null literal with type.
               :      
> I wonder how the ORC layer recognizes the need to filter null.  The logic h
It's not a column type. We are using this constructor to create a null literal with type:
https://github.com/apache/orc/blob/rel/release-1.7.0/c++/include/orc/sargs/Literal.hh#L72-L75

ORC has stats about nulls. They will be used in predicate pushdown. We have some tests verifying these:
https://github.com/apache/impala/blob/873fe2e5241c5714dfd94a186d524edc1cbad0ad/testdata/workloads/functional-query/queries/QueryTest/orc-stats.test#L787-L863


http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/service/query-options.cc
File be/src/service/query-options.cc:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/service/query-options.cc@974
PS17, Line 974: // Parse and verify the enabled runtime filter types.
> nit. Parse and verify the enabled runtime filter types.
Done


http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/util/in-list-filter-ir.cc
File be/src/util/in-list-filter-ir.cc:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/util/in-list-filter-ir.cc@18
PS17, Line 18: #include "common/object-pool.h"
             : #include "util/in-list-filter.h"
             : 
> nit. May arrange them in ascending order.
Done


http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/util/in-list-filter.cc
File be/src/util/in-list-filter.cc:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/util/in-list-filter.cc@38
PS17, Line 38: switch (col_type.type) {
             :     case TYPE_TINYINT:
             :       v = *reinterpret_cast<const int8_t*>(val);
             :       break;
             :     case TYPE_SMALLINT:
             :       v = *reinterpret_cast<const int16_t*>(val);
             :       break;
             :     case TYPE_INT:
             :       v = *reinterpret_cast<const int32_t*>(val);
             :       break;
             :     case TYPE_BIGINT:
             :       v = *reinterpret_cast<const int64_t*>(val);
             :       break;
             :     case TYPE_DATE:
             :       v = reinterpret_cast<const DateValue*>(val)->Value();
             :       break;
             :     case TYPE_STRING:
             :     case TYPE_VARCHAR:
             :       s = reinterpret_cast<const StringValue*>(val);
             :       return str_values_.find(string(s->ptr, s->len)) != str_values_.end();
             :     case TYPE_CHAR:
             :       return str_values_.find(string(reinterpret_cast<const char*>(val), col_type.len))
             :           != str_values_.end();
             :     default:
             :       DCHECK(false) << "Not support IN-list filter type: " << TypeToString(type_);
             :       return false;
             :   }
> Future improvement. 
Yeah, planned to do this in IMPALA-11141.


http://gerrit.cloudera.org:8080/#/c/18141/17/common/thrift/ImpalaService.thrift
File common/thrift/ImpalaService.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/17/common/thrift/ImpalaService.thrift@725
PS17, Line 725:   // Maximum number of distinct entries in a runtime in-list filter.
> nit. distinct
Done


http://gerrit.cloudera.org:8080/#/c/18141/17/common/thrift/PlanNodes.thrift
File common/thrift/PlanNodes.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/17/common/thrift/PlanNodes.thrift@a134
PS17, Line 134: 
> nit. dropped.
These are removed. Do you mean we should keep them?


http://gerrit.cloudera.org:8080/#/c/18141/17/testdata/datasets/functional/functional_schema_template.sql
File testdata/datasets/functional/functional_schema_template.sql:

http://gerrit.cloudera.org:8080/#/c/18141/17/testdata/datasets/functional/functional_schema_template.sql@274
PS17, Line 274: neg
> nit. may spell out the entire word: alltypestiny_negative
Done


http://gerrit.cloudera.org:8080/#/c/18141/17/testdata/datasets/functional/schema_constraints.csv
File testdata/datasets/functional/schema_constraints.csv:

http://gerrit.cloudera.org:8080/#/c/18141/17/testdata/datasets/functional/schema_constraints.csv@323
PS17, Line 323: alltypestiny_neg
> nit. alltypestiny_negative
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 18
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Fri, 25 Feb 2022 00:33:53 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Quanlong Huang has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 21:

I accidentally uploaded some unrelated files in Patch Set 20, which caused some other failures.
Patch Set 21 fixes the flaky test failure. Please review the difference between Patch Set 19 and 21. Thanks!


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 21
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sun, 27 Feb 2022 22:47:51 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 22:

Build started: https://jenkins.impala.io/job/gerrit-verify-dryrun/7890/ DRY_RUN=false


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 22
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 01 Mar 2022 13:49:57 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Quanlong Huang has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 16:

(9 comments)

Thanks for your feedbacks, Qifan!

Addressed the comments and added tests for
* empty string
* large string that exceeds the limit
* DATE type

Also added some logs that are useful for debugging.

http://gerrit.cloudera.org:8080/#/c/18141/14//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/18141/14//COMMIT_MSG@20
PS14, Line 20: Currently they
             : are generated for the build side of a broadcast join. They will only be
             : applied on ORC 
> nit. They are generated for the build side of a broadcast join.
Done


http://gerrit.cloudera.org:8080/#/c/18141/14//COMMIT_MSG@24
PS14, Line 24: ld side exceeds a threshold (default to 1024), we se
> nit. # of distinct values of the build side exceeds a threshold (default to
Done


http://gerrit.cloudera.org:8080/#/c/18141/14//COMMIT_MSG@39
PS14, Line 39: IN-list filters are disabled by default
> It seems a formal performance test against TPCDs can help decide the defaul
Yeah, hopefully we can turn it on if all the regressions are resolved.


http://gerrit.cloudera.org:8080/#/c/18141/14/be/src/runtime/runtime-filter-bank.cc
File be/src/runtime/runtime-filter-bank.cc:

http://gerrit.cloudera.org:8080/#/c/18141/14/be/src/runtime/runtime-filter-bank.cc@380
PS14, Line 380:    DCHECK(query_state_->query_options().__isset.runtime_in_list_filter_entry_limit);
              :     int entry_limit = query_state_->query_options().runtime_in_list_filter_entry_limit;
              :     in_list_filter = InListFilter::Create(params.in_list_filter(),
              :         fs->consumed_filter->type(), entry_limit, &obj_pool_);
              :     fs->in_list_filters.push_back(in_list_filter);
              :     total_in_list_filter_items_->Add(params.in_list_filter().value_size());
              :     details = Substitute(" with $0 items", params.
> Duplicated code with a portion of implementation of RuntimeFilterBank::Allo
The codes are much simpler now.


http://gerrit.cloudera.org:8080/#/c/18141/14/be/src/runtime/runtime-filter-bank.cc@444
PS14, Line 444: 
> I wonder if we need this, since runtime_in_list_filter_entry_limit is defau
Yeah, I think thrift will make sure the default value is set. Removed this.


http://gerrit.cloudera.org:8080/#/c/18141/14/be/src/util/in-list-filter-ir.cc
File be/src/util/in-list-filter-ir.cc:

http://gerrit.cloudera.org:8080/#/c/18141/14/be/src/util/in-list-filter-ir.cc@61
PS14, Line 61: str_va
> should set always_true_ to true here.
Oops, thanks for catching this!


http://gerrit.cloudera.org:8080/#/c/18141/14/common/thrift/ImpalaService.thrift
File common/thrift/ImpalaService.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/14/common/thrift/ImpalaService.thrift@725
PS14, Line 725:   // Maximum number of entries in a runtime in-list filter.
> nit. missing comment.
Done


http://gerrit.cloudera.org:8080/#/c/18141/14/common/thrift/Query.thrift
File common/thrift/Query.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/14/common/thrift/Query.thrift@578
PS14, Line 578:   // See comment in ImpalaService.thrift
> nit. missing comment.
Done


http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
File testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test:

http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test@127
PS13, Line 127: select STRAIGHT_JOIN count(*) from date_tbl a
> Okay.
Added the DATE tests.



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 16
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Thu, 24 Feb 2022 03:04:19 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#2).

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't leverage the bloom filters in
ORC files. Because only EQUALS and IN-list predicates can leverage them
to skip unrelated ORC RowGroups, and we can't convert runtime bloom
filters or min-max filters into such predicates.

This patch adds runtime IN-list filters for small build side (e.g. #rows
<= 1024) of a broadcast join. Currently the IN-list filters will only
apply to ORC tables and be pushed down to the ORC reader(i.e. ORC lib).
Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set). So we disable it at row level.

Example query that will benefit from this patch:
  use tpch_orc_def;
  select count(*) from lineitem_bf join (
    select * from partsupp, part
    where ps_partkey = p_partkey and p_size = 15
      and p_type like '%BRASS' and ps_availqty < 10) v
  on l_partkey = ps_partkey and l_suppkey = ps_suppkey;

The inline-view populates a runtime IN-list filter of 4 items. Note that
we need to re-generate the lineitem table with bloom filters enabled
(e.g. setting orc.bloom.filter.columns to
"l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity" in
tblproperties), so the pushed down IN-list filter can have a better
filter rate.

TODO: fix tests due to plan changes.

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/util/CMakeLists.txt
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/PlanNodes.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M tests/query_test/test_runtime_filters.py
30 files changed, 752 insertions(+), 120 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/2
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 2
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 5:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10113/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 5
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 08 Feb 2022 02:53:51 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 1:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10000/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 1
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Comment-Date: Tue, 11 Jan 2022 10:25:42 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 7:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10155/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 7
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Mon, 14 Feb 2022 23:42:00 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Qifan Chen (Code Review)" <ge...@cloudera.org>.
Qifan Chen has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 17:

(10 comments)

Looks great!

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.h
File be/src/exec/hdfs-orc-scanner.h:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.h@427
PS17, Line 427: Returns
nit. Decides


http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.h@433
PS17, Line 433: Usable
nit. IsPushableInListFilter.


http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.cc@1486
PS17, Line 1486: if (in_list_filter->ContainsNull()) {
               :       in_list.emplace_back(GetOrcPredicateDataType(col_type));
               :     }
I wonder how the ORC layer recognizes the need to filter null.  The logic here implies we append the column type at the end of the value list.


http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/service/query-options.cc
File be/src/service/query-options.cc:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/service/query-options.cc@974
PS17, Line 974: // Parse the enabled runtime filter types and validate it.
nit. Parse and verify the enabled runtime filter types.


http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/util/in-list-filter-ir.cc
File be/src/util/in-list-filter-ir.cc:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/util/in-list-filter-ir.cc@18
PS17, Line 18: #include "util/in-list-filter.h"
             : 
             : #include "common/object-pool.h"
nit. May arrange them in ascending order.


http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/util/in-list-filter.cc
File be/src/util/in-list-filter.cc:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/util/in-list-filter.cc@38
PS17, Line 38: switch (col_type.type) {
             :     case TYPE_TINYINT:
             :       v = *reinterpret_cast<const int8_t*>(val);
             :       break;
             :     case TYPE_SMALLINT:
             :       v = *reinterpret_cast<const int16_t*>(val);
             :       break;
             :     case TYPE_INT:
             :       v = *reinterpret_cast<const int32_t*>(val);
             :       break;
             :     case TYPE_BIGINT:
             :       v = *reinterpret_cast<const int64_t*>(val);
             :       break;
             :     case TYPE_DATE:
             :       v = reinterpret_cast<const DateValue*>(val)->Value();
             :       break;
             :     case TYPE_STRING:
             :     case TYPE_VARCHAR:
             :       s = reinterpret_cast<const StringValue*>(val);
             :       return str_values_.find(string(s->ptr, s->len)) != str_values_.end();
             :     case TYPE_CHAR:
             :       return str_values_.find(string(reinterpret_cast<const char*>(val), col_type.len))
             :           != str_values_.end();
             :     default:
             :       DCHECK(false) << "Not support IN-list filter type: " << TypeToString(type_);
             :       return false;
             :   }
Future improvement. 

We may need to something similar to MinMaxFilter class hierarchy to push the type into a subclass of InListFilter. In this way, we do not incur the cost of type switching for each Find() call.

182 #define NUMERIC_MIN_MAX_FILTER(NAME, TYPE)                                          \      
183   class NAME##MinMaxFilter : public MinMaxFilter {                                  \      
184    public:                                                                          \      
185     NAME##MinMaxFilter() {                                                          \      
186       min_ = std::numeric_limits<TYPE>::max();                                      \      
187       max_ = std::numeric_limits<TYPE>::lowest();                                   \      
188     }                                                                               \      
189     NAME##MinMaxFilter(const MinMaxFilterPB& protobuf);                             \      
190     virtual ~NAME##MinMaxFilter() {}                                                \      
191     virtual const void* GetMin() const override { return &min_; }                   \      
192     virtual const void* GetMax() const override { return &max_; }                   \      
193     virtual bool GetCastIntMinMax(                                                  \      
194         const ColumnType& type, int64_t* out_min, int64_t* out_max) const override; \      
195     bool EvalOverlap(                                                               \      
196         const ColumnType& type, void* data_min, void* data_max) const override;     \      
197     float ComputeOverlapRatio(                                                      \ 
min-max-filter.h


http://gerrit.cloudera.org:8080/#/c/18141/17/common/thrift/ImpalaService.thrift
File common/thrift/ImpalaService.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/17/common/thrift/ImpalaService.thrift@725
PS17, Line 725:   // Maximum number of entries in a runtime in-list filter.
nit. distinct


http://gerrit.cloudera.org:8080/#/c/18141/17/common/thrift/PlanNodes.thrift
File common/thrift/PlanNodes.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/17/common/thrift/PlanNodes.thrift@a134
PS17, Line 134: 
nit. dropped.


http://gerrit.cloudera.org:8080/#/c/18141/17/testdata/datasets/functional/functional_schema_template.sql
File testdata/datasets/functional/functional_schema_template.sql:

http://gerrit.cloudera.org:8080/#/c/18141/17/testdata/datasets/functional/functional_schema_template.sql@274
PS17, Line 274: neg
nit. may spell out the entire word: alltypestiny_negative


http://gerrit.cloudera.org:8080/#/c/18141/17/testdata/datasets/functional/schema_constraints.csv
File testdata/datasets/functional/schema_constraints.csv:

http://gerrit.cloudera.org:8080/#/c/18141/17/testdata/datasets/functional/schema_constraints.csv@323
PS17, Line 323: alltypestiny_neg
nit. alltypestiny_negative



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 17
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Thu, 24 Feb 2022 15:25:13 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has submitted this change and it was merged. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated for the build side of a broadcast join. They will only be
applied on ORC tables and be pushed down to the ORC reader(i.e. ORC
lib). To avoid exploding the IN-list, if # of distinct values of the
build side exceeds a threshold (default to 1024), we set the filter to
ALWAYS_TRUE and clear its entry. The threshold can be configured by a
new query option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set) and the lack of codegen. So we disable it at row
level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-11140: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-11141: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-11142: Consider IN-list filters in partitioned joins.

Tests:
 - Test IN-list filter on string, date and all integer types
 - Test IN-list filter with NULL
 - Test IN-list filter on complex exprs targets

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Reviewed-on: http://gerrit.cloudera.org:8080/18141
Reviewed-by: Qifan Chen <qc...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
A testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
M tests/query_test/test_runtime_filters.py
44 files changed, 1,430 insertions(+), 215 deletions(-)

Approvals:
  Qifan Chen: Looks good to me, approved
  Impala Public Jenkins: Verified

-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 24
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 22: Code-Review+2


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 22
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 01 Mar 2022 13:49:56 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#14).

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated only for small build side (e.g. #rows <= 1024) of a
broadcast join. They will only be applied on ORC tables and be pushed
down to the ORC reader(i.e. ORC lib). To avoid exploding the IN-list,
if #rows of the build side exceeds the threshold (1024), we set the
filter to ALWAYS_TRUE and clear its entry. The threshold can be
configured by a new query option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set) and the lack of codegen. So we disable it at row
level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-11140: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-11141: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-11142: Consider IN-list filters in partitioned joins.

Tests:
 - Test IN-list filter on string, date and all integer types
 - Test IN-list filter with NULL
 - Test IN-list filter on complex exprs targets

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
A testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
M tests/query_test/test_runtime_filters.py
44 files changed, 1,365 insertions(+), 213 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/14
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 14
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Qifan Chen (Code Review)" <ge...@cloudera.org>.
Qifan Chen has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 18: Code-Review+2

(2 comments)

Excellent!

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.cc@1486
PS17, Line 1486: if (in_list_filter->ContainsNull()) {
               :       // Add a null literal with type.
               :      
> It's not a column type. We are using this constructor to create a null lite
It makes a lot of sense now :-).  Thanks!

nit. Maybe add a comment or spell out the detail here?


http://gerrit.cloudera.org:8080/#/c/18141/17/common/thrift/PlanNodes.thrift
File common/thrift/PlanNodes.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/17/common/thrift/PlanNodes.thrift@a134
PS17, Line 134: 
> These are removed. Do you mean we should keep them?
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 18
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Fri, 25 Feb 2022 13:44:01 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Quanlong Huang has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 8:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1221
PS4, Line 1221: 
Sorry that I'm not quite understand these.

> I was originally thinking that when the target of a IN-list filter is partition columns, then the target can be removed in FE.
> Doing the test here means such targets are retained in the plan and do not contribute.

Do you mean eliminating the partitions in FE? The IN-list filters are generated in runtime based on the build side data of hash joins. I'm afraid we are unable to eliminate them in the plan. Instead, we will eliminate them in runtime in the code link you pasted, ie. HdfsScanNodeBase::PartitionPassesFilters(). Did I miss something?

> Personally, I feel we should allow the target to be a partition column in this patch to pick up good performance gain, especially for large tables with hundreds of partitions. The code to deal with partition column is here: https://github.com/apache/impala/blob/master/be/src/exec/hdfs-scan-node-base.cc#L922. Seems your code will work out of box in this situation if line @1221 is removed.

UpdateSearchArgumentWithFilters() is only used in the orc scanner to push down filters into the ORC lib. We need line 1221 since partition columns don't exist in the ORC files.

The logics of HdfsScanNodeBase::PartitionPassesFilters() still apply on IN-list filters. I don't see it skip using IN-list filters. So we already support it that filtering out unrelated partitions by the IN-list filters. Or did I miss something?


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1271
PS4, Line 1271: ataType predicate_type
> Calling PrepareSearchArguments() for each ORC stripe may be an overkill.

Yeah, it could be an overkill if we have lots of predicates and runtime IN-list filters to push down. Runtime filters arrive randomly so we need to call this whenever there is a new runtime filter arrive. I think we can improve this by checking the arrival filters count in PrepareSearchArguments() and return if no new IN-list filters arrive.

> My understanding is that there is a consolidation step to merge the filters from different partitions (for PARTITIONED HJ). Only the merged filter can arrive at the scan node. For BROADCAST HJ, such merge step os not needed.

Yeah, we don't have the merge step for IN-list filter. However, they can arrive here since the coordinator will still publish them.


http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
File fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java:

http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@394
PS4, Line 394: r
> It also depends on how ORC layer handles the types. 
The above casting is handled in BE in the orc scanner, because the underlying ORC files could have different schemas. We can only know the file schema after we parse the file footer. The casting codes are in HdfsOrcScanner::GetSearchArgumentLiteral().

I think in FE, we just need to make sure these types are supported in BE. The BE codes will cast values based on the ORC file schema, or skip using the filter if the casting failed.

BTW, the Java implementation of the ORC lib is slightly different to its C++ implementation. The ORC C++ lib currently supports these types: https://github.com/apache/orc/blob/rel/release-1.7.0/c++/include/orc/sargs/Literal.hh#L72-L110



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 8
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Wed, 16 Feb 2022 02:55:14 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Qifan Chen (Code Review)" <ge...@cloudera.org>.
Qifan Chen has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 6:

(6 comments)

Thanks!

http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG@30
PS4, Line 30: 
> change to "with"? It means the IN-list has 4 items.
Okay.


http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG@34
PS4, Line 34:  ps_partkey and l_suppkey = ps_suppkey;
            : 
> You are right but not sure we have misunderstanding here. There are two kin
Good to know! Thanks for the explanation.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1221
PS4, Line 1221: f (in_list_filter->AlwaysTrue()) continue;
> Yeah, the check is done by FE: https://github.com/apache/impala/blob/6c845e
I was originally thinking that when the target of a IN-list filter is partition columns, then the target can be removed in FE.

Doing the test here means such targets are retained in the plan and do not contribute.

Personally, I feel we should allow the target to be a partition column in this patch to pick up good performance gain, especially for large tables with hundreds of partitions. The code to deal with partition column is here: https://github.com/apache/impala/blob/master/be/src/exec/hdfs-scan-node-base.cc#L922. Seems your code will work out of box in this situation if line @1221 is removed.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1271
PS4, Line 1271: 
> PrepareSearchArguments() will be called multiple times after this patch. Th
Okay. 

Calling PrepareSearchArguments() for each ORC stripe may be an overkill. My understanding is that there is a consolidation step to merge the filters from different partitions (for PARTITIONED HJ). Only the merged filter can arrive at the scan node. For BROADCAST HJ, such merge step os not needed.


http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
File fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java:

http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@394
PS4, Line 394: r
> I think it's assumed that both sides are casted to the same type. EQUALS pr
It also depends on how ORC layer handles the types. 

From https://orc.apache.org/api/orc-core/org/apache/orc/Reader.Options.html, https://orc.apache.org/api/hive-storage-api/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.html?is-external=true and https://orc.apache.org/api/hive-storage-api/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.html, it seems the literal list can only take one of the four primitive typed objects: Integer, Long, Double, or String. Denote such a type T.  Then technically, it is sufficient that both the inner and the outer, after optional casting, are of type T. Note also that we need to verify the surviving column values because of IN-list predicates being mapped to ORC bloom filters. 

The rules of casting may be like this, in the order of priority. 

1. If either the inner or outer is small/tiny int, cast both to int;
2. If either is less than or equal to int, cast both to int;
3. If either is less than or equal to big int, cast both to big int;
4. If either is less than or equal to double, cast both to double;
5. If either is SQL character types, cast both to string;


I think it is a good idea to verify the types here to make it possible to detect type mismatch early.


http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@742
PS4, Line 742:   public int compare(RuntimeFilter a, RuntimeFilter b) {
> I think it's very likely that partitioned HJs will exceed the threshold. Bu
Sounds like a good idea to handle partitioned HJs in another JIRA.

We can borrow BE code from min/max filters to handle both 1) and 2).



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 6
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Mon, 14 Feb 2022 17:04:02 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 14:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10211/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 14
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Wed, 23 Feb 2022 07:57:41 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Quanlong Huang has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 14:

(14 comments)

Thank Qifan for the detailed review!

http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc
File be/src/util/in-list-filter-ir.cc:

http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc@26
PS13, Line 26:   if (UNLIKELY(val == nullptr)) {
> UNLIKELY
Done


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc@30
PS13, Line 30:   if (UNLIKELY(values_.size() >= entry_limit_ || str_values_.size() >= entry_limit_)) {
> UNLIKELY
Done


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc@55
PS13, Line 55: if (UNLIKELY(s->ptr == nullptr)) {
             :         contains_null_ = true
> nit. should we check null-ness again? See line at 26.
The default constructor of StringValue creates a null 'ptr'. I think we'd better check this.
https://github.com/apache/impala/blob/fe04c50/be/src/runtime/string-value.h#L51


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc@58
PS13, Line 58: str_total
> nit. Probably should be named as str_total_size_.
Done


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.h
File be/src/util/in-list-filter.h:

http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.h@39
PS13, Line 39:   InListFilter(ColumnType type, uint32_t entry_limit, bool contains_null = false);
> Include contains_null and column type here.
Done


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc
File be/src/util/in-list-filter.cc:

http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@63
PS13, Line 63: retur
> "return false" here helps with release code.
Done


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@71
PS13, Line 71: if (type.type == TYPE
> nit. it is better to supply the type in the cstr.
Done


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@78
PS13, Line 78: 
             : InListFilter* InListFilter::Create(const InListFilterPB& protobuf, ColumnType type,
             :     uint32_t entry_limit, ObjectPool* pool) {
> nit. probably should inited in the cstr.
Done


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@100
PS13, Line 100: break;
> return null?
Done


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@112
PS13, Line 112: 
              : 
              : void InListFilter::ToProtobuf(const InListFilter* filter, InListFilterPB* protobuf) {
              :   DCHECK(protobuf != nullptr);
              :   i
> Other fields that are not copied: type_, contains_null_,  str_size_ and ent
Oops, we don't need this method. Removed it.


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@142
PS13, Line 142:   if (type_ == TYPE_STRING || type_ == TYPE_VARCHAR || type_ == TYPE_CHAR) {
> Same comment for Copy method: missing fields.
Done


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@174
PS13, Line 174: 
> should handle null case.
Oops! Done.


http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
File testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test:

http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test@787
PS13, Line 787: broadcast
> May repeat this test with partition HJ to verify that in-list filters is no
Done. I change the test to verify distributed plan in order to show the partitioned join. Otherwise it's getting the single node plan by default.


http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
File testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test:

http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test@127
PS13, Line 127: 
> may add a test on date column type.
Sure. The ORC date_tbl is corrupted and need to wait for https://gerrit.cloudera.org/c/18262/ to be merged.



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 14
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Wed, 23 Feb 2022 07:33:09 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#17).

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated for the build side of a broadcast join. They will only be
applied on ORC tables and be pushed down to the ORC reader(i.e. ORC
lib). To avoid exploding the IN-list, if # of distinct values of the
build side exceeds a threshold (default to 1024), we set the filter to
ALWAYS_TRUE and clear its entry. The threshold can be configured by a
new query option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set) and the lack of codegen. So we disable it at row
level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-11140: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-11141: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-11142: Consider IN-list filters in partitioned joins.

Tests:
 - Test IN-list filter on string, date and all integer types
 - Test IN-list filter with NULL
 - Test IN-list filter on complex exprs targets

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
A testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
M tests/query_test/test_runtime_filters.py
44 files changed, 1,428 insertions(+), 214 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/17
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 17
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 19:

Build started: https://jenkins.impala.io/job/gerrit-verify-dryrun/7879/ DRY_RUN=false


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 19
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sat, 26 Feb 2022 01:23:08 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 21: Verified+1


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 21
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sun, 27 Feb 2022 18:11:04 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 23: Verified+1


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 23
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Thu, 03 Mar 2022 00:21:05 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#4).

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't leverage the bloom filters in
ORC files. Because only EQUALS and IN-list predicates can leverage them
to skip unrelated ORC RowGroups, and we can't convert runtime bloom
filters or min-max filters into such predicates.

This patch adds runtime IN-list filters for small build side (e.g. #rows
<= 1024) of a broadcast join. Currently the IN-list filters will only
apply to ORC tables and be pushed down to the ORC reader(i.e. ORC lib).
Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set). So we disable it at row level.

Example query that will benefit from this patch:
  use tpch_orc_def;
  select count(*) from lineitem_bf join (
    select * from partsupp, part
    where ps_partkey = p_partkey and p_size = 15
      and p_type like '%BRASS' and ps_availqty < 10) v
  on l_partkey = ps_partkey and l_suppkey = ps_suppkey;

The inline-view populates a runtime IN-list filter of 4 items. Note that
we need to re-generate the lineitem table with bloom filters enabled
(e.g. setting orc.bloom.filter.columns to
"l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity" in
tblproperties), so the pushed down IN-list filter can have a better
filter rate.

TODO: fix tests due to plan changes.

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/util/CMakeLists.txt
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/PlanNodes.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M tests/query_test/test_runtime_filters.py
31 files changed, 748 insertions(+), 122 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/4
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 4
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#23).

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated for the build side of a broadcast join. They will only be
applied on ORC tables and be pushed down to the ORC reader(i.e. ORC
lib). To avoid exploding the IN-list, if # of distinct values of the
build side exceeds a threshold (default to 1024), we set the filter to
ALWAYS_TRUE and clear its entry. The threshold can be configured by a
new query option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set) and the lack of codegen. So we disable it at row
level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-11140: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-11141: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-11142: Consider IN-list filters in partitioned joins.

Tests:
 - Test IN-list filter on string, date and all integer types
 - Test IN-list filter with NULL
 - Test IN-list filter on complex exprs targets

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
A testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
M tests/query_test/test_runtime_filters.py
44 files changed, 1,430 insertions(+), 215 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/23
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 23
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Quanlong Huang has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 5:

(16 comments)

http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG@10
PS4, Line 10: ++ reader supports pushing down predicates to skip
            : unreleated RowGroups. The pushed down predicates will be evaludated on
            : file indexes (i.e. statistics and bloom filter indexes). Note that only
            : EQUALS and IN-list predicates can leverage bloom
> May reword as
Sorry for making this unclear.

* The native ORC library can accept many kinds of predicates, not just EQUALS and IN-list predicates, but also comparison (e.g. <, >, >=) and IS-[NOT]-NULL predicates, etc. They can both be used to skip unreleated ORC RowGroups.
* Each ORC files can have optional bloom filters on different columns.
* Only EQUALS and IN-list predicates can leverage these file-level bloom filters.

Updated the sentenses. But not sure if they are clear enough.


http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG@18
PS4, Line 18: indexes.
            : 
            : This patch adds runtime IN-list filters for this 
> Suggest to mention it after the introduction section. That is, right before
Done


http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG@30
PS4, Line 30: 
> nit. remove
change to "with"? It means the IN-list has 4 items.


http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG@34
PS4, Line 34:  ps_partkey and l_suppkey = ps_suppkey;
            : 
> Not sure if this is right. I thought IN-list will be done inside ORC librar
You are right but not sure we have misunderstanding here. There are two kinds of bloom filters:

* Runtime bloom filters generated by Impala
* Bloom filter indexes in the ORC files (generated by Hive when inserting the table)

If the lineitem table is generated with bloom filter indexes, the runtime IN-list filter can have a better filter rate.
Updated the sentense.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1221
PS4, Line 1221: / Only apply runtime filters on non-partition columns.
> Looks like this can be done in FE.
Yeah, the check is done by FE: https://github.com/apache/impala/blob/6c845eb24b952972975126e07a36cd1565ada629/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java#L936

Here we only check the flag set by FE.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1271
PS4, Line 1271: < filter->id();
> I wonder if this method UpdateSearchArgumentWithFilters() is called only on
PrepareSearchArguments() will be called multiple times after this patch. Thus the same as UpdateSearchArgumentWithFilters(). The reason is runtime filters will arrive in runtime. So we re-build the SearchArgument each time we start reading a new ORC stripe.

However, the above situation seems impossible. When an IN-list filter arrived, it won't be updated anymore. So the predicate should remain the same.

BTW, I updated the method comment of PrepareSearchArguments(). Please let me know if it's unclear.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/partitioned-hash-join-builder.cc
File be/src/exec/partitioned-hash-join-builder.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/partitioned-hash-join-builder.cc@959
PS4, Line 959: //TODO: IN-list filter threshold (default 1024).
> Sounds like this is quite important.  When the # items in the list in HJ bu
Yeah, I added this in the commit message in PS5. Also added the query option.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/coordinator.cc@599
PS4, Line 599: In-l
> In-list size?
Done


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/runtime-filter-ir.cc
File be/src/runtime/runtime-filter-ir.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/runtime-filter-ir.cc@40
PS4, Line 40: 
            :     case TRuntimeFilterType::IN_LIST: {
> Seems to me IN_list will shine in performance when applied to partition col
Thanks for catching this! I thought this will only be evaludated in rows level. I should add the skip logic in scanners.

EDIT: moved the check to HdfsScanner::EvalRuntimeFilter()


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/runtime-filter.inline.h
File be/src/runtime/runtime-filter.inline.h:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/runtime-filter.inline.h@32
PS4, Line 32: switch (filter_desc()
> Switch on filter_desc().type to save some IF tests?
Good point! Done.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/runtime-filter.inline.h@43
PS4, Line 43: line bool RuntimeFilt
> Switch on filter_desc().type to save some IF tests?
Done


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/util/in-list-filter-ir.cc
File be/src/util/in-list-filter-ir.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/util/in-list-filter-ir.cc@30
PS4, Line 30:  ent
> Turn this into a query option?
Done


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/util/in-list-filter.h
File be/src/util/in-list-filter.h:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/util/in-list-filter.h@89
PS4, Line 89: ll nume
> May consider the exact type (int8_t, int16_t, int32_t or int64_t), similar 
Yeah, I planned to change the implementation in the future. So choose int64_t for simplicity. Added a TODO for this.


http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
File fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java:

http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@394
PS4, Line 394: r
> Seems we need also check the inner side too?
I think it's assumed that both sides are casted to the same type. EQUALS predicates are analyzed in BinaryPredicate#analyzeImpl(). A builtin function will be matched if the parameters can be casted [1]. All EQUALS builtin functions are registered in the way that the two args are in the same type [2]. The parameters are then wrapped with CAST expressions in castForFunctionCall() [3].

There is a check for this in BinaryPredicate#toThrift() [4]. I'll also add a check here in case I misunderstand something.

[1] https://github.com/apache/impala/blob/6c845eb24b952972975126e07a36cd1565ada629/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java#L205
[2] https://github.com/apache/impala/blob/6c845eb24b952972975126e07a36cd1565ada629/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java#L126-L127
[3] https://github.com/apache/impala/blob/6c845eb24b952972975126e07a36cd1565ada629/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java#L238
[4] https://github.com/apache/impala/blob/6c845eb24b952972975126e07a36cd1565ada629/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java#L184-L185


http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@742
PS4, Line 742:   public int compare(RuntimeFilter a, RuntimeFilter b) {
> Just wonder why not for partitioned HJs. We may use the following to decide
I think it's very likely that partitioned HJs will exceed the threshold. But if the build side has a selective predicate, it could still be a good scenerio for IN-list filter.

Can we do this in a follow-up JIRA? Some BE codes currently assume that only broadcast joins have runtime IN-list filtlers.


http://gerrit.cloudera.org:8080/#/c/18141/4/tests/query_test/test_runtime_filters.py
File tests/query_test/test_runtime_filters.py:

http://gerrit.cloudera.org:8080/#/c/18141/4/tests/query_test/test_runtime_filters.py@322
PS4, Line 322: 
> nit. remove
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 5
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 08 Feb 2022 02:30:28 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 5:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/18141/5/tests/query_test/test_runtime_filters.py
File tests/query_test/test_runtime_filters.py:

http://gerrit.cloudera.org:8080/#/c/18141/5/tests/query_test/test_runtime_filters.py@70
PS5, Line 70: [
flake8: E131 continuation line unaligned for hanging indent



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 5
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 08 Feb 2022 02:30:52 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 6:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/18141/6/tests/query_test/test_runtime_filters.py
File tests/query_test/test_runtime_filters.py:

http://gerrit.cloudera.org:8080/#/c/18141/6/tests/query_test/test_runtime_filters.py@70
PS6, Line 70: [
flake8: E131 continuation line unaligned for hanging indent



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 6
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 08 Feb 2022 03:04:50 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#8).

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated only for small build side (e.g. #rows <= 1024) of a
broadcast join. They will only be applied on ORC tables and be pushed
down to the ORC reader(i.e. ORC lib). To avoid exploding the IN-list,
if #rows of the build side exceeds the threshold (1024), we set the
filter to ALWAYS_TRUE. The threshold can be configured by a new query
option, runtime_in_list_filter_entry_limit.

Example query that will benefit from this patch:
  use tpch_orc_def;
  select count(*) from lineitem_bf join (
    select * from partsupp, part
    where ps_partkey = p_partkey and p_size = 15
      and p_type like '%BRASS' and ps_availqty < 10) v
  on l_partkey = ps_partkey and l_suppkey = ps_suppkey;

The inline-view populates a runtime IN-list filter with 4 items. Note that
we need to re-generate the lineitem table with bloom filter indexes enabled
(e.g. setting orc.bloom.filter.columns to
"l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity" in
tblproperties before inserting the data), so the runtime IN-list filter
can have a better filter rate.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set). So we disable it at row level.

TODO: Codegen InListFilter::Insert() and InListFilter::Find().

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Tests:
 - Many planner tests have changes in the runtime filter ids.
 - TODO: Test IN-list filter with NULLs
 - TODO: Test IN-list filter on complex exprs targets
 - TODO: Test IN-list filter on all types including DATE

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test
M testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
M testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
M testdata/workloads/functional-planner/queries/PlannerTest/bloom-filter-assignment.test
M testdata/workloads/functional-planner/queries/PlannerTest/card-inner-join.test
M testdata/workloads/functional-planner/queries/PlannerTest/card-multi-join.test
M testdata/workloads/functional-planner/queries/PlannerTest/card-outer-join.test
M testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
M testdata/workloads/functional-planner/queries/PlannerTest/conjunct-ordering.test
M testdata/workloads/functional-planner/queries/PlannerTest/constant-propagation.test
M testdata/workloads/functional-planner/queries/PlannerTest/convert-to-cnf.test
M testdata/workloads/functional-planner/queries/PlannerTest/ddl.test
M testdata/workloads/functional-planner/queries/PlannerTest/default-join-distr-mode-broadcast.test
M testdata/workloads/functional-planner/queries/PlannerTest/default-join-distr-mode-shuffle.test
M testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test
M testdata/workloads/functional-planner/queries/PlannerTest/disable-preaggregations.test
M testdata/workloads/functional-planner/queries/PlannerTest/disable-runtime-overlap-filter.test
M testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test
M testdata/workloads/functional-planner/queries/PlannerTest/distinct.test
M testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection-hdfs-num-rows-est-enabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
M testdata/workloads/functional-planner/queries/PlannerTest/hbase-no-key-est.test
M testdata/workloads/functional-planner/queries/PlannerTest/hbase.test
M testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test
M testdata/workloads/functional-planner/queries/PlannerTest/implicit-joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test
M testdata/workloads/functional-planner/queries/PlannerTest/insert-hdfs-writer-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/insert-sort-by-zorder.test
M testdata/workloads/functional-planner/queries/PlannerTest/insert.test
M testdata/workloads/functional-planner/queries/PlannerTest/join-order.test
M testdata/workloads/functional-planner/queries/PlannerTest/joins-hdfs-num-rows-est-enabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-delete.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-replica-selection-leader-only.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-update.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-upsert.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
M testdata/workloads/functional-planner/queries/PlannerTest/limit-pushdown-analytic.test
M testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
M testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters-hdfs-num-rows-est-enabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
M testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
M testdata/workloads/functional-planner/queries/PlannerTest/multiple-distinct-materialization.test
M testdata/workloads/functional-planner/queries/PlannerTest/multiple-distinct-predicates.test
M testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test
M testdata/workloads/functional-planner/queries/PlannerTest/optimize-simple-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/order.test
M testdata/workloads/functional-planner/queries/PlannerTest/outer-joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/outer-to-inner-joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering-disabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
M testdata/workloads/functional-planner/queries/PlannerTest/parquet-stats-agg.test
M testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
M testdata/workloads/functional-planner/queries/PlannerTest/preagg-bytes-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-propagation.test
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
M testdata/workloads/functional-planner/queries/PlannerTest/scan-node-fs-scheme.test
M testdata/workloads/functional-planner/queries/PlannerTest/semi-join-distinct.test
M testdata/workloads/functional-planner/queries/PlannerTest/setoperation-rewrite.test
M testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
M testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
M testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test
M testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
M testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit-small.test
M testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds-dist-method.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q02.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q06.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q09.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q22.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q28.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q38.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q50.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q51.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q59.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q62.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q65.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q72.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q76.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q77.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q78.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q79.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q82.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q83.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q86.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q87.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q88.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q89.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q90.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q92.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q93.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q96.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q97.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q98.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q99.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test
M testdata/workloads/functional-planner/queries/PlannerTest/union.test
M testdata/workloads/functional-planner/queries/PlannerTest/views.test
M testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test
M tests/query_test/test_runtime_filters.py
215 files changed, 7,294 insertions(+), 6,609 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/8
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 8
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 9:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/18141/9/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/9/be/src/exec/hdfs-orc-scanner.cc@318
PS9, Line 318:       ADD_COUNTER(scan_node_->runtime_profile(), "NumPushedDownRuntimeFilters", TUnit::UNIT);
line too long (93 > 90)


http://gerrit.cloudera.org:8080/#/c/18141/9/tests/query_test/test_runtime_filters.py
File tests/query_test/test_runtime_filters.py:

http://gerrit.cloudera.org:8080/#/c/18141/9/tests/query_test/test_runtime_filters.py@70
PS9, Line 70: [
flake8: E131 continuation line unaligned for hanging indent



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 9
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Fri, 18 Feb 2022 06:35:45 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#10).

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated only for small build side (e.g. #rows <= 1024) of a
broadcast join. They will only be applied on ORC tables and be pushed
down to the ORC reader(i.e. ORC lib). To avoid exploding the IN-list,
if #rows of the build side exceeds the threshold (1024), we set the
filter to ALWAYS_TRUE. The threshold can be configured by a new query
option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Example query that will benefit from this patch:
  use tpch_orc_def;
  select count(*) from lineitem_bf join (
    select * from partsupp, part
    where ps_partkey = p_partkey and p_size = 15
      and p_type like '%BRASS' and ps_availqty < 10) v
  on l_partkey = ps_partkey and l_suppkey = ps_suppkey;

The inline-view populates a runtime IN-list filter with 4 items. Note that
we need to re-generate the lineitem table with bloom filter indexes enabled
(e.g. setting orc.bloom.filter.columns to
"l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity" in
tblproperties before inserting the data), so the runtime IN-list filter
can have a better filter rate.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set). So we disable it at row level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-xxxxx: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-xxxxx: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-xxxxx: Consider IN-list filters in partitioned joins.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Tests:
 - TODO: Test IN-list filter with NULLs
 - TODO: Test IN-list filter on complex exprs targets
 - TODO: Test IN-list filter on all types including DATE

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M tests/query_test/test_runtime_filters.py
40 files changed, 1,017 insertions(+), 208 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/10
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 10
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 17:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10220/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 17
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Thu, 24 Feb 2022 13:22:36 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 20:

Build started: https://jenkins.impala.io/job/gerrit-verify-dryrun/7882/ DRY_RUN=true


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 20
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sun, 27 Feb 2022 00:23:29 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 20: Verified-1

Build failed: https://jenkins.impala.io/job/gerrit-verify-dryrun/7882/


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 20
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sun, 27 Feb 2022 04:43:36 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 6:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10115/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 6
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 08 Feb 2022 03:29:53 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Qifan Chen (Code Review)" <ge...@cloudera.org>.
Qifan Chen has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 4:

(16 comments)

Looks good to me!

http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG@10
PS4, Line 10: Unfortunately they can't leverage the bloom filters in
            : ORC files. Because only EQUALS and IN-list predicates can leverage them
            : to skip unrelated ORC RowGroups, and we can't convert runtime bloom
            : filters or min-max filters into such predicates.
May reword as

Unfortunately the native ORC library can only accept EQUALS and IN-list to skip related ORC RowGroups, to which both runtime bloom or min-max filters can't be converted.


http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG@18
PS4, Line 18: Evaluating runtime IN-list filters is much slower than evaluating
            : runtime bloom filters due to the current simple implementation (i.e.
            : std::unorder_set). So we disable it at row level.
Suggest to mention it after the introduction section. That is, right before the TODO.


http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG@30
PS4, Line 30: of
nit. remove


http://gerrit.cloudera.org:8080/#/c/18141/4//COMMIT_MSG@34
PS4, Line 34: so the pushed down IN-list filter can have a better
            : filter rate.
Not sure if this is right. I thought IN-list will be done inside ORC library layer and bloom in impala layer. 

Maybe say it as: Note that in-list filters and bloom filters are orthogonal because of different operation locations, it is desirable to keep the bloom filters in the query plan.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1221
PS4, Line 1221: f (filter->IsBoundByPartitionColumn(GetScanNodeId())) continue;
Looks like this can be done in FE.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1271
PS4, Line 1271: PrepareInListPredicate
I wonder if this method UpdateSearchArgumentWithFilters() is called only once. Since PrepareInListPredicate() can put the ORC predicate in two forms, and if this method is called multiple times, then we could end up with the following interesting situation:

1. List of 1 item -> EQUALS form;
2. List of 4 times -> IN-LIST form;

The final form should be the IN-LIST form including the item from 1.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/partitioned-hash-join-builder.cc
File be/src/exec/partitioned-hash-join-builder.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/partitioned-hash-join-builder.cc@959
PS4, Line 959: //TODO: IN-list filter threshold (default 1024).
Sounds like this is quite important.  When the # items in the list in HJ builder is over the threshold, we set the filter to ALWAYS TRUE.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/coordinator.cc@599
PS4, Line 599: List
In-list size?


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/runtime-filter-ir.cc
File be/src/runtime/runtime-filter-ir.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/runtime-filter-ir.cc@40
PS4, Line 40: Evaluating IN-list filter is much slower than evaluating the corresponding bloom
            :     // filter. Skip it until we improve its performance.
Seems to me IN_list will shine in performance when applied to partition columns.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/runtime-filter.inline.h
File be/src/runtime/runtime-filter.inline.h:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/runtime-filter.inline.h@32
PS4, Line 32: if (is_bloom_filter()
Switch on filter_desc().type to save some IF tests?


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/runtime/runtime-filter.inline.h@43
PS4, Line 43: if (is_bloom_filter()
Switch on filter_desc().type to save some IF tests?


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/util/in-list-filter-ir.cc
File be/src/util/in-list-filter-ir.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/util/in-list-filter-ir.cc@30
PS4, Line 30: 1024
Turn this into a query option?


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/util/in-list-filter.h
File be/src/util/in-list-filter.h:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/util/in-list-filter.h@89
PS4, Line 89: int64_t
May consider the exact type (int8_t, int16_t, int32_t or int64_t), similar to min/max filters, to save memory space.


http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
File fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java:

http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@394
PS4, Line 394: 0
Seems we need also check the inner side too?


http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@742
PS4, Line 742: // Don't generate IN-list filters for partitioned joins.
Just wonder why not for partitioned HJs. We may use the following to decide.

1) Partitioned or broadcasted HJ
2) Each join column =has stats and the total # of rows <= in-list threshold (say 1024)


http://gerrit.cloudera.org:8080/#/c/18141/4/tests/query_test/test_runtime_filters.py
File tests/query_test/test_runtime_filters.py:

http://gerrit.cloudera.org:8080/#/c/18141/4/tests/query_test/test_runtime_filters.py@322
PS4, Line 322: both
nit. remove



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 4
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Comment-Date: Fri, 04 Feb 2022 17:57:38 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Quanlong Huang has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 8:

> Patch Set 6:
> 
> (6 comments)
> 
> Thanks!

Thank Qifan! I'll address your comments in the next patch set.

Patch set 7 fixes the failed tests and add two profile counters.
Patch set 8 is a rebase to fix the merge conflicts.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 8
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Mon, 14 Feb 2022 23:43:24 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#7).

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated only for small build side (e.g. #rows <= 1024) of a
broadcast join. They will only be applied on ORC tables and be pushed
down to the ORC reader(i.e. ORC lib). To avoid exploding the IN-list,
if #rows of the build side exceeds the threshold (1024), we set the
filter to ALWAYS_TRUE. The threshold can be configured by a new query
option, runtime_in_list_filter_entry_limit.

Example query that will benefit from this patch:
  use tpch_orc_def;
  select count(*) from lineitem_bf join (
    select * from partsupp, part
    where ps_partkey = p_partkey and p_size = 15
      and p_type like '%BRASS' and ps_availqty < 10) v
  on l_partkey = ps_partkey and l_suppkey = ps_suppkey;

The inline-view populates a runtime IN-list filter with 4 items. Note that
we need to re-generate the lineitem table with bloom filter indexes enabled
(e.g. setting orc.bloom.filter.columns to
"l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity" in
tblproperties before inserting the data), so the runtime IN-list filter
can have a better filter rate.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set). So we disable it at row level.

TODO: Codegen InListFilter::Insert() and InListFilter::Find().

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Tests:
 - Many planner tests have changes in the runtime filter ids.
 - TODO: Test IN-list filter with NULLs
 - TODO: Test IN-list filter on complex exprs targets
 - TODO: Test IN-list filter on all types including DATE

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test
M testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
M testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
M testdata/workloads/functional-planner/queries/PlannerTest/bloom-filter-assignment.test
M testdata/workloads/functional-planner/queries/PlannerTest/card-inner-join.test
M testdata/workloads/functional-planner/queries/PlannerTest/card-multi-join.test
M testdata/workloads/functional-planner/queries/PlannerTest/card-outer-join.test
M testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
M testdata/workloads/functional-planner/queries/PlannerTest/conjunct-ordering.test
M testdata/workloads/functional-planner/queries/PlannerTest/constant-propagation.test
M testdata/workloads/functional-planner/queries/PlannerTest/convert-to-cnf.test
M testdata/workloads/functional-planner/queries/PlannerTest/ddl.test
M testdata/workloads/functional-planner/queries/PlannerTest/default-join-distr-mode-broadcast.test
M testdata/workloads/functional-planner/queries/PlannerTest/default-join-distr-mode-shuffle.test
M testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test
M testdata/workloads/functional-planner/queries/PlannerTest/disable-preaggregations.test
M testdata/workloads/functional-planner/queries/PlannerTest/disable-runtime-overlap-filter.test
M testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test
M testdata/workloads/functional-planner/queries/PlannerTest/distinct.test
M testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection-hdfs-num-rows-est-enabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
M testdata/workloads/functional-planner/queries/PlannerTest/hbase-no-key-est.test
M testdata/workloads/functional-planner/queries/PlannerTest/hbase.test
M testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test
M testdata/workloads/functional-planner/queries/PlannerTest/implicit-joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test
M testdata/workloads/functional-planner/queries/PlannerTest/insert-hdfs-writer-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/insert-sort-by-zorder.test
M testdata/workloads/functional-planner/queries/PlannerTest/insert.test
M testdata/workloads/functional-planner/queries/PlannerTest/join-order.test
M testdata/workloads/functional-planner/queries/PlannerTest/joins-hdfs-num-rows-est-enabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-delete.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-replica-selection-leader-only.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-update.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-upsert.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
M testdata/workloads/functional-planner/queries/PlannerTest/limit-pushdown-analytic.test
M testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
M testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters-hdfs-num-rows-est-enabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
M testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
M testdata/workloads/functional-planner/queries/PlannerTest/multiple-distinct-materialization.test
M testdata/workloads/functional-planner/queries/PlannerTest/multiple-distinct-predicates.test
M testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test
M testdata/workloads/functional-planner/queries/PlannerTest/optimize-simple-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/order.test
M testdata/workloads/functional-planner/queries/PlannerTest/outer-joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/outer-to-inner-joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering-disabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
M testdata/workloads/functional-planner/queries/PlannerTest/parquet-stats-agg.test
M testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
M testdata/workloads/functional-planner/queries/PlannerTest/preagg-bytes-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-propagation.test
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
M testdata/workloads/functional-planner/queries/PlannerTest/scan-node-fs-scheme.test
M testdata/workloads/functional-planner/queries/PlannerTest/semi-join-distinct.test
M testdata/workloads/functional-planner/queries/PlannerTest/setoperation-rewrite.test
M testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
M testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
M testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test
M testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
M testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit-small.test
M testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds-dist-method.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q02.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q06.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q09.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q22.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q28.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q38.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q50.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q51.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q59.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q62.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q65.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q72.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q76.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q77.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q78.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q79.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q82.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q83.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q86.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q87.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q88.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q89.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q90.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q92.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q93.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q96.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q97.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q98.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q99.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test
M testdata/workloads/functional-planner/queries/PlannerTest/union.test
M testdata/workloads/functional-planner/queries/PlannerTest/views.test
M testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test
M tests/query_test/test_runtime_filters.py
215 files changed, 7,294 insertions(+), 6,609 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/7
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 7
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 13:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/10189/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 13
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 22 Feb 2022 01:32:37 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Qifan Chen (Code Review)" <ge...@cloudera.org>.
Qifan Chen has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 13:

(18 comments)

Looks very good!

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1271
PS4, Line 1271: gumentBuilder* sarg) {
> hmm, I don't think we will use unmerged versions of runtime filters here. f
Yeah, since the focus of the patch is for broadcasting HJ, I think we are okay. 

The merge for partitioned HJ is done here https://github.com/apache/impala/blob/master/be/src/runtime/runtime-filter-bank.cc#L206 and here https://github.com/apache/impala/blob/master/be/src/runtime/runtime-filter.cc#L52. 

We probably should specifically handle in-list case for https://github.com/apache/impala/blob/master/be/src/runtime/runtime-filter.cc#L52.


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc
File be/src/util/in-list-filter-ir.cc:

http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc@26
PS13, Line 26:   if (val == nullptr) {
UNLIKELY


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc@30
PS13, Line 30:   if (values_.size() >= entry_limit_ || str_values_.size() >= entry_limit_) {
UNLIKELY


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc@55
PS13, Line 55: if (s->ptr == nullptr) {
             :         contains_null_ = true
nit. should we check null-ness again? See line at 26.


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter-ir.cc@58
PS13, Line 58: str_size_
nit. Probably should be named as str_total_size_.


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.h
File be/src/util/in-list-filter.h:

http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.h@39
PS13, Line 39:   InListFilter(uint32_t entry_limit) : always_true_(false), entry_limit_(entry_limit) {}
Include contains_null and column type here.


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc
File be/src/util/in-list-filter.cc:

http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@63
PS13, Line 63: break
"return false" here helps with release code.


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@71
PS13, Line 71: filter->type_ = type;
nit. it is better to supply the type in the cstr.


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@78
PS13, Line 78:   filter->type_ = type;
             :   filter->contains_null_ = protobuf.contains_null();
             :   filter->always_true_ = protobuf.always_true();
nit. probably should inited in the cstr.


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@100
PS13, Line 100: break;
return null?


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@112
PS13, Line 112: out->set_always_true(in.always_true());
              :   for (ColumnValuePB in_value : in.value()) {
              :     ColumnValuePB* out_value = out->add_value();
              :     *out_value = in_value;
              :   }
Other fields that are not copied: type_, contains_null_,  str_size_ and entry_limit_.


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@142
PS13, Line 142:   }
Same comment for Copy method: missing fields.


http://gerrit.cloudera.org:8080/#/c/18141/13/be/src/util/in-list-filter.cc@174
PS13, Line 174: ']'
should handle null case.


http://gerrit.cloudera.org:8080/#/c/18141/8/common/thrift/ImpalaService.thrift
File common/thrift/ImpalaService.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/8/common/thrift/ImpalaService.thrift@725
PS8, Line 725: RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT
> I think having the RUNTIME prefix is consistent with existing options, e.g.
Done


http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
File fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java:

http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@394
PS4, Line 394: o
> I added a check for the inner side started from patch set 5: https://gerrit
Okay. Thanks.


http://gerrit.cloudera.org:8080/#/c/18141/8/testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test
File testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test:

http://gerrit.cloudera.org:8080/#/c/18141/8/testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test@8
PS8, Line 8: 3.44K
> Sorry, these should not be introduced. I replace the test files so got thes
Great. Thanks a lot for taking care if it.

I wonder if we should also check that estimated memory or cardinality do not change when no in-list filter is present in a query.


http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
File testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test:

http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test@787
PS13, Line 787: broadcast
May repeat this test with partition HJ to verify that in-list filters is not present.


http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
File testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test:

http://gerrit.cloudera.org:8080/#/c/18141/13/testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test@127
PS13, Line 127: 
may add a test on date column type.



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 13
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 22 Feb 2022 17:37:45 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 23:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10240/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 23
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Wed, 02 Mar 2022 02:24:03 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#21).

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated for the build side of a broadcast join. They will only be
applied on ORC tables and be pushed down to the ORC reader(i.e. ORC
lib). To avoid exploding the IN-list, if # of distinct values of the
build side exceeds a threshold (default to 1024), we set the filter to
ALWAYS_TRUE and clear its entry. The threshold can be configured by a
new query option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set) and the lack of codegen. So we disable it at row
level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-11140: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-11141: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-11142: Consider IN-list filters in partitioned joins.

Tests:
 - Test IN-list filter on string, date and all integer types
 - Test IN-list filter with NULL
 - Test IN-list filter on complex exprs targets

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
A testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
M tests/query_test/test_runtime_filters.py
44 files changed, 1,434 insertions(+), 215 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/21
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 21
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 22: Verified-1

Build failed: https://jenkins.impala.io/job/gerrit-verify-dryrun/7890/


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 22
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 01 Mar 2022 18:33:20 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 9:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/10181/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 9
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Fri, 18 Feb 2022 06:47:20 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 10:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10184/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 10
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Fri, 18 Feb 2022 12:15:21 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#11).

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated only for small build side (e.g. #rows <= 1024) of a
broadcast join. They will only be applied on ORC tables and be pushed
down to the ORC reader(i.e. ORC lib). To avoid exploding the IN-list,
if #rows of the build side exceeds the threshold (1024), we set the
filter to ALWAYS_TRUE and clear its entry. The threshold can be
configured by a new query option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set) and the lack of codegen. So we disable it at row
level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-11140: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-11141: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-11142: Consider IN-list filters in partitioned joins.

Tests:
 - Test IN-list filter on string, date and all integer types
 - Test IN-list filter with NULL
 - Test IN-list filter on complex exprs targets

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
A testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
M tests/query_test/test_runtime_filters.py
44 files changed, 1,268 insertions(+), 213 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/11
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 11
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 11:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/18141/11/tests/query_test/test_runtime_filters.py
File tests/query_test/test_runtime_filters.py:

http://gerrit.cloudera.org:8080/#/c/18141/11/tests/query_test/test_runtime_filters.py@320
PS11, Line 320: class TestInListFilters(ImpalaTestSuite):
flake8: E302 expected 2 blank lines, found 1



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 11
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Mon, 21 Feb 2022 02:56:19 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Quanlong Huang has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 11:

(10 comments)

Thank Qifan's feedbacks! Addressed the comments and made this patch ready to merge.

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1271
PS4, Line 1271: gumentBuilder* sarg) {
> It seems to me starting filtering without waiting for the merge version to 
hmm, I don't think we will use unmerged versions of runtime filters here. filter->HasFilter() at line 1219 guards that the filter is published.

* For global filters, the coordinator will merge them before publishing them.
* For local filters, they are generated by local instances of broadcast joins so don't need to be merged. (Only global filters generated by partitioned joins need to be merged).


http://gerrit.cloudera.org:8080/#/c/18141/10/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/10/be/src/exec/hdfs-orc-scanner.cc@318
PS10, Line 318:       ADD_COUNTER(scan_node_->runtime_profile(), "NumPushedDownRuntimeFilters",
> line too long (93 > 90)
Done


http://gerrit.cloudera.org:8080/#/c/18141/8/common/thrift/ImpalaService.thrift
File common/thrift/ImpalaService.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/8/common/thrift/ImpalaService.thrift@725
PS8, Line 725: RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT
> nit. IN_LIST_FILTER_ENTRY_LIMIT?
I think having the RUNTIME prefix is consistent with existing options, e.g. RUNTIME_BLOOM_FILTER_SIZE, RUNTIME_FILTER_WAIT_TIME_MS, RUNTIME_FILTER_WAIT_TIME_MS. So I'd like to keep this name.


http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
File fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java:

http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@394
PS4, Line 394: o
> Okay. Agree casting in BE is the right way to go if data types in orc file 
I added a check for the inner side started from patch set 5: https://gerrit.cloudera.org/c/18141/4..5/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java#394

Or do I still miss something?


http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@742
PS4, Line 742:     double aSelectivity =
> Sounds like a good idea to handle partitioned HJs in another JIRA.
Filed IMPALA-11142 for this.


http://gerrit.cloudera.org:8080/#/c/18141/8/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
File fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java:

http://gerrit.cloudera.org:8080/#/c/18141/8/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@689
PS8, Line 689: 
> I wonder if this can be improved a little bit, especially for int type, to 
Yeah, I plan to optimize this in IMPALA-11141 (to avoid this patch growing too large to review).


http://gerrit.cloudera.org:8080/#/c/18141/8/testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test
File testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test:

http://gerrit.cloudera.org:8080/#/c/18141/8/testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test@8
PS8, Line 8: 3.44K
> Saw quite many changes on cardinality. Can you explain the reason?
Sorry, these should not be introduced. I replace the test files so got these difference. We actually don't verify the cardinality in these tests.

In the latest patch set, I disable IN-list filter by default. So we don't need to change these currently.


http://gerrit.cloudera.org:8080/#/c/18141/8/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
File testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test:

http://gerrit.cloudera.org:8080/#/c/18141/8/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test@13
PS8, Line 13: part
> Is this change introduced in this patch?
Sorry, removed these.


http://gerrit.cloudera.org:8080/#/c/18141/10/tests/query_test/test_runtime_filters.py
File tests/query_test/test_runtime_filters.py:

http://gerrit.cloudera.org:8080/#/c/18141/10/tests/query_test/test_runtime_filters.py@70
PS10, Line 70: .
> flake8: E131 continuation line unaligned for hanging indent
Done


http://gerrit.cloudera.org:8080/#/c/18141/11/tests/query_test/test_runtime_filters.py
File tests/query_test/test_runtime_filters.py:

http://gerrit.cloudera.org:8080/#/c/18141/11/tests/query_test/test_runtime_filters.py@320
PS11, Line 320: class TestInListFilters(ImpalaTestSuite):
> flake8: E302 expected 2 blank lines, found 1
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 11
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Mon, 21 Feb 2022 03:39:18 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 12:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10190/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 12
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Mon, 21 Feb 2022 04:00:40 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#13).

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated only for small build side (e.g. #rows <= 1024) of a
broadcast join. They will only be applied on ORC tables and be pushed
down to the ORC reader(i.e. ORC lib). To avoid exploding the IN-list,
if #rows of the build side exceeds the threshold (1024), we set the
filter to ALWAYS_TRUE and clear its entry. The threshold can be
configured by a new query option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set) and the lack of codegen. So we disable it at row
level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-11140: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-11141: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-11142: Consider IN-list filters in partitioned joins.

Tests:
 - Test IN-list filter on string, date and all integer types
 - Test IN-list filter with NULL
 - Test IN-list filter on complex exprs targets

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
A testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
M tests/query_test/test_runtime_filters.py
44 files changed, 1,268 insertions(+), 213 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/13
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 13
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 21:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10234/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 21
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sun, 27 Feb 2022 12:55:55 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 19: Code-Review+2


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 19
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sat, 26 Feb 2022 01:23:07 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 19: Verified-1

Build failed: https://jenkins.impala.io/job/gerrit-verify-dryrun/7879/


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 19
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sat, 26 Feb 2022 08:09:31 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#5).

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated only for small build side (e.g. #rows <= 1024) of a
broadcast join. They will only be applied on ORC tables and be pushed
down to the ORC reader(i.e. ORC lib). To avoid exploding the IN-list,
if #rows of the build side exceeds the threshold (1024), we set the
filter to ALWAYS_TRUE. The threshold can be configured by a new query
option, runtime_in_list_filter_entry_limit.

Example query that will benefit from this patch:
  use tpch_orc_def;
  select count(*) from lineitem_bf join (
    select * from partsupp, part
    where ps_partkey = p_partkey and p_size = 15
      and p_type like '%BRASS' and ps_availqty < 10) v
  on l_partkey = ps_partkey and l_suppkey = ps_suppkey;

The inline-view populates a runtime IN-list filter with 4 items. Note that
we need to re-generate the lineitem table with bloom filter indexes enabled
(e.g. setting orc.bloom.filter.columns to
"l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity" in
tblproperties before inserting the data), so the runtime IN-list filter
can have a better filter rate.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set). So we disable it at row level.

TODO: fix tests due to plan changes.

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M tests/query_test/test_runtime_filters.py
36 files changed, 857 insertions(+), 160 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/5
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 5
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 8:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/18141/8/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/8/be/src/exec/hdfs-orc-scanner.cc@318
PS8, Line 318:       ADD_COUNTER(scan_node_->runtime_profile(), "NumPushedDownRuntimeFilters", TUnit::UNIT);
line too long (93 > 90)


http://gerrit.cloudera.org:8080/#/c/18141/8/tests/query_test/test_runtime_filters.py
File tests/query_test/test_runtime_filters.py:

http://gerrit.cloudera.org:8080/#/c/18141/8/tests/query_test/test_runtime_filters.py@70
PS8, Line 70: [
flake8: E131 continuation line unaligned for hanging indent



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 8
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Mon, 14 Feb 2022 23:41:52 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 8:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/10156/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 8
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 15 Feb 2022 00:03:09 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#9).

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated only for small build side (e.g. #rows <= 1024) of a
broadcast join. They will only be applied on ORC tables and be pushed
down to the ORC reader(i.e. ORC lib). To avoid exploding the IN-list,
if #rows of the build side exceeds the threshold (1024), we set the
filter to ALWAYS_TRUE. The threshold can be configured by a new query
option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Example query that will benefit from this patch:
  use tpch_orc_def;
  select count(*) from lineitem_bf join (
    select * from partsupp, part
    where ps_partkey = p_partkey and p_size = 15
      and p_type like '%BRASS' and ps_availqty < 10) v
  on l_partkey = ps_partkey and l_suppkey = ps_suppkey;

The inline-view populates a runtime IN-list filter with 4 items. Note that
we need to re-generate the lineitem table with bloom filter indexes enabled
(e.g. setting orc.bloom.filter.columns to
"l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity" in
tblproperties before inserting the data), so the runtime IN-list filter
can have a better filter rate.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set). So we disable it at row level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-xxxxx: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-xxxxx: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-xxxxx: Consider IN-list filters in partitioned joins.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Tests:
 - TODO: Test IN-list filter with NULLs
 - TODO: Test IN-list filter on complex exprs targets
 - TODO: Test IN-list filter on all types including DATE

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test
M testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
M testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
M testdata/workloads/functional-planner/queries/PlannerTest/bloom-filter-assignment.test
M testdata/workloads/functional-planner/queries/PlannerTest/card-inner-join.test
M testdata/workloads/functional-planner/queries/PlannerTest/card-multi-join.test
M testdata/workloads/functional-planner/queries/PlannerTest/card-outer-join.test
M testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
M testdata/workloads/functional-planner/queries/PlannerTest/conjunct-ordering.test
M testdata/workloads/functional-planner/queries/PlannerTest/constant-propagation.test
M testdata/workloads/functional-planner/queries/PlannerTest/convert-to-cnf.test
M testdata/workloads/functional-planner/queries/PlannerTest/ddl.test
M testdata/workloads/functional-planner/queries/PlannerTest/default-join-distr-mode-broadcast.test
M testdata/workloads/functional-planner/queries/PlannerTest/default-join-distr-mode-shuffle.test
M testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test
M testdata/workloads/functional-planner/queries/PlannerTest/disable-preaggregations.test
M testdata/workloads/functional-planner/queries/PlannerTest/disable-runtime-overlap-filter.test
M testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test
M testdata/workloads/functional-planner/queries/PlannerTest/distinct.test
M testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection-hdfs-num-rows-est-enabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
M testdata/workloads/functional-planner/queries/PlannerTest/hbase-no-key-est.test
M testdata/workloads/functional-planner/queries/PlannerTest/hbase.test
M testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test
M testdata/workloads/functional-planner/queries/PlannerTest/implicit-joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test
M testdata/workloads/functional-planner/queries/PlannerTest/insert-hdfs-writer-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/insert-sort-by-zorder.test
M testdata/workloads/functional-planner/queries/PlannerTest/insert.test
M testdata/workloads/functional-planner/queries/PlannerTest/join-order.test
M testdata/workloads/functional-planner/queries/PlannerTest/joins-hdfs-num-rows-est-enabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-delete.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-replica-selection-leader-only.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-update.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu-upsert.test
M testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
M testdata/workloads/functional-planner/queries/PlannerTest/limit-pushdown-analytic.test
M testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
M testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters-hdfs-num-rows-est-enabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
M testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
M testdata/workloads/functional-planner/queries/PlannerTest/multiple-distinct-materialization.test
M testdata/workloads/functional-planner/queries/PlannerTest/multiple-distinct-predicates.test
M testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test
M testdata/workloads/functional-planner/queries/PlannerTest/optimize-simple-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/order.test
M testdata/workloads/functional-planner/queries/PlannerTest/outer-joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/outer-to-inner-joins.test
M testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering-disabled.test
M testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
M testdata/workloads/functional-planner/queries/PlannerTest/parquet-stats-agg.test
M testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
M testdata/workloads/functional-planner/queries/PlannerTest/preagg-bytes-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-propagation.test
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
M testdata/workloads/functional-planner/queries/PlannerTest/scan-node-fs-scheme.test
M testdata/workloads/functional-planner/queries/PlannerTest/semi-join-distinct.test
M testdata/workloads/functional-planner/queries/PlannerTest/setoperation-rewrite.test
M testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
M testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
M testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test
M testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
M testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit-small.test
M testdata/workloads/functional-planner/queries/PlannerTest/topn-bytes-limit.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds-dist-method.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q02.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q06.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q09.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q22.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q28.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q38.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q50.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q51.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q59.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q62.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q65.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q72.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q76.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q77.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q78.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q79.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q82.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q83.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q86.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q87.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q88.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q89.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q90.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q92.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q93.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q96.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q97.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q98.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q99.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
M testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test
M testdata/workloads/functional-planner/queries/PlannerTest/union.test
M testdata/workloads/functional-planner/queries/PlannerTest/views.test
M testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test
M tests/query_test/test_runtime_filters.py
219 files changed, 7,404 insertions(+), 6,651 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/9
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 9
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] WIP IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Qifan Chen (Code Review)" <ge...@cloudera.org>.
Qifan Chen has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: WIP IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 8:

(8 comments)

Replied to and added some more. 

Can you please also point out the explain output with in-list filters? Love to see them. 

It is unfortunate that there are massive number of filter Ids changes due to the introduction of the in-list type. I think some day we should re-assign the Ids at the end of compilation so that they are consecutive.

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1221
PS4, Line 1221: 
> Sorry that I'm not quite understand these.
 > 
 > > I was originally thinking that when the target of a IN-list
 > filter is partition columns, then the target can be removed in FE.
 > > Doing the test here means such targets are retained in the plan
 > and do not contribute.
 > 
 > Do you mean eliminating the partitions in FE? The IN-list filters
 > are generated in runtime based on the build side data of hash
 > joins. I'm afraid we are unable to eliminate them in the plan.
 > Instead, we will eliminate them in runtime in the code link you
 > pasted, ie. HdfsScanNodeBase::PartitionPassesFilters(). Did I miss
 > something?
 > 
 > > Personally, I feel we should allow the target to be a partition
 > column in this patch to pick up good performance gain, especially
 > for large tables with hundreds of partitions. The code to deal with
 > partition column is here: https://github.com/apache/impala/blob/master/be/src/exec/hdfs-scan-node-base.cc#L922.
 > Seems your code will work out of box in this situation if line
 > @1221 is removed.
 > 
 > UpdateSearchArgumentWithFilters() is only used in the orc scanner
 > to push down filters into the ORC lib. We need line 1221 since
 > partition columns don't exist in the ORC files.
 > 
 > The logics of HdfsScanNodeBase::PartitionPassesFilters() still
 > apply on IN-list filters. I don't see it skip using IN-list
 > filters. So we already support it that filtering out unrelated
 > partitions by the IN-list filters. Or did I miss something?

Okay. I think you are right. The line at 1221 is a protection for not applying the filter on the data files. Sorry I missed that one.


http://gerrit.cloudera.org:8080/#/c/18141/4/be/src/exec/hdfs-orc-scanner.cc@1271
PS4, Line 1271: ataType predicate_type
> > Calling PrepareSearchArguments() for each ORC stripe may be an overkill.
It seems to me starting filtering without waiting for the merge version to arrive can produce incorrect/non-deterministic results. For example, assume values [1, 2, 10] in the first stripe, and the merged filter is [1, 2].  If a partial filter [2] arrives and is applied, then [1, 10] will be eliminated. However [1] is the answer. 

Since all filter predicates are conjunctive, it is okay to use a subset of it, which may reduce the filtering efficiency. But the result is still correct. Each filter must be the merged version though.


http://gerrit.cloudera.org:8080/#/c/18141/8/common/thrift/ImpalaService.thrift
File common/thrift/ImpalaService.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/8/common/thrift/ImpalaService.thrift@725
PS8, Line 725: RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT
nit. IN_LIST_FILTER_ENTRY_LIMIT?


http://gerrit.cloudera.org:8080/#/c/18141/8/common/thrift/Query.thrift
File common/thrift/Query.thrift:

http://gerrit.cloudera.org:8080/#/c/18141/8/common/thrift/Query.thrift@578
PS8, Line 578: runtime_in_list_filter_entry_limit
nit. in_list_filter_entry_limit?


http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
File fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java:

http://gerrit.cloudera.org:8080/#/c/18141/4/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@394
PS4, Line 394: r
> The above casting is handled in BE in the orc scanner, because the underlyi
Okay. Agree casting in BE is the right way to go if data types in orc file can be different from table schema. 

But doing a feasibility check here for the inner should be done for the reasons mentioned.


http://gerrit.cloudera.org:8080/#/c/18141/8/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
File fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java:

http://gerrit.cloudera.org:8080/#/c/18141/8/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java@689
PS8, Line 689: 8
I wonder if this can be improved a little bit, especially for int type, to save some spaces. 

It is impossible for a column in ORC data file to contain 8-byte integer while the column type is 4-byte int, right?


http://gerrit.cloudera.org:8080/#/c/18141/8/testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test
File testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test:

http://gerrit.cloudera.org:8080/#/c/18141/8/testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test@8
PS8, Line 8: 3.42K
Saw quite many changes on cardinality. Can you explain the reason?


http://gerrit.cloudera.org:8080/#/c/18141/8/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
File testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test:

http://gerrit.cloudera.org:8080/#/c/18141/8/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test@13
PS8, Line 13: HDFS
Is this change introduced in this patch?



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 8
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Wed, 16 Feb 2022 17:01:53 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Qifan Chen (Code Review)" <ge...@cloudera.org>.
Qifan Chen has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 21: Code-Review+2


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 21
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 01 Mar 2022 13:47:20 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Quanlong Huang has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 18:

(1 comment)

Thank Qifan for the detailed review!

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.cc
File be/src/exec/hdfs-orc-scanner.cc:

http://gerrit.cloudera.org:8080/#/c/18141/17/be/src/exec/hdfs-orc-scanner.cc@1486
PS17, Line 1486: if (in_list_filter->ContainsNull()) {
               :       // Add a null literal with type.
               :      
> It makes a lot of sense now :-).  Thanks!
Sure. Added a comment in patch set 18.



-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 18
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sat, 26 Feb 2022 01:22:37 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#20).

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated for the build side of a broadcast join. They will only be
applied on ORC tables and be pushed down to the ORC reader(i.e. ORC
lib). To avoid exploding the IN-list, if # of distinct values of the
build side exceeds a threshold (default to 1024), we set the filter to
ALWAYS_TRUE and clear its entry. The threshold can be configured by a
new query option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set) and the lack of codegen. So we disable it at row
level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-11140: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-11141: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-11142: Consider IN-list filters in partitioned joins.

Tests:
 - Test IN-list filter on string, date and all integer types
 - Test IN-list filter with NULL
 - Test IN-list filter on complex exprs targets

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
A fe/src/test/resources/hive-site-events-cleanup/hive-site.xml
A fe/src/test/resources/hive-site-ranger-auth/hive-log4j2.properties
A fe/src/test/resources/hive-site-ranger-auth/hive-site.xml
A fe/src/test/resources/hive-site-ranger-auth/ranger-hive-audit.xml
A fe/src/test/resources/hive-site-ranger-auth/ranger-hive-security.xml
A fe/src/test/resources/hive-site-without-hms/hive-site.xml
A fe/src/test/resources/hive-site_events_cleanup.xml
A fe/src/test/resources/hive-site_ranger_auth.xml
A fe/src/test/resources/hive-site_without_hms.xml
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
A testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
A tests/catalog_service/test_partition_catalog_updates.py
M tests/query_test/test_runtime_filters.py
54 files changed, 1,896 insertions(+), 215 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/20
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 20
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 20:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/10233/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 20
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sun, 27 Feb 2022 00:42:38 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Quanlong Huang has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 23:

> Patch Set 22: Verified-1
> 
> Build failed: https://jenkins.impala.io/job/gerrit-verify-dryrun/7890/

The failure is due to runtime_filter_wait_time_ms being reset to default in a test file. We should explicitly setting it for each query, or just setting it in the test vector.

Patch Set 23 fixes this by setting runtime_filter_wait_time_ms in the test vector instead of setting it inside the test file.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 23
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Wed, 02 Mar 2022 02:05:43 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 21:

Build started: https://jenkins.impala.io/job/gerrit-verify-dryrun/7885/ DRY_RUN=true


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 21
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Sun, 27 Feb 2022 13:29:35 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Quanlong Huang (Code Review)" <ge...@cloudera.org>.
Hello Qifan Chen, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/18141

to look at the new patch set (#12).

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................

IMPALA-10898: Add runtime IN-list filters for ORC tables

ORC files have optional bloom filter indexes for each column. Since
ORC-1.7.0, the C++ reader supports pushing down predicates to skip
unreleated RowGroups. The pushed down predicates will be evaludated on
file indexes (i.e. statistics and bloom filter indexes). Note that only
EQUALS and IN-list predicates can leverage bloom filter indexes.

Currently Impala has two kinds of runtime filters: bloom filter and
min-max filter. Unfortunately they can't be converted into EQUALS or
IN-list predicates. So they can't leverage the file level bloom filter
indexes.

This patch adds runtime IN-list filters for this purpose. Currently they
are generated only for small build side (e.g. #rows <= 1024) of a
broadcast join. They will only be applied on ORC tables and be pushed
down to the ORC reader(i.e. ORC lib). To avoid exploding the IN-list,
if #rows of the build side exceeds the threshold (1024), we set the
filter to ALWAYS_TRUE and clear its entry. The threshold can be
configured by a new query option, RUNTIME_IN_LIST_FILTER_ENTRY_LIMIT.

Evaluating runtime IN-list filters is much slower than evaluating
runtime bloom filters due to the current simple implementation (i.e.
std::unorder_set) and the lack of codegen. So we disable it at row
level.

For visibility, this patch addes two counters in the HdfsScanNode:
 - NumPushedDownPredicates
 - NumPushedDownRuntimeFilters
They reflect the predicates and runtime filters that are pushed down to
the ORC reader.

Currently, runtime IN-list filters are disabled by default. This patch
extends the query option, ENABLED_RUNTIME_FILTER_TYPES, to support a
comma separated list of filter types. It defaults to be "BLOOM,MIN_MAX".
Add "IN_LIST" in it to enable runtime IN-list filters.

Ran perf tests on a 3 instances cluster on my desktop using TPC-DS with
scale factor 20. It shows significant improvements in some queries:

+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| Workload  | Query       | File Format        | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval   |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
| TPCDS(20) | TPCDS-Q67A  | orc / snap / block | 35.07  | 44.01       | I -20.32%  |   0.38%    |   1.38%        | 10    | I -25.69%      | -3.58   | -45.33 |
| TPCDS(20) | TPCDS-Q37   | orc / snap / block | 1.08   | 1.45        | I -25.23%  |   7.14%    |   3.09%        | 10    | I -34.09%      | -3.58   | -12.94 |
| TPCDS(20) | TPCDS-Q70A  | orc / snap / block | 6.30   | 8.60        | I -26.81%  |   5.24%    |   4.21%        | 10    | I -36.67%      | -3.58   | -14.88 |
| TPCDS(20) | TPCDS-Q16   | orc / snap / block | 1.33   | 1.85        | I -28.28%  |   4.98%    |   5.92%        | 10    | I -39.38%      | -3.58   | -12.93 |
| TPCDS(20) | TPCDS-Q18A  | orc / snap / block | 5.70   | 8.06        | I -29.25%  |   3.00%    |   4.12%        | 10    | I -40.30%      | -3.58   | -19.95 |
| TPCDS(20) | TPCDS-Q22A  | orc / snap / block | 2.01   | 2.97        | I -32.21%  |   6.12%    |   5.94%        | 10    | I -47.68%      | -3.58   | -14.05 |
| TPCDS(20) | TPCDS-Q77A  | orc / snap / block | 8.49   | 12.44       | I -31.75%  |   6.44%    |   3.96%        | 10    | I -49.71%      | -3.58   | -16.97 |
| TPCDS(20) | TPCDS-Q75   | orc / snap / block | 7.76   | 12.27       | I -36.76%  |   5.01%    |   3.87%        | 10    | I -59.56%      | -3.58   | -23.26 |
| TPCDS(20) | TPCDS-Q21   | orc / snap / block | 0.71   | 1.27        | I -44.26%  |   4.56%    |   4.24%        | 10    | I -77.31%      | -3.58   | -28.31 |
| TPCDS(20) | TPCDS-Q80A  | orc / snap / block | 9.24   | 20.42       | I -54.77%  |   4.03%    |   3.82%        | 10    | I -123.12%     | -3.58   | -40.90 |
| TPCDS(20) | TPCDS-Q39-1 | orc / snap / block | 1.07   | 2.26        | I -52.74%  | * 23.83% * |   2.60%        | 10    | I -149.68%     | -3.58   | -14.43 |
| TPCDS(20) | TPCDS-Q39-2 | orc / snap / block | 1.00   | 2.33        | I -56.95%  | * 19.53% * |   2.07%        | 10    | I -151.89%     | -3.58   | -20.81 |
+-----------+-------------+--------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
"Base Avg" is the avg of the original time. "Avg" is the current time.

However, we also see some regressions due to the suboptimal
implementation. The follow-up JIRAs will focus on improvements:
 - IMPALA-11140: Codegen InListFilter::Insert() and InListFilter::Find()
 - IMPALA-11141: Use exact data types in IN-list filters instead of
   casting data to a set of int64_t or a set of string.
 - IMPALA-11142: Consider IN-list filters in partitioned joins.

Tests:
 - Test IN-list filter on string, date and all integer types
 - Test IN-list filter with NULL
 - Test IN-list filter on complex exprs targets

Change-Id: I25080628233799aa0b6be18d5a832f1385414501
---
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/impala-ir.cc
M be/src/exec/filter-context.cc
M be/src/exec/filter-context.h
M be/src/exec/hdfs-orc-scanner.cc
M be/src/exec/hdfs-orc-scanner.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/join-builder.cc
M be/src/exec/nested-loop-join-builder.h
M be/src/exec/orc-metadata-utils.cc
M be/src/exec/partitioned-hash-join-builder.cc
M be/src/exec/partitioned-hash-join-builder.h
M be/src/exec/scan-node.cc
M be/src/runtime/coordinator-filter-state.h
M be/src/runtime/coordinator.cc
M be/src/runtime/runtime-filter-bank.cc
M be/src/runtime/runtime-filter-bank.h
M be/src/runtime/runtime-filter-ir.cc
M be/src/runtime/runtime-filter-test.cc
M be/src/runtime/runtime-filter.cc
M be/src/runtime/runtime-filter.h
M be/src/runtime/runtime-filter.inline.h
M be/src/service/child-query.cc
M be/src/service/data-stream-service.cc
M be/src/service/query-options-test.cc
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/util/CMakeLists.txt
M be/src/util/debug-util.cc
M be/src/util/debug-util.h
A be/src/util/in-list-filter-ir.cc
A be/src/util/in-list-filter.cc
A be/src/util/in-list-filter.h
M common/protobuf/data_stream_service.proto
M common/thrift/ImpalaService.thrift
M common/thrift/PlanNodes.thrift
M common/thrift/Query.thrift
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/test/java/org/apache/impala/planner/PlannerTest.java
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
A testdata/workloads/functional-query/queries/QueryTest/in_list_filters.test
M tests/query_test/test_runtime_filters.py
44 files changed, 1,268 insertions(+), 213 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/41/18141/12
-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 12
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>

[Impala-ASF-CR] IMPALA-10898: Add runtime IN-list filters for ORC tables

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/18141 )

Change subject: IMPALA-10898: Add runtime IN-list filters for ORC tables
......................................................................


Patch Set 13:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/10193/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/18141
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I25080628233799aa0b6be18d5a832f1385414501
Gerrit-Change-Number: 18141
Gerrit-PatchSet: 13
Gerrit-Owner: Quanlong Huang <hu...@gmail.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Qifan Chen <qc...@cloudera.com>
Gerrit-Reviewer: Quanlong Huang <hu...@gmail.com>
Gerrit-Comment-Date: Tue, 22 Feb 2022 01:30:07 +0000
Gerrit-HasComments: No