amaliujia commented on a change in pull request #13200: URL: https://github.com/apache/beam/pull/13200#discussion_r512943846
########## File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java ########## @@ -115,6 +136,51 @@ static boolean isEndOfInput(ParseResumeLocation parseResumeLocation) { return tables.build(); } + /** Returns the fully qualified name of the function defined in {@code statement}. */ + static String getFunctionQualifiedName(ResolvedCreateFunctionStmt createFunctionStmt) { + return String.format( + "%s:%s", + getFunctionGroup(createFunctionStmt), String.join(".", createFunctionStmt.getNamePath())); + } + + private static String getFunctionGroup(ResolvedCreateFunctionStmt createFunctionStmt) { + switch (createFunctionStmt.getLanguage().toUpperCase()) { + case "JAVA": + return createFunctionStmt.getIsAggregate() + ? USER_DEFINED_JAVA_AGGREGATE_FUNCTIONS + : USER_DEFINED_JAVA_SCALAR_FUNCTIONS; + case "SQL": + if (createFunctionStmt.getIsAggregate()) { + throw new UnsupportedOperationException( + "Native SQL aggregate functions are not supported (BEAM-9954)."); + } + return USER_DEFINED_FUNCTIONS; + case "PY": + case "PYTHON": + case "JS": + case "JAVASCRIPT": Review comment: Nice to have these in place! ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java ########## @@ -82,8 +80,6 @@ // Rules for window functions ProjectToWindowRule.PROJECT, // Rules so we only have to implement Calc - FilterCalcMergeRule.INSTANCE, - ProjectCalcMergeRule.INSTANCE, Review comment: https://github.com/apache/beam/pull/12400/files I have done this part in ZetaSQL planner so you don't need to disable such rules in the default rulesets ########## File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java ########## @@ -97,22 +102,43 @@ public RelRoot rel(String sql, QueryParameters params) { SimpleCatalog catalog = analyzer.createPopulatedCatalog(defaultSchemaPlus.getName(), options, tables); - ImmutableMap.Builder<String, ResolvedCreateFunctionStmt> udfBuilder = ImmutableMap.builder(); + ImmutableMap.Builder<List<String>, ResolvedCreateFunctionStmt> udfBuilder = + ImmutableMap.builder(); ImmutableMap.Builder<List<String>, ResolvedNode> udtvfBuilder = ImmutableMap.builder(); + ImmutableMap.Builder<List<String>, Method> javaScalarFunctionBuilder = ImmutableMap.builder(); + ImmutableMap.Builder<List<String>, Combine.CombineFn> javaAggregateFunctionBuilder = + ImmutableMap.builder(); + JavaUdfLoader javaUdfLoader = new JavaUdfLoader(); ResolvedStatement statement; ParseResumeLocation parseResumeLocation = new ParseResumeLocation(sql); do { statement = analyzer.analyzeNextStatement(parseResumeLocation, options, catalog); if (statement.nodeKind() == RESOLVED_CREATE_FUNCTION_STMT) { ResolvedCreateFunctionStmt createFunctionStmt = (ResolvedCreateFunctionStmt) statement; - // ResolvedCreateFunctionStmt does not include the full function name, so build it here. - String functionFullName = - String.format( - "%s:%s", - SqlAnalyzer.USER_DEFINED_FUNCTIONS, - String.join(".", createFunctionStmt.getNamePath())); - udfBuilder.put(functionFullName, createFunctionStmt); + if (createFunctionStmt.getLanguage().toUpperCase().equals("SQL")) { + udfBuilder.put(createFunctionStmt.getNamePath(), createFunctionStmt); + } else if (createFunctionStmt.getLanguage().toUpperCase().equals("JAVA")) { Review comment: Will it better to replace `"JAVA"` with a static constant or use a helper function `isJavaUDF(createFunctionStmt.getLanguage())`? ########## File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java ########## @@ -97,22 +102,43 @@ public RelRoot rel(String sql, QueryParameters params) { SimpleCatalog catalog = analyzer.createPopulatedCatalog(defaultSchemaPlus.getName(), options, tables); - ImmutableMap.Builder<String, ResolvedCreateFunctionStmt> udfBuilder = ImmutableMap.builder(); + ImmutableMap.Builder<List<String>, ResolvedCreateFunctionStmt> udfBuilder = + ImmutableMap.builder(); ImmutableMap.Builder<List<String>, ResolvedNode> udtvfBuilder = ImmutableMap.builder(); + ImmutableMap.Builder<List<String>, Method> javaScalarFunctionBuilder = ImmutableMap.builder(); + ImmutableMap.Builder<List<String>, Combine.CombineFn> javaAggregateFunctionBuilder = + ImmutableMap.builder(); + JavaUdfLoader javaUdfLoader = new JavaUdfLoader(); ResolvedStatement statement; ParseResumeLocation parseResumeLocation = new ParseResumeLocation(sql); do { statement = analyzer.analyzeNextStatement(parseResumeLocation, options, catalog); if (statement.nodeKind() == RESOLVED_CREATE_FUNCTION_STMT) { ResolvedCreateFunctionStmt createFunctionStmt = (ResolvedCreateFunctionStmt) statement; - // ResolvedCreateFunctionStmt does not include the full function name, so build it here. - String functionFullName = - String.format( - "%s:%s", - SqlAnalyzer.USER_DEFINED_FUNCTIONS, - String.join(".", createFunctionStmt.getNamePath())); - udfBuilder.put(functionFullName, createFunctionStmt); + if (createFunctionStmt.getLanguage().toUpperCase().equals("SQL")) { Review comment: same for `"SQL"` ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java ########## @@ -82,8 +80,6 @@ // Rules for window functions ProjectToWindowRule.PROJECT, // Rules so we only have to implement Calc - FilterCalcMergeRule.INSTANCE, - ProjectCalcMergeRule.INSTANCE, Review comment: Meanwhile, one rule is required for ZetaSQL planner is, we probably need a `BeamZetaSQLCalcMergeRule`, which does not merge Calc when one Calc has Java UDF but another Calc does not. This will be useful when users write query like ``` select java_udf(x) from (select builtin_function(y) as x from table) as t; ``` ########## File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java ########## @@ -184,7 +190,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)); Review comment: Remove this LOG.info before merge the code? ########## File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/JavaUdfLoader.java ########## @@ -0,0 +1,217 @@ +/* + * 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.translation; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Method; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.file.ProviderNotFoundException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.ServiceLoader; +import org.apache.beam.sdk.extensions.sql.UdfProvider; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Loads {@link UdfProvider} implementations from user-provided jars. + * + * <p>All UDFs are loaded and cached for each jar to mitigate IO costs. + */ +public class JavaUdfLoader { + private static final Logger LOG = LoggerFactory.getLogger(JavaUdfLoader.class); + + /** + * Maps the external jar location to the functions the jar defines. Static so it can persist + * across multiple SQL transforms. + */ + private static final Map<String, UserFunctionDefinitions> cache = new HashMap<>(); + + private static ClassLoader originalClassLoader = null; + + /** + * Load a user-defined scalar function from the specified jar. + * + * <p><strong>WARNING</strong>: The first time a jar is loaded, it is added to the thread's + * context {@link ClassLoader} so that the jar can be staged by the runner. + */ + public Method loadScalarFunction(List<String> functionPath, String jarPath) { + String functionFullName = String.join(".", functionPath); + try { + UserFunctionDefinitions functionDefinitions = loadJar(jarPath); + if (!functionDefinitions.javaScalarFunctions.containsKey(functionPath)) { + throw new IllegalArgumentException( + String.format( + "No implementation of scalar function %s found in %s.\n" + + " 1. Create a class implementing %s and annotate it with @AutoService(%s.class).\n" + + " 2. Add function %s to the class's userDefinedScalarFunctions implementation.", + functionFullName, + jarPath, + UdfProvider.class.getSimpleName(), + UdfProvider.class.getSimpleName(), + functionFullName)); + } + return functionDefinitions.javaScalarFunctions.get(functionPath); + } catch (IOException e) { + throw new RuntimeException( + String.format( + "Failed to load user-defined scalar function %s from %s", functionFullName, jarPath), + e); + } + } + + /** + * Load a user-defined aggregate function from the specified jar. + * + * <p><strong>WARNING</strong>: The first time a jar is loaded, it is added to the thread's + * context {@link ClassLoader} so that the jar can be staged by the runner. + */ + public Combine.CombineFn loadAggregateFunction(List<String> functionPath, String jarPath) { + String functionFullName = String.join(".", functionPath); + try { + UserFunctionDefinitions functionDefinitions = loadJar(jarPath); + if (!functionDefinitions.javaAggregateFunctions.containsKey(functionPath)) { + throw new IllegalArgumentException( + String.format( + "No implementation of aggregate function %s found in %s.\n" + + " 1. Create a class implementing %s and annotate it with @AutoService(%s.class).\n" + + " 2. Add function %s to the class's userDefinedAggregateFunctions implementation.", + functionFullName, + jarPath, + UdfProvider.class.getSimpleName(), + UdfProvider.class.getSimpleName(), + functionFullName)); + } + return functionDefinitions.javaAggregateFunctions.get(functionPath); + } catch (IOException e) { + throw new RuntimeException( + String.format( + "Failed to load user-defined aggregate function %s from %s", + functionFullName, jarPath), + e); + } + } + + private ClassLoader createAndSetClassLoader(String inputJarPath) throws IOException { + Preconditions.checkArgument(!inputJarPath.isEmpty(), "Jar path cannot be empty."); + ResourceId inputJar = FileSystems.matchNewResource(inputJarPath, false /* is directory */); + File tmpJar = File.createTempFile("sql-udf-", inputJar.getFilename()); + FileSystems.copy( + Collections.singletonList(inputJar), + Collections.singletonList( + FileSystems.matchNewResource(tmpJar.getAbsolutePath(), false /* is directory */))); Review comment: Is it possible to have a verification after the copy is done? E.g. verify file size or MD5 hash? I am thinking if user query has an `function not found` issue due to a bad downloaded jar, we can locate the root cause based on log? Otherwise such issue might be hard to debug. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org