From 812149c75c699485518d30a703f9e08217369bb2 Mon Sep 17 00:00:00 2001 From: Andrew Pilloud Date: Tue, 2 Feb 2021 14:17:35 -0800 Subject: [PATCH] [BEAM-10925] Support selecting Calc for ZetaSQL testing --- .../sql/zetasql/ZetaSQLQueryPlanner.java | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java index 76b0f528e1f8..e4a13522d982 100644 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java +++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java @@ -81,6 +81,12 @@ "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) }) public class ZetaSQLQueryPlanner implements QueryPlanner { + public static final Collection DEFAULT_CALC = + ImmutableList.builder() + .add(BeamZetaSqlCalcRule.INSTANCE) + .add(BeamJavaUdfCalcRule.INSTANCE) + .build(); + private static final Logger LOG = LoggerFactory.getLogger(ZetaSQLQueryPlanner.class); private final ZetaSQLPlannerImpl plannerImpl; @@ -95,7 +101,8 @@ public ZetaSQLQueryPlanner(FrameworkConfig config) { */ public ZetaSQLQueryPlanner(JdbcConnection jdbcConnection, Collection ruleSets) { plannerImpl = - new ZetaSQLPlannerImpl(defaultConfig(jdbcConnection, modifyRuleSetsForZetaSql(ruleSets))); + new ZetaSQLPlannerImpl( + defaultConfig(jdbcConnection, modifyRuleSetsForZetaSql(ruleSets, DEFAULT_CALC))); setDefaultTimezone( jdbcConnection .getPipelineOptions() @@ -113,7 +120,11 @@ public QueryPlanner createPlanner( }; public static Collection getZetaSqlRuleSets() { - return modifyRuleSetsForZetaSql(BeamRuleSets.getRuleSets()); + return modifyRuleSetsForZetaSql(BeamRuleSets.getRuleSets(), DEFAULT_CALC); + } + + public static Collection getZetaSqlRuleSets(Collection calc) { + return modifyRuleSetsForZetaSql(BeamRuleSets.getRuleSets(), calc); } /** @@ -178,7 +189,8 @@ static boolean hasNoJavaUdfInProjects(RelOptRuleCall x) { return true; } - private static Collection modifyRuleSetsForZetaSql(Collection ruleSets) { + private static Collection modifyRuleSetsForZetaSql( + Collection ruleSets, Collection calc) { ImmutableList.Builder ret = ImmutableList.builder(); for (RuleSet ruleSet : ruleSets) { ImmutableList.Builder bd = ImmutableList.builder(); @@ -196,8 +208,7 @@ private static Collection modifyRuleSetsForZetaSql(Collection // planning result eventually. continue; } else if (rule instanceof BeamCalcRule) { - bd.add(BeamZetaSqlCalcRule.INSTANCE); - bd.add(BeamJavaUdfCalcRule.INSTANCE); + bd.addAll(calc); } else if (rule instanceof BeamUnnestRule) { bd.add(BeamZetaSqlUnnestRule.INSTANCE); } else if (rule instanceof BeamUncollectRule) {