Skip to content

Commit

Permalink
HIVE-22256: Rewriting fails when IN clause has items in different o…
Browse files Browse the repository at this point in the history
…rder in MV and query (Jesus Camacho Rodriguez, reviewed by Vineet Garg)

Close apache#1002
  • Loading branch information
jcamachor committed May 4, 2020
1 parent c75412f commit 5a119b9
Show file tree
Hide file tree
Showing 8 changed files with 610 additions and 38 deletions.
1 change: 1 addition & 0 deletions itests/src/test/resources/testconfiguration.properties
Original file line number Diff line number Diff line change
Expand Up @@ -658,6 +658,7 @@ minillaplocal.query.files=\
materialized_view_rewrite_8.q,\
materialized_view_rewrite_9.q,\
materialized_view_rewrite_10.q,\
materialized_view_rewrite_in_between.q,\
materialized_view_rewrite_no_join_opt.q,\
materialized_view_rewrite_no_join_opt_2.q,\
materialized_view_rewrite_part_1.q,\
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,198 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hive.ql.optimizer.calcite.rules;

import java.util.ArrayList;
import java.util.List;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.core.Filter;
import org.apache.calcite.rel.core.Join;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.rex.RexShuttle;
import org.apache.calcite.rex.RexUtil;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
import org.apache.hadoop.hive.ql.optimizer.calcite.translator.RexNodeConverter;

/**
* This class contains rules to rewrite IN/BETWEEN clauses into their
* corresponding AND/OR versions.
* It is the counterpart to {@link HivePointLookupOptimizerRule}.
*/
public class HiveInBetweenExpandRule {

public static final FilterRule FILTER_INSTANCE = new FilterRule();
public static final JoinRule JOIN_INSTANCE = new JoinRule();
public static final ProjectRule PROJECT_INSTANCE = new ProjectRule();

/** Rule adapter to apply the transformation to Filter conditions. */
private static class FilterRule extends RelOptRule {

FilterRule() {
super(operand(Filter.class, any()), HiveRelFactories.HIVE_BUILDER, null);
}

@Override
public void onMatch(RelOptRuleCall call) {
final Filter filter = call.rel(0);
RexInBetweenExpander expander = new RexInBetweenExpander(
filter.getCluster().getRexBuilder());
RexNode condition = expander.apply(filter.getCondition());

if (!expander.modified) {
return;
}

RelNode newFilter = filter.copy(filter.getTraitSet(),
filter.getInput(), condition);

call.transformTo(newFilter);
}
}

/** Rule adapter to apply the transformation to Join conditions. */
private static class JoinRule extends RelOptRule {

JoinRule() {
super(operand(Join.class, any()), HiveRelFactories.HIVE_BUILDER, null);
}

@Override
public void onMatch(RelOptRuleCall call) {
final Join join = call.rel(0);
RexInBetweenExpander expander = new RexInBetweenExpander(
join.getCluster().getRexBuilder());
RexNode condition = expander.apply(join.getCondition());

if (!expander.modified) {
return;
}

RelNode newJoin = join.copy(join.getTraitSet(),
condition,
join.getLeft(),
join.getRight(),
join.getJoinType(),
join.isSemiJoinDone());

call.transformTo(newJoin);
}
}

/** Rule adapter to apply the transformation to Project expressions. */
private static class ProjectRule extends RelOptRule {

ProjectRule() {
super(operand(Project.class, any()), HiveRelFactories.HIVE_BUILDER, null);
}

@Override
public void onMatch(RelOptRuleCall call) {
final Project project = call.rel(0);
RexInBetweenExpander expander = new RexInBetweenExpander(
project.getCluster().getRexBuilder());
List<RexNode> newProjects = new ArrayList<>();
for (RexNode expr : project.getProjects()) {
newProjects.add(expander.apply(expr));
}

if (!expander.modified) {
return;
}

Project newProject = project.copy(project.getTraitSet(),
project.getInput(), newProjects, project.getRowType());

call.transformTo(newProject);
}
}


/**
* Class that transforms IN/BETWEEN clauses in an expression.
* If any call is modified, the modified flag will be set to
* true after its execution.
*/
private static final class RexInBetweenExpander extends RexShuttle {

private final RexBuilder rexBuilder;
private boolean modified;

private RexInBetweenExpander(RexBuilder rexBuilder) {
this.rexBuilder = rexBuilder;
this.modified = false;
}

@Override
public RexNode visitCall(final RexCall call) {
switch (call.getKind()) {
case AND: {
boolean[] update = {false};
List<RexNode> newOperands = visitList(call.operands, update);
if (update[0]) {
return RexUtil.composeConjunction(rexBuilder, newOperands);
}
return call;
}
case OR: {
boolean[] update = {false};
List<RexNode> newOperands = visitList(call.operands, update);
if (update[0]) {
return RexUtil.composeDisjunction(rexBuilder, newOperands);
}
return call;
}
case IN: {
List<RexNode> newOperands = RexNodeConverter.transformInToOrOperands(
call.getOperands(), rexBuilder);
if (newOperands == null) {
// We could not execute transformation, return expression
return call;
}
modified = true;
if (newOperands.size() > 1) {
return rexBuilder.makeCall(SqlStdOperatorTable.OR, newOperands);
}
return newOperands.get(0);
}
case BETWEEN: {
List<RexNode> newOperands = RexNodeConverter.rewriteBetweenChildren(
call.getOperands(), rexBuilder);
modified = true;
if (call.getOperands().get(0).isAlwaysTrue()) {
return rexBuilder.makeCall(SqlStdOperatorTable.OR, newOperands);
}
return rexBuilder.makeCall(SqlStdOperatorTable.AND, newOperands);
}
default:
return super.visitCall(call);
}
}

}

private HiveInBetweenExpandRule() {
// Utility class, defeat instantiation
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -277,21 +277,12 @@ private RexNode convert(ExprNodeGenericFuncDesc func) throws SemanticException {
childRexNodeLst = rewriteToDateChildren(childRexNodeLst, rexBuilder);
} else if (calciteOp.getKind() == SqlKind.BETWEEN) {
assert childRexNodeLst.get(0).isAlwaysTrue() || childRexNodeLst.get(0).isAlwaysFalse();
boolean invert = childRexNodeLst.get(0).isAlwaysTrue();
SqlBinaryOperator cmpOp;
if (invert) {
childRexNodeLst = rewriteBetweenChildren(childRexNodeLst, rexBuilder);
if (childRexNodeLst.get(0).isAlwaysTrue()) {
calciteOp = SqlStdOperatorTable.OR;
cmpOp = SqlStdOperatorTable.GREATER_THAN;
} else {
calciteOp = SqlStdOperatorTable.AND;
cmpOp = SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
}
RexNode op = childRexNodeLst.get(1);
RexNode rangeL = childRexNodeLst.get(2);
RexNode rangeH = childRexNodeLst.get(3);
childRexNodeLst.clear();
childRexNodeLst.add(rexBuilder.makeCall(cmpOp, rangeL, op));
childRexNodeLst.add(rexBuilder.makeCall(cmpOp, op, rangeH));
}
expr = rexBuilder.makeCall(retType, calciteOp, childRexNodeLst);
} else {
Expand Down Expand Up @@ -606,6 +597,23 @@ public static List<RexNode> rewriteCoalesceChildren(
return convertedChildList;
}

public static List<RexNode> rewriteBetweenChildren(List<RexNode> childRexNodeLst,
RexBuilder rexBuilder) {
final List<RexNode> convertedChildList = Lists.newArrayList();
SqlBinaryOperator cmpOp;
if (childRexNodeLst.get(0).isAlwaysTrue()) {
cmpOp = SqlStdOperatorTable.GREATER_THAN;
} else {
cmpOp = SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
}
RexNode op = childRexNodeLst.get(1);
RexNode rangeL = childRexNodeLst.get(2);
RexNode rangeH = childRexNodeLst.get(3);
convertedChildList.add(rexBuilder.makeCall(cmpOp, rangeL, op));
convertedChildList.add(rexBuilder.makeCall(cmpOp, op, rangeH));
return convertedChildList;
}

private static boolean checkForStatefulFunctions(List<ExprNodeDesc> list) {
for (ExprNodeDesc node : list) {
if (node instanceof ExprNodeGenericFuncDesc) {
Expand Down
25 changes: 22 additions & 3 deletions ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
Original file line number Diff line number Diff line change
Expand Up @@ -212,6 +212,7 @@
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterSetOpTransposeRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterSortPredicates;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterSortTransposeRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveInBetweenExpandRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveInsertExchange4JoinRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveIntersectMergeRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveIntersectRewriteRule;
Expand Down Expand Up @@ -2151,7 +2152,7 @@ public RelOptMaterialization apply(RelOptMaterialization materialization) {
// There is a Project on top (due to nullability)
final Project pq = (Project) viewScan;
newViewScan = HiveProject.create(optCluster, copyNodeScan(pq.getInput()),
pq.getChildExps(), pq.getRowType(), Collections.<RelCollation> emptyList());
pq.getChildExps(), pq.getRowType(), Collections.emptyList());
} else {
newViewScan = copyNodeScan(viewScan);
}
Expand Down Expand Up @@ -2192,10 +2193,19 @@ private RelNode copyNodeScan(RelNode scan) {

perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);

// We need to expand IN/BETWEEN expressions when materialized view rewriting
// is triggered since otherwise this may prevent some rewritings from happening
HepProgramBuilder program = new HepProgramBuilder();
generatePartialProgram(program, false, HepMatchOrder.DEPTH_FIRST,
HiveInBetweenExpandRule.FILTER_INSTANCE,
HiveInBetweenExpandRule.JOIN_INSTANCE,
HiveInBetweenExpandRule.PROJECT_INSTANCE);
basePlan = executeProgram(basePlan, program.build(), mdProvider, executorProvider);

if (mvRebuild) {
// If it is a materialized view rebuild, we use the HepPlanner, since we only have
// one MV and we would like to use it to create incremental maintenance plans
final HepProgramBuilder program = new HepProgramBuilder();
program = new HepProgramBuilder();
generatePartialProgram(program, true, HepMatchOrder.TOP_DOWN,
HiveMaterializedViewRule.MATERIALIZED_VIEW_REWRITING_RULES);
// Add materialization for rebuild to planner
Expand Down Expand Up @@ -2253,7 +2263,7 @@ private RelNode copyNodeScan(RelNode scan) {
visitor.go(basePlan);
if (visitor.isRewritingAllowed()) {
// Trigger rewriting to remove UNION branch with MV
final HepProgramBuilder program = new HepProgramBuilder();
program = new HepProgramBuilder();
if (visitor.isContainsAggregate()) {
generatePartialProgram(program, false, HepMatchOrder.DEPTH_FIRST,
HiveAggregateIncrementalRewritingRule.INSTANCE);
Expand Down Expand Up @@ -2463,6 +2473,15 @@ private RelNode applyPostJoinOrderingTransform(RelNode basePlan, RelMetadataProv
HiveProjectSortExchangeTransposeRule.INSTANCE, HiveProjectMergeRule.INSTANCE);
}

// 10. We need to expand IN/BETWEEN expressions when loading a materialized view
// since otherwise this may prevent some rewritings from happening
if (ctx.isLoadingMaterializedView()) {
generatePartialProgram(program, false, HepMatchOrder.DEPTH_FIRST,
HiveInBetweenExpandRule.FILTER_INSTANCE,
HiveInBetweenExpandRule.JOIN_INSTANCE,
HiveInBetweenExpandRule.PROJECT_INSTANCE);
}

// Trigger program
perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
basePlan = executeProgram(basePlan, program.build(), mdProvider, executorProvider);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
SET hive.cli.errors.ignore=true;
SET hive.support.concurrency=true;
SET hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
SET metastore.strict.managed.tables=true;
SET hive.default.fileformat=textfile;
SET hive.default.fileformat.managed=orc;
SET metastore.create.as.acid=true;
SET hive.groupby.position.alias=true;

drop database if exists expr2 cascade;
create database expr2;
use expr2;
create table sales(prod_id int, cust_id int, store_id int, sale_date timestamp, qty int, amt double, descr string);
insert into sales values
(11,1,101,'12/24/2013',1000,1234.00,'onedummytwo');

create materialized view mv1 stored as orc as (select prod_id, cust_id, store_id, sale_date, qty, amt, descr from sales where cust_id in (1,2,3,4,5));
-- SAME ORDER
explain cbo
select prod_id, cust_id from sales where cust_id in (1,2,3,4,5);
-- DIFFERENT ORDER
explain cbo
select prod_id, cust_id from sales where cust_id in (5,1,2,3,4);

drop materialized view mv1;

drop database if exists in_pred cascade;
create database in_pred;
use in_pred;
create table census_pop (state string, year int, population bigint);
insert into census_pop values("AZ", 2010, 200), ("CA", 2011, 100), ("CA", 2010, 200), ("AZ", 2010, 100), ("NY", 2011, 121), ("AZ", 2011, 1000), ("OR", 2015, 1001), ("WA", 2016, 121), ("NJ", 2010, 500), ("NJ", 2010, 5000), ("AZ", 2014, 1004), ("TX", 2010, 1000), ("AZ", 2010, 1000), ("PT", 2017, 1200), ("NM", 2018, 120), ("CA", 2010, 200);

create materialized view mv2 stored as orc as select state, year, sum(population) from census_pop where year IN (2010, 2018) group by state, year;
-- SAME
explain cbo
select state, year, sum(population) from census_pop where year IN (2010, 2018) group by state, year;
-- PARTIAL IN EQUALS
explain cbo
select state, year, sum(population) from census_pop where year = 2010 group by state, year;
-- PARTIAL
explain cbo
select state, year, sum(population) from census_pop where year in (2010) group by state, year;

drop materialized view mv2;

drop database if exists expr9 cascade;
create database expr9;
use expr9;
create table sales(prod_id int, cust_id int, store_id int, sale_date timestamp, qty int, amt double, descr string);
insert into sales values
(11,1,101,'12/24/2013',1000,1234.00,'onedummytwo');

create materialized view mv3 stored as orc as (select prod_id, cust_id, store_id, sale_date, qty, amt, descr from sales where cust_id >= 1 and prod_id < 31);
-- SAME
explain cbo
select * from sales where cust_id >= 1 and prod_id < 31;
-- BETWEEN AND RANGE
explain cbo
select * from sales where cust_id between 1 and 20 and prod_id < 31;

drop materialized view mv3;
Loading

0 comments on commit 5a119b9

Please sign in to comment.