kasakrisz commented on code in PR #3588:
URL: https://github.com/apache/hive/pull/3588#discussion_r985743640
##########
ql/src/test/results/clientpositive/llap/subquery_ALL.q.out:
##########
@@ -413,8 +413,7 @@ POSTHOOK: Input: default@part
POSTHOOK: Input: default@part_null_n0
#### A masked pattern was here ####
26
-Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_1, $hdt$_2]] in Stage
'Reducer 3' is a cross product
-Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in
Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[22][tables = [$hdt$_0, $hdt$_1]] in Stage
'Reducer 2' is a cross product
Review Comment:
> Is there a branch here that is simplified to empty? Why?
This is the plan after subquery rewrite and decorrelation:
```
HiveProject(_o__c0=[$0])
HiveAggregate(group=[{}], agg#0=[count()])
HiveProject($f0=[$0])
HiveProject(p_partkey=[$0], p_name=[$1], p_mfgr=[$2], p_brand=[$3],
p_type=[$4], p_size=[$5], p_container=[$6], p_retailprice=[$7], p_comment=[$8],
BLOCK__OFFSET__INSIDE__FILE=[$9], INPUT__FILE__NAME=[$10], ROW__ID=[$11],
ROW__IS__DELETED=[$12])
HiveFilter(condition=[IS NULL(OR(AND(IS NOT NULL($16), <>($13, 0)),
AND(OR(IS NULL($0), <($14, $13)), null, <>($13, 0), IS NULL($16))))])
HiveJoin(condition=[=($0, $15)], joinType=[left],
algorithm=[none], cost=[not available])
HiveJoin(condition=[true], joinType=[inner], algorithm=[none],
cost=[not available])
HiveTableScan(table=[[default, part]], table:alias=[part])
HiveAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)])
HiveProject(p_partkey=[$0])
HiveFilter(condition=[IS NULL($0)])
HiveTableScan(table=[[default, part_null_n0]],
table:alias=[part_null_n0])
HiveAggregate(group=[{0, 1}])
HiveProject(p_partkey=[$0], literalTrue=[true])
HiveFilter(condition=[IS NULL($0)])
HiveTableScan(table=[[default, part_null_n0]],
table:alias=[part_null_n0])
```
`HiveJoin(condition=[=($0, $15)], joinType=[left]` is removed because
ReduceExpressionRule transforms the join condition always `false`.
It does because `$15` coming from the right branch is always `null` due to
`HiveFilter(condition=[IS NULL($0)])`
##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveValues.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.reloperators;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLiteral;
+
+import java.util.List;
+
+/**
+ * Subclass of {@link org.apache.calcite.rel.core.Values}.
+ * Targeting Hive engine.
+ */
+public class HiveValues extends Values {
+
+ public HiveValues(
+ RelOptCluster cluster,
+ RelDataType rowType,
+ ImmutableList<ImmutableList<RexLiteral>> tuples,
+ RelTraitSet traits) {
+ super(cluster, rowType, tuples, traits);
+ }
+
+ @Override
+ public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+ if (getInputs().equals(inputs) && traitSet.equals(getTraitSet())) {
+ return this;
+ }
+
Review Comment:
Changed this to always create new instance.
##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/Bug.java:
##########
@@ -74,4 +74,14 @@ public final class Bug {
* Whether <a
href="https://issues.apache.org/jira/browse/CALCITE-4704">CALCITE-4704</a> is
fixed.
*/
public static final boolean CALCITE_4704_FIXED = false;
+
+ /**
+ * Whether <a
href="https://issues.apache.org/jira/browse/CALCITE-4704">CALCITE-5293</a> is
fixed.
Review Comment:
fixed
##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/Bug.java:
##########
@@ -74,4 +74,14 @@ public final class Bug {
* Whether <a
href="https://issues.apache.org/jira/browse/CALCITE-4704">CALCITE-4704</a> is
fixed.
*/
public static final boolean CALCITE_4704_FIXED = false;
+
+ /**
+ * Whether <a
href="https://issues.apache.org/jira/browse/CALCITE-4704">CALCITE-5293</a> is
fixed.
+ */
+ public static final boolean CALCITE_5293_FIXED = false;
+
+ /**
+ * Whether <a
href="https://issues.apache.org/jira/browse/CALCITE-4704">CALCITE-5294</a> is
fixed.
Review Comment:
fixed
##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRemoveEmptySingleRules.java:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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 org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.rules.PruneEmptyRules;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.hadoop.hive.ql.optimizer.calcite.Bug;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+
+import java.util.Collections;
+import java.util.List;
+
+import static com.google.common.collect.Iterables.concat;
+
+/**
+ * This class provides access to Calcite's {@link PruneEmptyRules}.
+ * The instances of the rules use {@link
org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelBuilder}.
+ */
+public class HiveRemoveEmptySingleRules extends PruneEmptyRules {
+
+ public static final RelOptRule PROJECT_INSTANCE =
+ RelRule.Config.EMPTY
+ .withDescription("HivePruneEmptyProject")
+ .as(PruneEmptyRules.RemoveEmptySingleRule.Config.class)
+ .withOperandFor(Project.class, project -> true)
Review Comment:
Yeah, it is not an issue if we end up with a plan which is a single
`HiveValues[tuple[]]`.
Finally I decided to reduce the scope of these rules to Hive operators. We
can extend the scope in follow-up patches if necessary.
##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java:
##########
@@ -121,7 +127,87 @@ public static ASTNode convert(final RelNode relNode,
List<FieldSchema> resultSch
return c.convert();
}
+ // TOK_QUERY
+ // TOK_INSERT
+ // TOK_DESTINATION
+ // TOK_DIR
+ // TOK_TMP_FILE
+ // TOK_SELECT
+ // TOK_SELEXPR
+ // TOK_FUNCTION
+ // TOK_<type>
+ // TOK_NULL
+ // alias0
+ // ...
+ // TOK_SELEXPR
+ // TOK_FUNCTION
+ // TOK_<type>
+ // TOK_NULL
+ // aliasn
+ // TOK_LIMIT
+ // 0
+ // 0
+ public static ASTNode emptyPlan(RelDataType dataType) {
+ if (dataType.getFieldCount() == 0) {
+ throw new IllegalArgumentException("Schema is empty.");
+ }
+
+ ASTBuilder select = ASTBuilder.construct(HiveParser.TOK_SELECT,
"TOK_SELECT");
+ for (int i = 0; i < dataType.getFieldCount(); ++i) {
+ RelDataTypeField fieldType = dataType.getFieldList().get(i);
+ if (fieldType.getValue().getSqlTypeName() == SqlTypeName.NULL) {
+ select.add(ASTBuilder.selectExpr(
+ ASTBuilder.construct(HiveParser.TOK_NULL, "TOK_NULL").node(),
+ fieldType.getName()));
+ } else {
+ ASTNode typeNode = createCast(fieldType);
+ select.add(ASTBuilder.selectExpr(
+ ASTBuilder.construct(HiveParser.TOK_FUNCTION, "TOK_FUNCTION")
+ .add(typeNode)
+ .add(ASTBuilder.construct(HiveParser.TOK_NULL,
"TOK_NULL").node()).node(),
+ fieldType.getName()));
+ }
+ }
+
+ ASTNode insert = ASTBuilder.
+ construct(HiveParser.TOK_INSERT, "TOK_INSERT").
+ add(ASTBuilder.destNode()).
+ add(select).
+ add(ASTBuilder.limit(0, 0)).
+ node();
+
+ return ASTBuilder.
+ construct(HiveParser.TOK_QUERY, "TOK_QUERY").
+ add(insert).
+ node();
+ }
+
+ private static ASTNode createCast(RelDataTypeField fieldType) {
+ HiveToken ht = TypeConverter.hiveToken(fieldType.getType());
+ ASTNode typeNode;
+ if (ht == null) {
+ typeNode = ASTBuilder.construct(
+ HiveParser.Identifier,
fieldType.getType().getSqlTypeName().getName().toLowerCase()).node();
+ } else {
+ ASTBuilder typeNodeBuilder = ASTBuilder.construct(ht.type, ht.text);
+ if (ht.args != null) {
+ for (String castArg : ht.args) {
+ typeNodeBuilder.add(HiveParser.Identifier, castArg);
+ }
+ }
+ typeNode = typeNodeBuilder.node();
+ }
+ return typeNode;
+ }
+
private ASTNode convert() throws CalciteSemanticException {
+ if (root instanceof HiveValues) {
+ HiveValues values = (HiveValues) root;
+ if (isEmpty(values)) {
+ select = values;
+ return emptyPlan(values.getRowType());
+ }
Review Comment:
added exception
##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java:
##########
@@ -121,7 +127,87 @@ public static ASTNode convert(final RelNode relNode,
List<FieldSchema> resultSch
return c.convert();
}
+ // TOK_QUERY
+ // TOK_INSERT
+ // TOK_DESTINATION
+ // TOK_DIR
+ // TOK_TMP_FILE
+ // TOK_SELECT
+ // TOK_SELEXPR
+ // TOK_FUNCTION
+ // TOK_<type>
+ // TOK_NULL
+ // alias0
+ // ...
+ // TOK_SELEXPR
+ // TOK_FUNCTION
+ // TOK_<type>
+ // TOK_NULL
+ // aliasn
+ // TOK_LIMIT
+ // 0
+ // 0
+ public static ASTNode emptyPlan(RelDataType dataType) {
+ if (dataType.getFieldCount() == 0) {
+ throw new IllegalArgumentException("Schema is empty.");
+ }
+
+ ASTBuilder select = ASTBuilder.construct(HiveParser.TOK_SELECT,
"TOK_SELECT");
+ for (int i = 0; i < dataType.getFieldCount(); ++i) {
+ RelDataTypeField fieldType = dataType.getFieldList().get(i);
+ if (fieldType.getValue().getSqlTypeName() == SqlTypeName.NULL) {
+ select.add(ASTBuilder.selectExpr(
+ ASTBuilder.construct(HiveParser.TOK_NULL, "TOK_NULL").node(),
+ fieldType.getName()));
+ } else {
+ ASTNode typeNode = createCast(fieldType);
+ select.add(ASTBuilder.selectExpr(
+ ASTBuilder.construct(HiveParser.TOK_FUNCTION, "TOK_FUNCTION")
+ .add(typeNode)
+ .add(ASTBuilder.construct(HiveParser.TOK_NULL,
"TOK_NULL").node()).node(),
+ fieldType.getName()));
+ }
+ }
+
+ ASTNode insert = ASTBuilder.
+ construct(HiveParser.TOK_INSERT, "TOK_INSERT").
+ add(ASTBuilder.destNode()).
+ add(select).
+ add(ASTBuilder.limit(0, 0)).
+ node();
+
+ return ASTBuilder.
+ construct(HiveParser.TOK_QUERY, "TOK_QUERY").
+ add(insert).
+ node();
+ }
+
+ private static ASTNode createCast(RelDataTypeField fieldType) {
Review Comment:
Improved this to handle complex types: struct, map.
Array is also supported: `array(NULL)`
##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveValuesVisitor.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.translator.opconventer;
+
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
+import org.apache.hadoop.hive.ql.exec.RowSchema;
+import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveValues;
+import org.apache.hadoop.hive.ql.optimizer.calcite.translator.TypeConverter;
+import
org.apache.hadoop.hive.ql.optimizer.calcite.translator.opconventer.HiveOpConverter.OpAttr;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.LimitDesc;
+import org.apache.hadoop.hive.ql.plan.SelectDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+class HiveValuesVisitor extends HiveRelNodeVisitor<HiveValues> {
+ HiveValuesVisitor(HiveOpConverter hiveOpConverter) {
+ super(hiveOpConverter);
+ }
+
+ @Override
+ OpAttr visit(HiveValues valuesRel) throws SemanticException {
+
+ LOG.debug("Translating operator rel#{}:{} with row type: [{}]",
+ valuesRel.getId(), valuesRel.getRelTypeName(),
valuesRel.getRowType());
+ LOG.debug("Operator rel#{}:{} has {} tuples.",
+ valuesRel.getId(), valuesRel.getRelTypeName(),
valuesRel.tuples.size());
Review Comment:
removed
##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveValuesVisitor.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.translator.opconventer;
+
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
+import org.apache.hadoop.hive.ql.exec.RowSchema;
+import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveValues;
+import org.apache.hadoop.hive.ql.optimizer.calcite.translator.TypeConverter;
+import
org.apache.hadoop.hive.ql.optimizer.calcite.translator.opconventer.HiveOpConverter.OpAttr;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.LimitDesc;
+import org.apache.hadoop.hive.ql.plan.SelectDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+class HiveValuesVisitor extends HiveRelNodeVisitor<HiveValues> {
+ HiveValuesVisitor(HiveOpConverter hiveOpConverter) {
+ super(hiveOpConverter);
+ }
+
+ @Override
+ OpAttr visit(HiveValues valuesRel) throws SemanticException {
+
+ LOG.debug("Translating operator rel#{}:{} with row type: [{}]",
+ valuesRel.getId(), valuesRel.getRelTypeName(),
valuesRel.getRowType());
+ LOG.debug("Operator rel#{}:{} has {} tuples.",
+ valuesRel.getId(), valuesRel.getRelTypeName(),
valuesRel.tuples.size());
+
+ if (!Values.isEmpty(valuesRel)) {
+ LOG.error("Empty {} operator translation not supported yet in return
path.",
+ valuesRel.getClass().getCanonicalName());
+ return null;
+ }
+
+ // 1. collect columns for project row schema
Review Comment:
Refactored:
* removed the redundant comments with enumerations. I agree that this is not
the clean code way to do things. Unfortunately I saw it too many times in Hive
code.
* Extracted the steps to methods.
* Added javadocs to the class
##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveValuesVisitor.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.translator.opconventer;
+
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
+import org.apache.hadoop.hive.ql.exec.RowSchema;
+import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveValues;
+import org.apache.hadoop.hive.ql.optimizer.calcite.translator.TypeConverter;
+import
org.apache.hadoop.hive.ql.optimizer.calcite.translator.opconventer.HiveOpConverter.OpAttr;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.LimitDesc;
+import org.apache.hadoop.hive.ql.plan.SelectDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+class HiveValuesVisitor extends HiveRelNodeVisitor<HiveValues> {
+ HiveValuesVisitor(HiveOpConverter hiveOpConverter) {
+ super(hiveOpConverter);
+ }
+
+ @Override
+ OpAttr visit(HiveValues valuesRel) throws SemanticException {
+
+ LOG.debug("Translating operator rel#{}:{} with row type: [{}]",
+ valuesRel.getId(), valuesRel.getRelTypeName(),
valuesRel.getRowType());
+ LOG.debug("Operator rel#{}:{} has {} tuples.",
+ valuesRel.getId(), valuesRel.getRelTypeName(),
valuesRel.tuples.size());
+
+ if (!Values.isEmpty(valuesRel)) {
+ LOG.error("Empty {} operator translation not supported yet in return
path.",
+ valuesRel.getClass().getCanonicalName());
+ return null;
+ }
+
+ // 1. collect columns for project row schema
+ List<String> columnNames = new ArrayList<>();
+ List<ExprNodeDesc> exprNodeDescList = new ArrayList<>();
+ Map<String, ExprNodeDesc> colExprMap = new HashMap<>();
+
+ List<ColumnInfo> colInfoList = new ArrayList<>();
+ for (int i = 0; i < valuesRel.getRowType().getFieldList().size(); i++) {
+ RelDataTypeField typeField =
valuesRel.getRowType().getFieldList().get(i);
+
+ ColumnInfo ci = new ColumnInfo(
+ typeField.getName(), TypeConverter.convert(typeField.getType()),
SemanticAnalyzer.DUMMY_TABLE, false);
+ colInfoList.add(ci);
+ columnNames.add(typeField.getName());
+
+ ExprNodeDesc exprNodeDesc = new
ExprNodeConstantDesc(TypeConverter.convert(typeField.getType()), null);
+ colExprMap.put(typeField.getName(), exprNodeDesc);
+ exprNodeDescList.add(exprNodeDesc);
+ }
+
+ // 2. Create TS on dummy table
+ Table metadata = hiveOpConverter.getSemanticAnalyzer().getDummyTable();
+ TableScanDesc tsd = new TableScanDesc(SemanticAnalyzer.DUMMY_TABLE,
Collections.emptyList(), metadata);
+
+ TableScanOperator ts = (TableScanOperator) OperatorFactory.get(
+ hiveOpConverter.getSemanticAnalyzer().getOpContext(), tsd, new
RowSchema(Collections.emptyList()));
+
+ hiveOpConverter.getTopOps().put(SemanticAnalyzer.DUMMY_TABLE, ts);
+
+ // 3. Create Select operator
+ SelectDesc sd = new SelectDesc(exprNodeDescList, columnNames);
+ SelectOperator selOp = (SelectOperator)
OperatorFactory.getAndMakeChild(sd, new RowSchema(colInfoList), ts);
+ selOp.setColumnExprMap(colExprMap);
+
+ // 4. Create Limit 0 operator
Review Comment:
removed
##########
ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TestASTConverter.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.translator;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.sql.type.ArraySqlType;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveTypeSystemImpl;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+import static java.util.Arrays.asList;
+import static
org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.emptyPlan;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+
+class TestASTConverter {
+ @Test
+ void testEmptyPlanWhenInputSchemaIsEmpty() {
+ RelRecordType dataType = new RelRecordType(Collections.emptyList());
+ IllegalArgumentException thrown =
Assertions.assertThrows(IllegalArgumentException.class, () -> {
+ emptyPlan(dataType);
+ });
+
+ Assertions.assertTrue(thrown.getMessage().contains("Schema is empty"));
+ }
+
+ @Test
+ void testEmptyPlan() {
+ List<RelDataTypeField> fields = asList(
+ new RelDataTypeFieldImpl("a", 0, new BasicSqlType(new
HiveTypeSystemImpl(), SqlTypeName.INTEGER)),
+ new RelDataTypeFieldImpl("b", 1, new BasicSqlType(new
HiveTypeSystemImpl(), SqlTypeName.CHAR, 30)),
+ new RelDataTypeFieldImpl("c", 2, new BasicSqlType(new
HiveTypeSystemImpl(), SqlTypeName.NULL)),
+ new RelDataTypeFieldImpl("d", 3, new ArraySqlType(new
BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER), false)),
+ new RelDataTypeFieldImpl("e", 4, new ArraySqlType(new
BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER), true)));
+ RelDataType dataType = new RelRecordType(fields);
+
+ ASTNode tree = emptyPlan(dataType);
+
+ // TOK_QUERY -> TOK_INSERT -> TOK_SELECT
+ assertThat(tree.getChild(0).getChild(1).getChildCount(),
is(fields.size()));
Review Comment:
removed
##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java:
##########
@@ -121,7 +127,87 @@ public static ASTNode convert(final RelNode relNode,
List<FieldSchema> resultSch
return c.convert();
}
+ // TOK_QUERY
+ // TOK_INSERT
+ // TOK_DESTINATION
+ // TOK_DIR
+ // TOK_TMP_FILE
+ // TOK_SELECT
+ // TOK_SELEXPR
+ // TOK_FUNCTION
+ // TOK_<type>
+ // TOK_NULL
+ // alias0
+ // ...
+ // TOK_SELEXPR
+ // TOK_FUNCTION
+ // TOK_<type>
+ // TOK_NULL
+ // aliasn
+ // TOK_LIMIT
+ // 0
+ // 0
+ public static ASTNode emptyPlan(RelDataType dataType) {
Review Comment:
Added javadocs
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]