From 53722713e72934109646a8ffa11121b611cf1067 Mon Sep 17 00:00:00 2001 From: jiefli_LinkedIn Date: Wed, 21 Aug 2024 11:17:51 -0400 Subject: [PATCH] Address comment --- .../VersionedSqlUserDefinedFunction.java | 24 ++++++++++ .../CoralToTrinoSqlCallConverter.java | 22 +--------- ...istryOperatorRenameSqlCallTransformer.java | 9 ---- ...istryOperatorRenameSqlCallTransformer.java | 44 ------------------- .../transformers/HiveUDFTransformer.java | 39 ++++++++++++++++ 5 files changed, 65 insertions(+), 73 deletions(-) delete mode 100644 coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/GenericCoralRegistryOperatorRenameSqlCallTransformer.java create mode 100644 coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/HiveUDFTransformer.java diff --git a/coral-hive/src/main/java/com/linkedin/coral/hive/hive2rel/functions/VersionedSqlUserDefinedFunction.java b/coral-hive/src/main/java/com/linkedin/coral/hive/hive2rel/functions/VersionedSqlUserDefinedFunction.java index e3fac8596..814fdc6dc 100644 --- a/coral-hive/src/main/java/com/linkedin/coral/hive/hive2rel/functions/VersionedSqlUserDefinedFunction.java +++ b/coral-hive/src/main/java/com/linkedin/coral/hive/hive2rel/functions/VersionedSqlUserDefinedFunction.java @@ -6,8 +6,10 @@ package com.linkedin.coral.hive.hive2rel.functions; import java.util.List; +import java.util.Map; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.schema.Function; @@ -23,6 +25,9 @@ import org.apache.calcite.sql.validate.SqlValidator; import org.apache.calcite.sql.validate.SqlValidatorScope; +import com.linkedin.coral.com.google.common.base.CaseFormat; +import com.linkedin.coral.com.google.common.base.Converter; + import static com.linkedin.coral.hive.hive2rel.functions.utils.FunctionUtils.*; @@ -30,6 +35,15 @@ * Class that represents Dali versioned UDFs */ public class VersionedSqlUserDefinedFunction extends SqlUserDefinedFunction { + private static final Map TRINO_FUNC_NAME_MAP = ImmutableMap. builder() + .put("com.linkedin.dali.udf.watbotcrawlerlookup.hive.WATBotCrawlerLookup", "wat_bot_crawler_lookup") + .put("com.linkedin.stdudfs.parsing.hive.Ip2Str", "ip2str") + .put("com.linkedin.stdudfs.parsing.hive.UserAgentParser", "useragentparser") + .put("com.linkedin.stdudfs.lookup.hive.BrowserLookup", "browserlookup") + .put("com.linkedin.jobs.udf.hive.ConvertIndustryCode", "converttoindustryv1") + .put("com.linkedin.stdudfs.urnextractor.hive.UrnExtractorFunctionWrapper", "urn_extractor") + .put("com.linkedin.stdudfs.hive.daliudfs.UrnExtractorFunctionWrapper", "urn_extractor") + .put("com.linkedin.coral.hive.hive2rel.CoralTestUDF", "coral_test").build(); // The list of dependencies specified found in the view's "dependencies" property. // Example: "ivy://com.linkedin.udf-group:udf-artifact:0.1.8" @@ -70,6 +84,16 @@ public String getViewDependentFunctionName() { return viewDependentFunctionName; } + public String getTrinoFunctionName() { + final String unversionedClassName = removeVersioningPrefix(getName()); + if (TRINO_FUNC_NAME_MAP.containsKey(unversionedClassName)) { + return TRINO_FUNC_NAME_MAP.get(unversionedClassName); + } + Converter caseConverter = CaseFormat.UPPER_CAMEL.converterTo(CaseFormat.LOWER_UNDERSCORE); + String[] nameSplit = unversionedClassName.split("\\."); + return caseConverter.convert(nameSplit[nameSplit.length - 1]); + } + // This method is called during SQL validation. The super-class implementation resets the call's sqlOperator to one // that is looked up from the StaticHiveFunctionRegistry or inferred dynamically if it's a Dali UDF. Since UDFs in the StaticHiveFunctionRegistry are not // versioned, this method overrides the super-class implementation to properly restore the call's operator as diff --git a/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/CoralToTrinoSqlCallConverter.java b/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/CoralToTrinoSqlCallConverter.java index 0949c72a4..ca2cde7f6 100644 --- a/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/CoralToTrinoSqlCallConverter.java +++ b/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/CoralToTrinoSqlCallConverter.java @@ -28,7 +28,7 @@ import com.linkedin.coral.trino.rel2trino.transformers.CoralRegistryOperatorRenameSqlCallTransformer; import com.linkedin.coral.trino.rel2trino.transformers.CurrentTimestampTransformer; import com.linkedin.coral.trino.rel2trino.transformers.FromUnixtimeOperatorTransformer; -import com.linkedin.coral.trino.rel2trino.transformers.GenericCoralRegistryOperatorRenameSqlCallTransformer; +import com.linkedin.coral.trino.rel2trino.transformers.HiveUDFTransformer; import com.linkedin.coral.trino.rel2trino.transformers.JoinSqlCallTransformer; import com.linkedin.coral.trino.rel2trino.transformers.MapValueConstructorTransformer; import com.linkedin.coral.trino.rel2trino.transformers.NullOrderingTransformer; @@ -107,25 +107,7 @@ protected SqlCall transform(SqlCall sqlCall) { new ToDateOperatorTransformer(configs.getOrDefault(AVOID_TRANSFORM_TO_DATE_UDF, false)), new CurrentTimestampTransformer(), new FromUnixtimeOperatorTransformer(), - // LinkedIn specific functions - new CoralRegistryOperatorRenameSqlCallTransformer( - "com.linkedin.dali.udf.watbotcrawlerlookup.hive.WATBotCrawlerLookup", 3, "wat_bot_crawler_lookup"), - new CoralRegistryOperatorRenameSqlCallTransformer("com.linkedin.stdudfs.parsing.hive.Ip2Str", 1, "ip2str"), - new CoralRegistryOperatorRenameSqlCallTransformer("com.linkedin.stdudfs.parsing.hive.Ip2Str", 3, "ip2str"), - new CoralRegistryOperatorRenameSqlCallTransformer("com.linkedin.stdudfs.parsing.hive.UserAgentParser", 2, - "useragentparser"), - new CoralRegistryOperatorRenameSqlCallTransformer("com.linkedin.stdudfs.lookup.hive.BrowserLookup", 3, - "browserlookup"), - new CoralRegistryOperatorRenameSqlCallTransformer("com.linkedin.jobs.udf.hive.ConvertIndustryCode", 1, - "converttoindustryv1"), - new CoralRegistryOperatorRenameSqlCallTransformer( - "com.linkedin.stdudfs.urnextractor.hive.UrnExtractorFunctionWrapper", 1, "urn_extractor"), - new CoralRegistryOperatorRenameSqlCallTransformer( - "com.linkedin.stdudfs.hive.daliudfs.UrnExtractorFunctionWrapper", 1, "urn_extractor"), - // For coral-trino unit test - new CoralRegistryOperatorRenameSqlCallTransformer("com.linkedin.coral.hive.hive2rel.CoralTestUDF", 1, - "coral_test"), - new GenericCoralRegistryOperatorRenameSqlCallTransformer(), + new HiveUDFTransformer(), new ReturnTypeAdjustmentTransformer(configs), new UnnestOperatorTransformer(), new AsOperatorTransformer(), new JoinSqlCallTransformer(), new NullOrderingTransformer(), new SubstrIndexTransformer()); diff --git a/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/CoralRegistryOperatorRenameSqlCallTransformer.java b/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/CoralRegistryOperatorRenameSqlCallTransformer.java index d314285d1..562e823ef 100644 --- a/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/CoralRegistryOperatorRenameSqlCallTransformer.java +++ b/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/CoralRegistryOperatorRenameSqlCallTransformer.java @@ -5,11 +5,8 @@ */ package com.linkedin.coral.trino.rel2trino.transformers; -import org.apache.calcite.sql.SqlCall; - import com.linkedin.coral.common.transformers.OperatorRenameSqlCallTransformer; import com.linkedin.coral.hive.hive2rel.functions.StaticHiveFunctionRegistry; -import com.linkedin.coral.hive.hive2rel.functions.utils.FunctionUtils; /** @@ -19,12 +16,6 @@ public class CoralRegistryOperatorRenameSqlCallTransformer extends OperatorRenameSqlCallTransformer { private static final StaticHiveFunctionRegistry HIVE_FUNCTION_REGISTRY = new StaticHiveFunctionRegistry(); - @Override - protected boolean condition(SqlCall sqlCall) { - return sourceOpName.equalsIgnoreCase(FunctionUtils.removeVersioningPrefix(sqlCall.getOperator().getName())) - && sqlCall.getOperandList().size() == numOperands; - } - public CoralRegistryOperatorRenameSqlCallTransformer(String sourceOpName, int numOperands, String targetOpName) { super(HIVE_FUNCTION_REGISTRY.lookup(sourceOpName).iterator().next().getSqlOperator(), numOperands, targetOpName); } diff --git a/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/GenericCoralRegistryOperatorRenameSqlCallTransformer.java b/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/GenericCoralRegistryOperatorRenameSqlCallTransformer.java deleted file mode 100644 index b32611f4c..000000000 --- a/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/GenericCoralRegistryOperatorRenameSqlCallTransformer.java +++ /dev/null @@ -1,44 +0,0 @@ -/** - * Copyright 2023-2024 LinkedIn Corporation. All rights reserved. - * Licensed under the BSD-2 Clause license. - * See LICENSE in the project root for license information. - */ -package com.linkedin.coral.trino.rel2trino.transformers; - -import org.apache.calcite.sql.SqlCall; -import org.apache.calcite.sql.SqlNodeList; -import org.apache.calcite.sql.SqlOperator; -import org.apache.calcite.sql.parser.SqlParserPos; - -import com.linkedin.coral.com.google.common.base.CaseFormat; -import com.linkedin.coral.com.google.common.base.Converter; -import com.linkedin.coral.common.transformers.SqlCallTransformer; -import com.linkedin.coral.hive.hive2rel.functions.StaticHiveFunctionRegistry; - - -/** - * This is a subclass of {@link SqlCallTransformer}. It transforms any LinkedIn specific Coral operator - * to Trino operator which are not handled by {@link CoralRegistryOperatorRenameSqlCallTransformer}. - * e.g. from "com.linkedin.dali.udf.IsTestMemberId"("id") to "is_test_member_id("id") - */ -public class GenericCoralRegistryOperatorRenameSqlCallTransformer extends SqlCallTransformer { - - private static final StaticHiveFunctionRegistry HIVE_FUNCTION_REGISTRY = new StaticHiveFunctionRegistry(); - - @Override - protected boolean condition(SqlCall sqlCall) { - return sqlCall.getOperator().getName().contains("com.linkedin"); - } - - @Override - protected SqlCall transform(SqlCall sqlCall) { - Converter caseConverter = CaseFormat.UPPER_CAMEL.converterTo(CaseFormat.LOWER_UNDERSCORE); - SqlOperator sourceOp = HIVE_FUNCTION_REGISTRY.getRegistry().containsKey(sqlCall.getOperator().getName()) - ? HIVE_FUNCTION_REGISTRY.lookup(sqlCall.getOperator().getName()).iterator().next().getSqlOperator() - : sqlCall.getOperator(); - String[] nameSplit = sourceOp.getName().split("\\."); - String targetName = caseConverter.convert(nameSplit[nameSplit.length - 1]); - SqlOperator targetOp = createSqlOperator(targetName, sourceOp.getReturnTypeInference()); - return targetOp.createCall(new SqlNodeList(sqlCall.getOperandList(), SqlParserPos.ZERO)); - } -} diff --git a/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/HiveUDFTransformer.java b/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/HiveUDFTransformer.java new file mode 100644 index 000000000..1b49b8001 --- /dev/null +++ b/coral-trino/src/main/java/com/linkedin/coral/trino/rel2trino/transformers/HiveUDFTransformer.java @@ -0,0 +1,39 @@ +/** + * Copyright 2023-2024 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.coral.trino.rel2trino.transformers; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.parser.SqlParserPos; + +import com.linkedin.coral.common.transformers.SqlCallTransformer; +import com.linkedin.coral.hive.hive2rel.functions.VersionedSqlUserDefinedFunction; + + +/** + * This transformer converts the Hive UDF SqlCall name from the UDF class name to the + * corresponding Trino function name. + * i.e. from `com.linkedin.stdudfs.parsing.hive.Ip2Str` to `ip2str`. + */ +public class HiveUDFTransformer extends SqlCallTransformer { + + @Override + protected boolean condition(SqlCall sqlCall) { + final SqlOperator operator = sqlCall.getOperator(); + final String operatorName = operator.getName(); + return operator instanceof VersionedSqlUserDefinedFunction && operatorName.contains(".") + && !operatorName.equals("."); + } + + @Override + protected SqlCall transform(SqlCall sqlCall) { + final SqlOperator operator = sqlCall.getOperator(); + final String trinoFunctionName = ((VersionedSqlUserDefinedFunction) operator).getTrinoFunctionName(); + return createSqlOperator(trinoFunctionName, operator.getReturnTypeInference()) + .createCall(new SqlNodeList(sqlCall.getOperandList(), SqlParserPos.ZERO)); + } +}