Skip to content

Commit

Permalink
[Enhancement]Rewrite unnest(bitmap_to_array) to unnest_bitmap (#52870)
Browse files Browse the repository at this point in the history
Signed-off-by: before-Sunrise <[email protected]>
  • Loading branch information
before-Sunrise committed Nov 15, 2024
1 parent d66c09c commit af76d2b
Show file tree
Hide file tree
Showing 7 changed files with 141 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -488,6 +488,9 @@ public class FunctionSet {
public static final String SECONDS_SUB = "seconds_sub";
public static final String MILLISECONDS_ADD = "milliseconds_add";
public static final String MILLISECONDS_SUB = "milliseconds_sub";
// table function
public static final String UNNEST = "unnest";
public static final String UNNEST_BITMAP = "unnest_bitmap";

public static final String CONNECTION_ID = "connection_id";

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,8 +91,8 @@ public static void initBuiltins(FunctionSet functionSet) {
functionSet.addBuiltin(func);
}

TableFunction funcUnnestBitmap = new TableFunction(new FunctionName("unnest_bitmap"),
Lists.newArrayList("unnest_bitmap"), Lists.newArrayList(Type.BITMAP), Lists.newArrayList(Type.BIGINT));
TableFunction funcUnnestBitmap = new TableFunction(new FunctionName(FunctionSet.UNNEST_BITMAP),
Lists.newArrayList(FunctionSet.UNNEST_BITMAP), Lists.newArrayList(Type.BITMAP), Lists.newArrayList(Type.BIGINT));
functionSet.addBuiltin(funcUnnestBitmap);

for (Type type : Lists.newArrayList(Type.TINYINT, Type.SMALLINT, Type.INT, Type.BIGINT, Type.LARGEINT)) {
Expand Down
10 changes: 10 additions & 0 deletions fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java
Original file line number Diff line number Diff line change
Expand Up @@ -695,6 +695,8 @@ public static MaterializedViewRewriteMode parse(String str) {

public static final String DISABLE_GENERATED_COLUMN_REWRITE = "disable_generated_column_rewrite";

public static final String ENABLE_REWRITE_UNNEST_BITMAP_TO_ARRAY = "enable_rewrite_unnest_bitmap_to_array";

public static final List<String> DEPRECATED_VARIABLES = ImmutableList.<String>builder()
.add(CODEGEN_LEVEL)
.add(MAX_EXECUTION_TIME)
Expand Down Expand Up @@ -1920,6 +1922,10 @@ public Optional<Boolean> isFollowerForwardToLeaderOpt() {

@VarAttr(name = LIKE_PREDICATE_CONSOLIDATE_MIN)
private int likePredicateConsolidateMin = 2;

@VarAttr(name = ENABLE_REWRITE_UNNEST_BITMAP_TO_ARRAY)
private boolean enableRewriteUnnestBitmapToArray = true;

public int getExprChildrenLimit() {
return exprChildrenLimit;
}
Expand Down Expand Up @@ -3543,6 +3549,10 @@ public boolean isDisableGeneratedColumnRewrite() {
return disableGeneratedColumnRewrite;
}

public boolean isEnableRewriteUnnestBitmapToArray() {
return enableRewriteUnnestBitmapToArray;
}

// Serialize to thrift object
// used for rest api
public TQueryOptions toThrift() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@
import com.starrocks.sql.optimizer.rule.transformation.RewriteMultiDistinctRule;
import com.starrocks.sql.optimizer.rule.transformation.RewriteSimpleAggToHDFSScanRule;
import com.starrocks.sql.optimizer.rule.transformation.RewriteSimpleAggToMetaScanRule;
import com.starrocks.sql.optimizer.rule.transformation.RewriteUnnestBitmapRule;
import com.starrocks.sql.optimizer.rule.transformation.SeparateProjectRule;
import com.starrocks.sql.optimizer.rule.transformation.SkewJoinOptimizeRule;
import com.starrocks.sql.optimizer.rule.transformation.SplitDatePredicateRule;
Expand Down Expand Up @@ -534,6 +535,8 @@ private OptExpression logicalRuleRewrite(
ruleRewriteIterative(tree, rootTaskContext, new RewriteSimpleAggToMetaScanRule());
ruleRewriteOnlyOnce(tree, rootTaskContext, new MinMaxCountOptOnScanRule());
ruleRewriteOnlyOnce(tree, rootTaskContext, new PartitionColumnValueOnlyOnScanRule());
// before MergeProjectWithChildRule, after INLINE_CTE and MergeApplyWithTableFunction
ruleRewriteIterative(tree, rootTaskContext, RewriteUnnestBitmapRule.getInstance());

// After this rule, we shouldn't generate logical project operator
ruleRewriteIterative(tree, rootTaskContext, new MergeProjectWithChildRule());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,7 @@ public enum RuleType {
TF_REWRITE_PARTITION_COLUMN_ONLY_AGG,
TF_REWRITE_SUM_BY_ASSOCIATIVE_RULE,
TF_REWRITE_COUNT_IF_RULE,
TF_REWRITE_UNNEST_BITMAP_RULE,

TF_INTERSECT_REORDER,
TF_INTERSECT_DISTINCT,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
// Copyright 2021-present StarRocks, Inc. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package com.starrocks.sql.optimizer.rule.transformation;

import com.google.common.collect.Lists;
import com.starrocks.analysis.Expr;
import com.starrocks.catalog.Function;
import com.starrocks.catalog.FunctionSet;
import com.starrocks.catalog.TableFunction;
import com.starrocks.catalog.Type;
import com.starrocks.common.Pair;
import com.starrocks.sql.optimizer.OptExpression;
import com.starrocks.sql.optimizer.OptimizerContext;
import com.starrocks.sql.optimizer.operator.OperatorType;
import com.starrocks.sql.optimizer.operator.logical.LogicalProjectOperator;
import com.starrocks.sql.optimizer.operator.logical.LogicalTableFunctionOperator;
import com.starrocks.sql.optimizer.operator.pattern.Pattern;
import com.starrocks.sql.optimizer.operator.scalar.CallOperator;
import com.starrocks.sql.optimizer.operator.scalar.ColumnRefOperator;
import com.starrocks.sql.optimizer.operator.scalar.ScalarOperator;
import com.starrocks.sql.optimizer.rule.RuleType;

import java.util.List;
import java.util.Map;

public class RewriteUnnestBitmapRule extends TransformationRule {
private static final RewriteUnnestBitmapRule INSTANCE = new RewriteUnnestBitmapRule();

private RewriteUnnestBitmapRule() {
super(RuleType.TF_REWRITE_UNNEST_BITMAP_RULE, Pattern.create(OperatorType.LOGICAL_TABLE_FUNCTION)
.addChildren(Pattern.create(OperatorType.LOGICAL_PROJECT)));
}

public static RewriteUnnestBitmapRule getInstance() {
return INSTANCE;
}

@Override
public boolean check(OptExpression input, OptimizerContext context) {
if (!context.getSessionVariable().isEnableRewriteUnnestBitmapToArray()) {
return false;
}

LogicalTableFunctionOperator tableFunctionOperator = (LogicalTableFunctionOperator) input.getOp();
if (tableFunctionOperator.getFn() != null &&
!tableFunctionOperator.getFn().functionName().equals(FunctionSet.UNNEST)) {
return false;
}

LogicalProjectOperator projectOperator = (LogicalProjectOperator) input.inputAt(0).getOp();

boolean existBitmapToArray = projectOperator.getColumnRefMap().values().stream().filter(scalarOperator -> {
if (scalarOperator instanceof CallOperator) {
CallOperator callOperator = (CallOperator) scalarOperator;
return callOperator.getFnName().equals(FunctionSet.BITMAP_TO_ARRAY);
}
return false;
}).count() == 1;

return existBitmapToArray;
}

@Override
public List<OptExpression> transform(OptExpression input, OptimizerContext context) {
LogicalTableFunctionOperator originalTableFunctionOperator = (LogicalTableFunctionOperator) input.getOp();
LogicalProjectOperator projectOperator = (LogicalProjectOperator) input.inputAt(0).getOp();

Map<ColumnRefOperator, ScalarOperator> columnRefMap = projectOperator.getColumnRefMap();
Pair<ColumnRefOperator, ScalarOperator> bitmapToArray = columnRefMap.entrySet().stream().filter(entry -> {
if (entry.getValue() instanceof CallOperator) {
CallOperator callOperator = (CallOperator) entry.getValue();
return callOperator.getFnName().equals(FunctionSet.BITMAP_TO_ARRAY);
}
return false;
}).map(entry -> new Pair<>(entry.getKey(), entry.getValue())).findFirst().get();

// if bitmap_to_array's output will be used by upper nodes, it's not safe to rewrite
if (originalTableFunctionOperator.getOuterColRefs().contains(bitmapToArray.first)) {
return Lists.newArrayList();
}

ColumnRefOperator bitmapColumn = bitmapToArray.second.getColumnRefs().get(0);
columnRefMap.remove(bitmapToArray.first);
columnRefMap.putIfAbsent(bitmapColumn, bitmapColumn);

TableFunction unnestBitmapFn =
(TableFunction) Expr.getBuiltinFunction(FunctionSet.UNNEST_BITMAP, new Type[] {Type.BITMAP},
Function.CompareMode.IS_NONSTRICT_SUPERTYPE_OF);
List<Pair<ColumnRefOperator, ScalarOperator>> fnParamColumnProject =
Lists.newArrayList(Pair.create(bitmapColumn, bitmapColumn));

LogicalTableFunctionOperator newTableFunctionOperator =
new LogicalTableFunctionOperator(originalTableFunctionOperator.getFnResultColRefs(), unnestBitmapFn,
fnParamColumnProject, originalTableFunctionOperator.getOuterColRefs());

return Lists.newArrayList(OptExpression.create(newTableFunctionOperator, input.inputAt(0)));
}
}



Original file line number Diff line number Diff line change
Expand Up @@ -270,4 +270,14 @@ public void testTableFunctionAlias() throws Exception {

Assert.assertTrue(e.getMessage().contains("Not unique table/alias: 'table_function_unnest'"));
}

@Test
public void testRewrite() throws Exception {
String sql = "SELECT k1, unnest AS c3\n" +
" FROM test_agg,unnest(bitmap_to_array(b1)) ORDER BY k1 ASC, c3 ASC\n" +
"LIMIT 5;";
String plan = getFragmentPlan(sql);
assertContains(plan, "tableFunctionName: unnest_bitmap");
assertNotContains(plan, "bitmap_to_array");
}
}

0 comments on commit af76d2b

Please sign in to comment.