Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* 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.beam.sdk.extensions.sql.zetasql;

import org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalCalc;

/** {@link ConverterRule} to replace {@link Calc} with {@link BeamCalcRel}. */
public class BeamJavaUdfCalcRule extends ConverterRule {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: You can drop the convert method if you extend BeamCalcRule instead...

public static final BeamJavaUdfCalcRule INSTANCE = new BeamJavaUdfCalcRule();

private BeamJavaUdfCalcRule() {
super(
LogicalCalc.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamJavaUdfCalcRule");
}

@Override
public boolean matches(RelOptRuleCall x) {
return ZetaSQLQueryPlanner.hasUdfInProjects(x);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is insufficient and likely introduces data corruption bugs if UDFs and regular operators are mixed. I plan to review this pull request on Monday.

}

@Override
public RelNode convert(RelNode rel) {
final Calc calc = (Calc) rel;
final RelNode input = calc.getInput();

return new BeamCalcRel(
calc.getCluster(),
calc.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
RelOptRule.convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
calc.getProgram());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ private BeamZetaSqlCalcRule() {

@Override
public boolean matches(RelOptRuleCall x) {
return true;
return !ZetaSQLQueryPlanner.hasUdfInProjects(x);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,9 @@ public class SqlAnalyzer {
*/
public static final String ZETASQL_FUNCTION_GROUP_NAME = "ZetaSQL";

public static final String USER_DEFINED_JAVA_SCALAR_FUNCTIONS =
"user_defined_java_scalar_functions";

private static final ImmutableSet<ResolvedNodeKind> SUPPORTED_STATEMENT_KINDS =
ImmutableSet.of(
RESOLVED_QUERY_STMT, RESOLVED_CREATE_FUNCTION_STMT, RESOLVED_CREATE_TABLE_FUNCTION_STMT);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,41 +36,53 @@
import org.apache.beam.sdk.extensions.sql.impl.rule.BeamCalcRule;
import org.apache.beam.sdk.extensions.sql.impl.rule.BeamUncollectRule;
import org.apache.beam.sdk.extensions.sql.impl.rule.BeamUnnestRule;
import org.apache.beam.sdk.extensions.sql.zetasql.translation.ZetaSqlScalarFunctionImpl;
import org.apache.beam.sdk.extensions.sql.zetasql.unnest.BeamZetaSqlUncollectRule;
import org.apache.beam.sdk.extensions.sql.zetasql.unnest.BeamZetaSqlUnnestRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionConfig;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.ConventionTraitDef;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptUtil;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitDef;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.prepare.CalciteCatalogReader;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelRoot;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalCalc;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterCalcMergeRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.JoinCommuteRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectCalcMergeRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperatorTable;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParser;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserImplFactory;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.util.ChainedSqlOperatorTable;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.FrameworkConfig;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Frameworks;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSets;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/** ZetaSQLQueryPlanner. */
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class ZetaSQLQueryPlanner implements QueryPlanner {
private static final Logger LOG = LoggerFactory.getLogger(ZetaSQLQueryPlanner.class);

private final ZetaSQLPlannerImpl plannerImpl;

public ZetaSQLQueryPlanner(FrameworkConfig config) {
Expand Down Expand Up @@ -104,6 +116,30 @@ public static Collection<RuleSet> getZetaSqlRuleSets() {
return modifyRuleSetsForZetaSql(BeamRuleSets.getRuleSets());
}

/** Returns true if the argument contains any user-defined Java functions. */
static boolean hasUdfInProjects(RelOptRuleCall x) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I would probably stick this method in BeamZetaSqlCalcRule. What you have here are operators not supported by ZetaSqlCalc.

List<RelNode> resList = x.getRelList();
for (RelNode relNode : resList) {
if (relNode instanceof LogicalCalc) {
LogicalCalc logicalCalc = (LogicalCalc) relNode;
for (RexNode rexNode : logicalCalc.getProgram().getExprList()) {
if (rexNode instanceof RexCall) {
RexCall call = (RexCall) rexNode;
if (call.getOperator() instanceof SqlUserDefinedFunction) {
SqlUserDefinedFunction udf = (SqlUserDefinedFunction) call.op;
if (udf.function instanceof ZetaSqlScalarFunctionImpl) {
ZetaSqlScalarFunctionImpl scalarFunction = (ZetaSqlScalarFunctionImpl) udf.function;
return scalarFunction.functionGroup.equals(
SqlAnalyzer.USER_DEFINED_JAVA_SCALAR_FUNCTIONS);
}
}
}
}
}
}
return false;
}

private static Collection<RuleSet> modifyRuleSetsForZetaSql(Collection<RuleSet> ruleSets) {
ImmutableList.Builder<RuleSet> ret = ImmutableList.builder();
for (RuleSet ruleSet : ruleSets) {
Expand All @@ -123,6 +159,7 @@ private static Collection<RuleSet> modifyRuleSetsForZetaSql(Collection<RuleSet>
continue;
} else if (rule instanceof BeamCalcRule) {
bd.add(BeamZetaSqlCalcRule.INSTANCE);
bd.add(BeamJavaUdfCalcRule.INSTANCE);
} else if (rule instanceof BeamUnnestRule) {
bd.add(BeamZetaSqlUnnestRule.INSTANCE);
} else if (rule instanceof BeamUncollectRule) {
Expand Down Expand Up @@ -196,7 +233,9 @@ private BeamRelNode convertToBeamRelInternal(String sql, QueryParameters queryPa
RelMetadataQuery.THREAD_PROVIDERS.set(
JaninoRelMetadataProvider.of(root.rel.getCluster().getMetadataProvider()));
root.rel.getCluster().invalidateMetadataQuery();
return (BeamRelNode) plannerImpl.transform(0, desiredTraits, root.rel);
BeamRelNode beamRelNode = (BeamRelNode) plannerImpl.transform(0, desiredTraits, root.rel);
LOG.info("BEAMPlan>\n" + RelOptUtil.toString(beamRelNode));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: do we need this LOG.info?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not really, but it may be useful someday. I actually left this in from your original PR (#12398).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I guess that was added for the purpose of debugging.

Keep this LOG.info is fine. I find that in CalciteQueryPlanner there is also a such LOG.info.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I guess LOG.debug would probably be better. I might change both planners in a separate PR.

return beamRelNode;
}

private static FrameworkConfig defaultConfig(
Expand Down