From 63ebcf0715ce8dea4e02a43713837b825945d297 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Thu, 19 Mar 2026 16:24:32 -0700 Subject: [PATCH 01/14] UDF support: CREATE FUNCTION DDL with pipeline SQL integration Add support for user-defined functions (UDFs) that can be registered via CREATE FUNCTION and referenced in SQL queries. Registered functions are included in pipeline SQL so Flink can execute them at runtime. DDL syntax: CREATE FUNCTION name [RETURNS type] AS 'class' [LANGUAGE lang] [WITH (...)] DROP FUNCTION name Phase 1 - JDBC driver + pipeline SQL: - UserFunction API model (Deployable) - OpaqueFunction: permissive ScalarFunction for Calcite validation with configurable return type (RETURNS clause) and ANY-typed parameters - Session-scoped function registry on HoptimatorConnection - CREATE/DROP FUNCTION handling in HoptimatorDdlExecutor - FunctionImplementor in ScriptImplementor generates CREATE FUNCTION DDL - PipelineRel.Implementor tracks functions and emits DDL before connectors - Parser extended with RETURNS and LANGUAGE clauses Phase 2 - Python code delivery: - Job API gains files field for inline code (e.g., Python UDF sources) - SqlJob CRD spec gains files field - FlinkStreamingSqlJob and reconciler pass files through - K8sJobDeployer exports files to template environment Tests: - ScriptImplementorTest: FunctionImplementor DDL generation - Quidem unit test (create-function-ddl.id): DDL parsing, type validation - Quidem integration test (k8s-ddl-udf.id): pipeline SQL with !specify Co-Authored-By: Claude Opus 4.6 (1M context) --- .../java/com/linkedin/hoptimator/Job.java | 12 ++ .../com/linkedin/hoptimator/UserFunction.java | 41 ++++++ .../catalog/flink/FlinkStreamingSqlJob.java | 9 ++ .../flink/FlinkStreamingSqlJobReconciler.java | 4 +- .../src/main/codegen/includes/parserImpls.ftl | 6 +- .../hoptimator/jdbc/HoptimatorConnection.java | 14 ++ .../jdbc/HoptimatorDdlExecutor.java | 64 +++++++++ .../hoptimator/jdbc/OpaqueFunction.java | 128 ++++++++++++++++++ .../jdbc/ddl/HoptimatorDdlParserImpl.java | 14 +- .../jdbc/ddl/SqlCreateFunction.java | 33 ++++- .../hoptimator/jdbc/TestSqlScripts.java | 5 + .../src/test/resources/create-function-ddl.id | 95 +++++++++++++ .../hoptimator/jdbc/QuidemTestBase.java | 5 + .../hoptimator/k8s/K8sJobDeployer.java | 1 + .../k8s/models/V1alpha1SqlJobSpec.java | 39 +++++- .../src/main/resources/sqljobs.crd.yaml | 5 + .../hoptimator/k8s/TestSqlScripts.java | 5 + .../src/test/resources/k8s-ddl-udf.id | 72 ++++++++++ .../hoptimator/util/planner/PipelineRel.java | 26 +++- .../util/planner/ScriptImplementor.java | 32 +++++ .../util/planner/ScriptImplementorTest.java | 64 +++++++++ 21 files changed, 665 insertions(+), 9 deletions(-) create mode 100644 hoptimator-api/src/main/java/com/linkedin/hoptimator/UserFunction.java create mode 100644 hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/OpaqueFunction.java create mode 100644 hoptimator-jdbc/src/test/resources/create-function-ddl.id create mode 100644 hoptimator-k8s/src/test/resources/k8s-ddl-udf.id diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Job.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Job.java index bdc2849e..4e87735a 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Job.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Job.java @@ -1,5 +1,6 @@ package com.linkedin.hoptimator; +import java.util.Collections; import java.util.Map; import java.util.Set; @@ -16,6 +17,7 @@ public class Job implements Deployable { private final String name; private final Set sources; private final Sink sink; + private final Map files; /** * Lazy-evaluated template functions that generate various outputs for the job. @@ -30,10 +32,16 @@ public class Job implements Deployable { private final Map> lazyEvals; public Job(String name, Set sources, Sink sink, Map> lazyEvals) { + this(name, sources, sink, lazyEvals, Collections.emptyMap()); + } + + public Job(String name, Set sources, Sink sink, Map> lazyEvals, + Map files) { this.name = name; this.sources = sources; this.sink = sink; this.lazyEvals = lazyEvals; + this.files = files != null ? files : Collections.emptyMap(); } public String name() { @@ -60,6 +68,10 @@ public ThrowingFunction fieldMap() { return eval("fieldMap"); } + public Map files() { + return files; + } + /** * Retrieves a lazy-evaluated template function by key. * diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/UserFunction.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/UserFunction.java new file mode 100644 index 00000000..9a51c330 --- /dev/null +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/UserFunction.java @@ -0,0 +1,41 @@ +package com.linkedin.hoptimator; + +import java.util.Collections; +import java.util.Map; + + +public class UserFunction implements Deployable { + + private final String name; + private final String as; + private final String namespace; + private final Map options; + + public UserFunction(String name, String as, String namespace, Map options) { + this.name = name; + this.as = as; + this.namespace = namespace; + this.options = options != null ? options : Collections.emptyMap(); + } + + public String name() { + return name; + } + + public String as() { + return as; + } + + public String namespace() { + return namespace; + } + + public Map options() { + return options; + } + + @Override + public String toString() { + return "UserFunction[" + name + "]"; + } +} diff --git a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java index e697631e..f32b0992 100644 --- a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java +++ b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java @@ -2,6 +2,8 @@ import com.linkedin.hoptimator.catalog.Resource; +import java.util.Map; + public class FlinkStreamingSqlJob extends Resource { @@ -11,4 +13,11 @@ public FlinkStreamingSqlJob(String namespace, String name, String sql) { export("name", name); export("sql", sql); } + + public FlinkStreamingSqlJob(String namespace, String name, String sql, Map files) { + this(namespace, name, sql); + if (files != null && !files.isEmpty()) { + export("files", files); + } + } } diff --git a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java index 7cf4f250..086e7290 100644 --- a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java +++ b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java @@ -16,6 +16,7 @@ import org.slf4j.LoggerFactory; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; @@ -70,7 +71,8 @@ public Result reconcile(Request request) { } Resource.TemplateFactory templateFactory = new Resource.SimpleTemplateFactory(Resource.Environment.EMPTY); - Resource sqlJob = new FlinkStreamingSqlJob(namespace, name, script); + Map files = object.getSpec().getFiles(); + Resource sqlJob = new FlinkStreamingSqlJob(namespace, name, script, files); boolean allReady = true; boolean anyFailed = false; for (String yaml : sqlJob.render(templateFactory)) { diff --git a/hoptimator-jdbc/src/main/codegen/includes/parserImpls.ftl b/hoptimator-jdbc/src/main/codegen/includes/parserImpls.ftl index 40c80ee3..db572cb0 100644 --- a/hoptimator-jdbc/src/main/codegen/includes/parserImpls.ftl +++ b/hoptimator-jdbc/src/main/codegen/includes/parserImpls.ftl @@ -333,17 +333,21 @@ SqlCreate SqlCreateFunction(Span s, boolean replace) : final SqlIdentifier id; final SqlNode template; SqlNode namespace = null; + SqlDataTypeSpec returnType = null; + SqlIdentifier language = null; SqlNodeList optionList = null; } { ifNotExists = IfNotExistsOpt() id = CompoundIdentifier() + [ returnType = DataType() ] template = StringLiteral() [ namespace = StringLiteral() ] + [ language = SimpleIdentifier() ] [ optionList = Options() ] { return new SqlCreateFunction(s.end(this), replace, ifNotExists, id, template, - namespace, optionList); + namespace, returnType, language, optionList); } } diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorConnection.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorConnection.java index 3cb26b4c..1d8472f9 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorConnection.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorConnection.java @@ -3,6 +3,7 @@ import com.linkedin.hoptimator.Database; import com.linkedin.hoptimator.Sink; import com.linkedin.hoptimator.Source; +import com.linkedin.hoptimator.UserFunction; import com.linkedin.hoptimator.avro.AvroConverter; import com.linkedin.hoptimator.util.ConnectionService; import com.linkedin.hoptimator.util.DelegatingConnection; @@ -32,6 +33,7 @@ public class HoptimatorConnection extends DelegatingConnection { private final CalciteConnection connection; private final Properties connectionProperties; private final List materializations = new ArrayList<>(); + private final List functions = new ArrayList<>(); private final List> logHooks = new ArrayList<>(); @@ -96,6 +98,18 @@ public HoptimatorConnection withProperties(Properties properties) { return new HoptimatorConnection(connection, properties); } + public void registerFunction(UserFunction func) { + functions.add(func); + } + + public void removeFunction(String name) { + functions.removeIf(f -> f.name().equals(name)); + } + + public List functions() { + return functions; + } + public void registerMaterialization(List viewPath, String querySql) { String tableSql = "SELECT * FROM " + viewPath.stream().map(x -> "\"" + x + "\"").collect(Collectors.joining(".")); RelNode tableRel = HoptimatorDriver.convert(this, tableSql).root.rel; diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java index 3025cbab..f53afc74 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java @@ -26,9 +26,11 @@ import com.linkedin.hoptimator.Pipeline; import com.linkedin.hoptimator.Source; import com.linkedin.hoptimator.Trigger; +import com.linkedin.hoptimator.UserFunction; import com.linkedin.hoptimator.UserJob; import com.linkedin.hoptimator.View; import com.linkedin.hoptimator.jdbc.ddl.HoptimatorDdlParserImpl; +import com.linkedin.hoptimator.jdbc.ddl.SqlCreateFunction; import com.linkedin.hoptimator.jdbc.ddl.SqlCreateMaterializedView; import com.linkedin.hoptimator.jdbc.ddl.SqlCreateTable; import com.linkedin.hoptimator.jdbc.ddl.SqlCreateTrigger; @@ -264,6 +266,10 @@ public void execute(SqlCreateMaterializedView create, CalcitePrepare.Context con // Plan a pipeline to materialize the view. RelRoot root = new HoptimatorDriver.Prepare(connection).convert(context, sql).root; PipelineRel.Implementor plan = DeploymentService.plan(root, connection.materializations(), connectionProperties); + // Pass registered functions to the implementor + for (UserFunction func : connection.functions()) { + plan.addFunction(func); + } schemaSnapshot = HoptimatorDdlUtils.snapshotAndSetSinkSchema(context, new HoptimatorDriver.Prepare(connection), plan, sql, pair); logger.info("Added materialized view {} to schema {}", viewName, schemaPlus.getName()); Pipeline pipeline = plan.pipeline(viewName, connection); @@ -366,6 +372,52 @@ public void execute(SqlCreateTrigger create, CalcitePrepare.Context context) { } } + /** Executes a {@code CREATE FUNCTION} command. */ + public void execute(SqlCreateFunction create, CalcitePrepare.Context context) { + logger.info("Validating statement: {}", create); + try { + ValidationService.validateOrThrow(create); + } catch (SQLException e) { + throw new DdlException(create, e.getMessage(), e); + } + + String name = create.name.names.get(create.name.names.size() - 1); + String as = ((SqlLiteral) create.job).getValueAs(String.class); + String namespace = create.namespace != null + ? ((SqlLiteral) create.namespace).getValueAs(String.class) : null; + Map options = HoptimatorDdlUtils.options(create.options); + + // Extract LANGUAGE from the dedicated clause (elevated from WITH options) + if (create.language != null) { + options.put("LANGUAGE", create.language.getSimple()); + } + + UserFunction userFunction = new UserFunction(name, as, namespace, options); + + // Determine return type from RETURNS clause (default: VARCHAR) + String returnsName = create.returnType != null + ? create.returnType.getTypeName().getSimple() : "VARCHAR"; + org.apache.calcite.sql.type.SqlTypeName returnType; + try { + returnType = org.apache.calcite.sql.type.SqlTypeName.valueOf(returnsName.toUpperCase(java.util.Locale.ROOT)); + } catch (IllegalArgumentException e) { + throw new DdlException(create, "Unknown return type: " + returnsName); + } + + // Register opaque function overloads in Calcite schema so queries pass validation. + // Each overload accepts ANY-typed parameters and returns the specified type. + final Pair pair = HoptimatorDdlUtils.schema(context, true, create.name); + SchemaPlus schemaPlus = pair.left != null ? pair.left.plus() : context.getRootSchema().plus(); + for (OpaqueFunction fn : OpaqueFunction.overloads(returnType)) { + schemaPlus.add(name, fn); + } + + // Register in session for pipeline DDL emission + connection.registerFunction(userFunction); + + logger.info("CREATE FUNCTION {} completed", name); + } + // N.B. originally copy-pasted from Apache Calcite /** Executes a {@code CREATE TABLE} command. */ @@ -629,6 +681,18 @@ public void execute(SqlDropObject drop, CalcitePrepare.Context context) { throw new DdlException(drop, e.getMessage(), e); } + // Handle DROP FUNCTION + if (drop.getKind().equals(SqlKind.DROP_FUNCTION)) { + String name = drop.name.names.get(drop.name.names.size() - 1); + final Pair fnPair = HoptimatorDdlUtils.schema(context, false, drop.name); + if (fnPair.left != null) { + fnPair.left.removeFunction(name); + } + connection.removeFunction(name); + logger.info("DROP FUNCTION {} completed", name); + return; + } + // The logic below is only applicable for DROP VIEW and DROP MATERIALIZED VIEW. if (!drop.getKind().equals(SqlKind.DROP_MATERIALIZED_VIEW) && !drop.getKind().equals(SqlKind.DROP_VIEW) diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/OpaqueFunction.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/OpaqueFunction.java new file mode 100644 index 00000000..dc672423 --- /dev/null +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/OpaqueFunction.java @@ -0,0 +1,128 @@ +package com.linkedin.hoptimator.jdbc; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.calcite.adapter.enumerable.CallImplementor; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.schema.FunctionParameter; +import org.apache.calcite.schema.ImplementableFunction; +import org.apache.calcite.schema.ScalarFunction; +import org.apache.calcite.sql.type.SqlTypeName; + + +/** + * A permissive opaque function used for Calcite schema registration. + * + *

When users register UDFs via CREATE FUNCTION, this class provides a + * placeholder so that queries referencing the UDF pass Calcite validation. + * The actual function implementation is provided by the runtime (e.g., Flink). + * + *

Each overload accepts a fixed number of ANY-typed parameters and returns + * a configurable type (defaulting to VARCHAR). ANY parameters allow any column + * type to be passed without casting. The return type can be specified via the + * RETURNS option in CREATE FUNCTION ... WITH (RETURNS 'INTEGER'). + * + *

At execution time (e.g., in test query previews), the function returns null. + */ +public final class OpaqueFunction implements ScalarFunction, ImplementableFunction { + + private static final int MAX_ARITY = 10; + private final int arity; + private final SqlTypeName returnType; + + private OpaqueFunction(int arity, SqlTypeName returnType) { + this.arity = arity; + this.returnType = returnType; + } + + /** + * Creates a list of opaque function overloads covering arities 0 through {@link #MAX_ARITY} + * with the given return type. + */ + public static List overloads(SqlTypeName returnType) { + return overloads(MAX_ARITY, returnType); + } + + /** + * Creates a list of opaque function overloads covering arities 0 through maxArity. + */ + public static List overloads(int maxArity, SqlTypeName returnType) { + List functions = new ArrayList<>(); + for (int i = 0; i <= maxArity; i++) { + functions.add(new OpaqueFunction(i, returnType)); + } + return Collections.unmodifiableList(functions); + } + + @Override + public RelDataType getReturnType(RelDataTypeFactory typeFactory) { + return typeFactory.createTypeWithNullability( + typeFactory.createSqlType(returnType), true); + } + + @Override + public List getParameters() { + List params = new ArrayList<>(); + for (int i = 0; i < arity; i++) { + final int ordinal = i; + params.add(new FunctionParameter() { + @Override + public int getOrdinal() { + return ordinal; + } + + @Override + public String getName() { + return "param" + ordinal; + } + + @Override + public RelDataType getType(RelDataTypeFactory typeFactory) { + return typeFactory.createTypeWithNullability( + typeFactory.createSqlType(SqlTypeName.ANY), true); + } + + @Override + public boolean isOptional() { + return false; + } + }); + } + return params; + } + + @Override + public CallImplementor getImplementor() { + // Returns null for any invocation. UDFs are opaque placeholders in the + // JDBC driver; they only truly execute in the data plane (e.g., Flink). + return (translator, call, nullAs) -> Expressions.constant(null, javaType(returnType)); + } + + private static Class javaType(SqlTypeName typeName) { + switch (typeName) { + case BOOLEAN: + return Boolean.class; + case TINYINT: + return Byte.class; + case SMALLINT: + return Short.class; + case INTEGER: + return Integer.class; + case BIGINT: + return Long.class; + case FLOAT: + case REAL: + return Float.class; + case DOUBLE: + return Double.class; + case DECIMAL: + return java.math.BigDecimal.class; + default: + return String.class; + } + } +} diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/HoptimatorDdlParserImpl.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/HoptimatorDdlParserImpl.java index 01f8b133..4ea70433 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/HoptimatorDdlParserImpl.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/HoptimatorDdlParserImpl.java @@ -6367,10 +6367,17 @@ final public SqlCreate SqlCreateFunction(Span s, boolean replace) throws ParseEx final SqlIdentifier id; final SqlNode template; SqlNode namespace = null; + SqlDataTypeSpec returnType = null; + SqlIdentifier language = null; SqlNodeList optionList = null; jj_consume_token(FUNCTION); ifNotExists = IfNotExistsOpt(); id = CompoundIdentifier(); + // optional: RETURNS + if (jj_ntk == RETURNS || (jj_ntk == -1 && jj_ntk() == RETURNS)) { + jj_consume_token(RETURNS); + returnType = DataType(); + } jj_consume_token(AS); template = StringLiteral(); switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { @@ -6382,6 +6389,11 @@ final public SqlCreate SqlCreateFunction(Span s, boolean replace) throws ParseEx jj_la1[59] = jj_gen; ; } + // optional: LANGUAGE + if (jj_ntk == LANGUAGE || (jj_ntk == -1 && jj_ntk() == LANGUAGE)) { + jj_consume_token(LANGUAGE); + language = SimpleIdentifier(); + } switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { case WITH: optionList = Options(); @@ -6391,7 +6403,7 @@ final public SqlCreate SqlCreateFunction(Span s, boolean replace) throws ParseEx ; } {if (true) return new SqlCreateFunction(s.end(this), replace, ifNotExists, id, template, - namespace, optionList);} + namespace, returnType, language, optionList);} throw new Error("Missing return statement in function"); } diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/SqlCreateFunction.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/SqlCreateFunction.java index 2386ebe3..37827f79 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/SqlCreateFunction.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/SqlCreateFunction.java @@ -19,6 +19,7 @@ package com.linkedin.hoptimator.jdbc.ddl; import org.apache.calcite.sql.SqlCreate; +import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; @@ -35,30 +36,46 @@ /** - * Parse tree for `CREATE FUNCTION` statement. + * Parse tree for {@code CREATE FUNCTION} statement. + * + *

{@code
+ * CREATE FUNCTION name [RETURNS type] AS 'class' [IN 'namespace'] [LANGUAGE lang] [WITH (...)]
+ * }
*/ public class SqlCreateFunction extends SqlCreate { public final SqlIdentifier name; public final SqlNode job; public final SqlNode namespace; + public final SqlDataTypeSpec returnType; + public final SqlIdentifier language; public final SqlNodeList options; private static final SqlOperator OPERATOR = new SqlSpecialOperator("CREATE FUNCTION", SqlKind.CREATE_FUNCTION); + /** Constructor with returnType and language. */ public SqlCreateFunction(SqlParserPos pos, boolean replace, boolean ifNotExists, - SqlIdentifier name, SqlNode job, SqlNode namespace, SqlNodeList options) { + SqlIdentifier name, SqlNode job, SqlNode namespace, SqlDataTypeSpec returnType, + SqlIdentifier language, SqlNodeList options) { super(OPERATOR, pos, replace, ifNotExists); this.name = requireNonNull(name, "name"); this.job = requireNonNull(job, "job"); - this.namespace = namespace; + this.namespace = namespace; + this.returnType = returnType; + this.language = language; this.options = options; } + /** Backward-compatible constructor without returnType and language. */ + public SqlCreateFunction(SqlParserPos pos, boolean replace, boolean ifNotExists, + SqlIdentifier name, SqlNode job, SqlNode namespace, SqlNodeList options) { + this(pos, replace, ifNotExists, name, job, namespace, null, null, options); + } + @SuppressWarnings("nullness") @Override public List getOperandList() { - return ImmutableNullableList.of(name, job, namespace, options); + return ImmutableNullableList.of(name, job, namespace, returnType, language, options); } @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { @@ -68,12 +85,20 @@ public SqlCreateFunction(SqlParserPos pos, boolean replace, boolean ifNotExists, writer.keyword("IF NOT EXISTS"); } name.unparse(writer, leftPrec, rightPrec); + if (returnType != null) { + writer.keyword("RETURNS"); + returnType.unparse(writer, 0, 0); + } writer.keyword("AS"); job.unparse(writer, 0, 0); if (namespace != null) { writer.keyword("IN"); namespace.unparse(writer, 0, 0); } + if (language != null) { + writer.keyword("LANGUAGE"); + language.unparse(writer, 0, 0); + } if (options != null) { writer.keyword("WITH"); options.unparse(writer, 0, 0); diff --git a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/TestSqlScripts.java b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/TestSqlScripts.java index 1c74f5ff..6015e6b8 100644 --- a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/TestSqlScripts.java +++ b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/TestSqlScripts.java @@ -23,6 +23,11 @@ public void basicDdlScript() throws Exception { run("basic-ddl.id"); } + @Test + public void createFunctionDdlScript() throws Exception { + run("create-function-ddl.id"); + } + @Test public void createViewWithAValidatorRejectingCreateViewThrowsException() throws Exception { // Runs the test in a separate thread to isolate the context class loader changes. diff --git a/hoptimator-jdbc/src/test/resources/create-function-ddl.id b/hoptimator-jdbc/src/test/resources/create-function-ddl.id new file mode 100644 index 00000000..9cd01669 --- /dev/null +++ b/hoptimator-jdbc/src/test/resources/create-function-ddl.id @@ -0,0 +1,95 @@ +!set outputformat mysql +!use util + +# Test CREATE FUNCTION with default return type (VARCHAR) +create function my_udf as 'com.example.MyUdf'; +(0 rows modified) + +!update + +# VARCHAR UDF validates against inline data; returns null (opaque placeholder) +select my_udf(x) as val from (values ('hello'), ('world')) as t(x); ++-----+ +| VAL | ++-----+ +| | +| | ++-----+ +(2 rows) + +!ok + +# Test CREATE FUNCTION with explicit INTEGER return type +create function int_udf returns integer as 'com.example.IntUdf'; +(0 rows modified) + +!update + +# Integer UDF validates against integer input +select int_udf(x) as val from (values (10), (20), (30)) as t(x); ++-----+ +| VAL | ++-----+ +| | +| | +| | ++-----+ +(3 rows) + +!ok + +# Test CREATE FUNCTION with BOOLEAN return type +create function bool_udf returns boolean as 'com.example.BoolUdf'; +(0 rows modified) + +!update + +# Boolean UDF validates against varchar input +select bool_udf(x) as val from (values ('a')) as t(x); ++-----+ +| VAL | ++-----+ +| | ++-----+ +(1 row) + +!ok + +# Test CREATE FUNCTION with LANGUAGE clause +create function py_udf returns varchar as 'my_module.my_func' language python; +(0 rows modified) + +!update + +# Multi-arg UDF validates with two arguments +select py_udf(x, y) as val from (values ('a', 'b'), ('c', 'd')) as t(x, y); ++-----+ +| VAL | ++-----+ +| | +| | ++-----+ +(2 rows) + +!ok + +# Clean up +drop function my_udf; +(0 rows modified) + +!update + +drop function int_udf; +(0 rows modified) + +!update + +drop function bool_udf; +(0 rows modified) + +!update + +drop function py_udf; +(0 rows modified) + +!update diff --git a/hoptimator-jdbc/src/testFixtures/java/com/linkedin/hoptimator/jdbc/QuidemTestBase.java b/hoptimator-jdbc/src/testFixtures/java/com/linkedin/hoptimator/jdbc/QuidemTestBase.java index 4df59b71..c3f8f13f 100644 --- a/hoptimator-jdbc/src/testFixtures/java/com/linkedin/hoptimator/jdbc/QuidemTestBase.java +++ b/hoptimator-jdbc/src/testFixtures/java/com/linkedin/hoptimator/jdbc/QuidemTestBase.java @@ -41,6 +41,7 @@ import com.linkedin.hoptimator.Pipeline; import com.linkedin.hoptimator.Source; +import com.linkedin.hoptimator.UserFunction; import com.linkedin.hoptimator.util.DeploymentService; @@ -264,6 +265,10 @@ public void execute(Context context, boolean execute) throws Exception { } PipelineRel.Implementor plan = DeploymentService.plan(root, Collections.emptyList(), connectionProperties); + // Pass registered functions to the implementor + for (UserFunction func : conn.functions()) { + plan.addFunction(func); + } if (create != null) { HoptimatorDdlUtils.snapshotAndSetSinkSchema(conn.createPrepareContext(), new HoptimatorDriver.Prepare(conn), plan, create, querySql); diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sJobDeployer.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sJobDeployer.java index 491c1a4b..781d71ea 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sJobDeployer.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sJobDeployer.java @@ -55,6 +55,7 @@ public List specify() throws SQLException { .with("flinksql", () -> sql.apply(SqlDialect.FLINK)) .with("flinkconfigs", properties) .with("fieldMap", () -> "'" + fieldMap.apply(SqlDialect.ANSI) + "'") + .with("files", job.files()) .with(properties); List templates = jobTemplateApi.list() .stream() diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/models/V1alpha1SqlJobSpec.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/models/V1alpha1SqlJobSpec.java index bda08a83..cad50681 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/models/V1alpha1SqlJobSpec.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/models/V1alpha1SqlJobSpec.java @@ -140,6 +140,10 @@ public ExecutionModeEnum read(final JsonReader jsonReader) throws IOException { @SerializedName(SERIALIZED_NAME_EXECUTION_MODE) private ExecutionModeEnum executionMode; + public static final String SERIALIZED_NAME_FILES = "files"; + @SerializedName(SERIALIZED_NAME_FILES) + private Map files = null; + public static final String SERIALIZED_NAME_SQL = "sql"; @SerializedName(SERIALIZED_NAME_SQL) private List sql = new ArrayList<>(); @@ -222,6 +226,37 @@ public void setExecutionMode(ExecutionModeEnum executionMode) { } + public V1alpha1SqlJobSpec files(Map files) { + + this.files = files; + return this; + } + + public V1alpha1SqlJobSpec putFilesItem(String key, String filesItem) { + if (this.files == null) { + this.files = new HashMap<>(); + } + this.files.put(key, filesItem); + return this; + } + + /** + * Inline file content to make available to the SQL job. + * @return files + **/ + @javax.annotation.Nullable + @ApiModelProperty(value = "Inline file content to make available to the SQL job.") + + public Map getFiles() { + return files; + } + + + public void setFiles(Map files) { + this.files = files; + } + + public V1alpha1SqlJobSpec sql(List sql) { this.sql = sql; @@ -261,12 +296,13 @@ public boolean equals(Object o) { return Objects.equals(this.configs, v1alpha1SqlJobSpec.configs) && Objects.equals(this.dialect, v1alpha1SqlJobSpec.dialect) && Objects.equals(this.executionMode, v1alpha1SqlJobSpec.executionMode) && + Objects.equals(this.files, v1alpha1SqlJobSpec.files) && Objects.equals(this.sql, v1alpha1SqlJobSpec.sql); } @Override public int hashCode() { - return Objects.hash(configs, dialect, executionMode, sql); + return Objects.hash(configs, dialect, executionMode, files, sql); } @@ -277,6 +313,7 @@ public String toString() { sb.append(" configs: ").append(toIndentedString(configs)).append("\n"); sb.append(" dialect: ").append(toIndentedString(dialect)).append("\n"); sb.append(" executionMode: ").append(toIndentedString(executionMode)).append("\n"); + sb.append(" files: ").append(toIndentedString(files)).append("\n"); sb.append(" sql: ").append(toIndentedString(sql)).append("\n"); sb.append("}"); return sb.toString(); diff --git a/hoptimator-k8s/src/main/resources/sqljobs.crd.yaml b/hoptimator-k8s/src/main/resources/sqljobs.crd.yaml index c3442bbf..7ee1a862 100644 --- a/hoptimator-k8s/src/main/resources/sqljobs.crd.yaml +++ b/hoptimator-k8s/src/main/resources/sqljobs.crd.yaml @@ -57,6 +57,11 @@ spec: type: object additionalProperties: type: string + files: + description: Inline file content to make available to the SQL job (e.g., Python UDF code). + type: object + additionalProperties: + type: string required: - sql status: diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java index c88c66ec..116d7630 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java @@ -19,6 +19,11 @@ public void k8sDdlScriptFunction() throws Exception { run("k8s-ddl-function.id", "fun=mysql"); } + @Test + public void k8sDdlScriptUdf() throws Exception { + run("k8s-ddl-udf.id"); + } + @Test public void k8sValidationScript() throws Exception { run("k8s-validation.id"); diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id new file mode 100644 index 00000000..7ab51bde --- /dev/null +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id @@ -0,0 +1,72 @@ +!set outputformat mysql +!use k8s + +# Register a VARCHAR UDF (default return type) +create function my_udf as 'com.example.MyUdf'; +(0 rows modified) + +!update + +# UDF validates against real data +select my_udf("FIRST_NAME") as val from profile.members; ++-----+ +| VAL | ++-----+ +| | +| | +| | ++-----+ +(3 rows) + +!ok + +# Register an INTEGER UDF with explicit return type +create function int_udf returns integer as 'com.example.IntUdf'; +(0 rows modified) + +!update + +# Register a Python UDF with LANGUAGE clause +create function py_udf returns varchar as 'my_module.my_func' language python; +(0 rows modified) + +!update + +# Verify pipeline SQL includes CREATE FUNCTION before connectors +insert into ads.page_views select my_udf("FIRST_NAME") as page_urn, "MEMBER_URN" as member_urn from profile.members; +apiVersion: flink.apache.org/v1beta1 +kind: FlinkSessionJob +metadata: + name: ads-database-pageviews +spec: + deploymentName: basic-session-deployment + job: + entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner + args: + - CREATE FUNCTION IF NOT EXISTS `my_udf` AS 'com.example.MyUdf' + - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () + - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') + - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `my_udf`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` + jarURI: file:///opt/hoptimator-flink-runner.jar + parallelism: 1 + upgradeMode: stateless + state: running +!specify PAGE_VIEWS + +# Clean up +drop function my_udf; +(0 rows modified) + +!update + +drop function int_udf; +(0 rows modified) + +!update + +drop function py_udf; +(0 rows modified) + +!update diff --git a/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/PipelineRel.java b/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/PipelineRel.java index f48e4c0c..e73e07aa 100644 --- a/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/PipelineRel.java +++ b/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/PipelineRel.java @@ -2,8 +2,10 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.hoptimator.ThrowingFunction; +import com.linkedin.hoptimator.UserFunction; import java.sql.Connection; import java.sql.SQLException; +import java.util.ArrayList; import java.sql.SQLNonTransientException; import java.util.HashMap; import java.util.LinkedHashMap; @@ -51,6 +53,7 @@ public interface PipelineRel extends RelNode { /** Implements a deployable Pipeline. */ class Implementor { private final Map sources = new LinkedHashMap<>(); + private final List functions = new ArrayList<>(); private final ImmutablePairList targetFields; private final Map hints; private RelNode query; @@ -98,6 +101,10 @@ public void setSink(String database, List path, RelDataType rowType, Map this.sink = new Sink(database, path, newOptions); } + public void addFunction(UserFunction func) { + functions.add(func); + } + public void setQuery(RelNode query) { this.query = query; } @@ -109,12 +116,29 @@ public Pipeline pipeline(String name, Connection connection) throws SQLException templateEvals.put("query", query(connection)); templateEvals.put("fieldMap", fieldMap()); - Job job = new Job(name, sources.keySet(), sink, templateEvals); + // Collect inline file content from functions with CODE option + Map files = new HashMap<>(); + for (UserFunction func : functions) { + String code = func.options().get("CODE"); + if (code != null) { + // Derive filename from AS clause (e.g., 'my_module.my_func' -> 'my_module.py') + String module = func.as(); + int dotIdx = module.indexOf('.'); + String filename = (dotIdx >= 0 ? module.substring(0, dotIdx) : module) + ".py"; + files.put(filename, code); + } + } + + Job job = new Job(name, sources.keySet(), sink, templateEvals, files); return new Pipeline(sources.keySet(), sink, job); } private ScriptImplementor script(Connection connection) throws SQLException { ScriptImplementor script = ScriptImplementor.empty(); + // Emit function definitions first + for (UserFunction func : functions) { + script = script.function(func.name(), func.as(), func.options()); + } // Check if we need to add suffixes to avoid table name collisions boolean needsSuffixes = hasTableNameCollision(); diff --git a/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/ScriptImplementor.java b/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/ScriptImplementor.java index 45c73881..1edb09b9 100644 --- a/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/ScriptImplementor.java +++ b/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/ScriptImplementor.java @@ -116,6 +116,11 @@ default ScriptImplementor database(@Nullable String catalog, String database) { return with(new DatabaseImplementor(catalog, database)); } + /** Append a function definition, e.g. `CREATE FUNCTION IF NOT EXISTS ... AS ...` */ + default ScriptImplementor function(String name, String as, Map options) { + return with(new FunctionImplementor(name, as, options)); + } + /** Append an insert statement, e.g. `INSERT INTO ... SELECT ...` */ default ScriptImplementor insert(@Nullable String catalog, String schema, String table, RelNode relNode) { return insert(catalog, schema, table, null, relNode, null, Collections.emptyMap()); @@ -519,6 +524,33 @@ public void implement(SqlWriter w) { } } + /** Implements a CREATE FUNCTION IF NOT EXISTS statement */ + class FunctionImplementor implements ScriptImplementor { + private final String name; + private final String as; + private final Map options; + + public FunctionImplementor(String name, String as, Map options) { + this.name = name; + this.as = as; + this.options = options != null ? options : Collections.emptyMap(); + } + + @Override + public void implement(SqlWriter w) { + w.keyword("CREATE FUNCTION IF NOT EXISTS"); + w.identifier(name, true); + w.keyword("AS"); + w.literal("'" + as + "'"); + String language = options.get("LANGUAGE"); + if (language != null) { + w.keyword("LANGUAGE"); + w.literal(language); + } + w.literal(";"); + } + } + /** Implements an identifier like TRACKING.'PageViewEvent' */ class CompoundIdentifierImplementor implements ScriptImplementor { private final String catalog; diff --git a/hoptimator-util/src/test/java/com/linkedin/hoptimator/util/planner/ScriptImplementorTest.java b/hoptimator-util/src/test/java/com/linkedin/hoptimator/util/planner/ScriptImplementorTest.java index 9d7e1f5b..12ecceb9 100644 --- a/hoptimator-util/src/test/java/com/linkedin/hoptimator/util/planner/ScriptImplementorTest.java +++ b/hoptimator-util/src/test/java/com/linkedin/hoptimator/util/planner/ScriptImplementorTest.java @@ -16,10 +16,12 @@ import java.util.AbstractMap; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; /** @@ -174,6 +176,68 @@ public void testFullPipelineWithCollision() { "Should insert into sink table. Got: " + sql); } + @Test + public void testFunctionWithoutLanguage() { + String sql = ScriptImplementor.empty() + .function("my_udf", "com.example.MyUdf", Collections.emptyMap()) + .sql(); + + assertTrue(sql.contains("CREATE FUNCTION IF NOT EXISTS"), + "Should contain CREATE FUNCTION IF NOT EXISTS. Got: " + sql); + assertTrue(sql.contains("my_udf") || sql.contains("`my_udf`"), + "Should contain function name. Got: " + sql); + assertTrue(sql.contains("'com.example.MyUdf'"), + "Should contain AS clause. Got: " + sql); + assertFalse(sql.contains("LANGUAGE"), + "Should not contain LANGUAGE clause when not specified. Got: " + sql); + } + + @Test + public void testFunctionWithLanguage() { + Map options = new HashMap<>(); + options.put("LANGUAGE", "PYTHON"); + + String sql = ScriptImplementor.empty() + .function("my_udf", "my_module.my_func", options) + .sql(); + + assertTrue(sql.contains("CREATE FUNCTION IF NOT EXISTS"), + "Should contain CREATE FUNCTION IF NOT EXISTS. Got: " + sql); + assertTrue(sql.contains("'my_module.my_func'"), + "Should contain AS clause. Got: " + sql); + assertTrue(sql.contains("LANGUAGE"), + "Should contain LANGUAGE clause. Got: " + sql); + assertTrue(sql.contains("PYTHON"), + "Should contain PYTHON language. Got: " + sql); + } + + @Test + public void testFunctionBeforeConnector() { + RelDataTypeFactory typeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT); + RelDataType rowType = typeFactory.builder() + .add("ID", typeFactory.createSqlType(SqlTypeName.INTEGER)) + .build(); + + Map config = new HashMap<>(); + config.put("connector", "datagen"); + + Map funcOptions = new HashMap<>(); + funcOptions.put("LANGUAGE", "PYTHON"); + + String sql = ScriptImplementor.empty() + .function("my_udf", "my_module.my_func", funcOptions) + .database(null, "TEST") + .connector(null, "TEST", "MY_TABLE", rowType, config) + .sql(); + + int funcIdx = sql.indexOf("CREATE FUNCTION"); + int tableIdx = sql.indexOf("CREATE TABLE"); + assertTrue(funcIdx >= 0, "Should contain CREATE FUNCTION. Got: " + sql); + assertTrue(tableIdx >= 0, "Should contain CREATE TABLE. Got: " + sql); + assertTrue(funcIdx < tableIdx, + "CREATE FUNCTION should appear before CREATE TABLE. Got: " + sql); + } + @Test public void testExplicitColumnEnumeration() { // Test for Flink 1.20 regression where INSERT with SELECT * fails From 5af8d9f68637522088a27cea3f0cb4cc3e7bb3e3 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Thu, 19 Mar 2026 16:56:16 -0700 Subject: [PATCH 02/14] Fix k8s-ddl-udf.id expected output: uppercase identifiers, all session functions Calcite normalizes identifiers to uppercase, and all session-registered functions are emitted in pipeline SQL (not just the one used in the query). Co-Authored-By: Claude Opus 4.6 (1M context) --- hoptimator-k8s/src/test/resources/k8s-ddl-udf.id | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id index 7ab51bde..133d4b3e 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id @@ -32,7 +32,7 @@ create function py_udf returns varchar as 'my_module.my_func' language python; !update -# Verify pipeline SQL includes CREATE FUNCTION before connectors +# Verify pipeline SQL includes CREATE FUNCTION for all registered UDFs before connectors insert into ads.page_views select my_udf("FIRST_NAME") as page_urn, "MEMBER_URN" as member_urn from profile.members; apiVersion: flink.apache.org/v1beta1 kind: FlinkSessionJob @@ -43,12 +43,14 @@ spec: job: entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner args: - - CREATE FUNCTION IF NOT EXISTS `my_udf` AS 'com.example.MyUdf' + - CREATE FUNCTION IF NOT EXISTS `MY_UDF` AS 'com.example.MyUdf' + - CREATE FUNCTION IF NOT EXISTS `INT_UDF` AS 'com.example.IntUdf' + - CREATE FUNCTION IF NOT EXISTS `PY_UDF` AS 'my_module.my_func' LANGUAGE PYTHON - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `my_udf`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` + - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `MY_UDF`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` jarURI: file:///opt/hoptimator-flink-runner.jar parallelism: 1 upgradeMode: stateless From 83a3785294a208141908d78b57ba124959785cdf Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Mon, 23 Mar 2026 13:47:34 -0500 Subject: [PATCH 03/14] Demo UDFs in Flink data plane with integration test Add Java and Python UDF implementations baked into the Flink runner image so CREATE FUNCTION DDL resolves real functions at runtime: - Greet: scalar VARCHAR UDF (Java) - StringLength: scalar INTEGER UDF (Java) - reverse_string: scalar VARCHAR UDF (Python/PyFlink) Update Dockerfile to install Python/PyFlink and copy Python UDFs. Configure Flink session cluster with Python executable paths. Add k8s-ddl-udf-demo.id integration test using real UDF class names. Co-Authored-By: Claude Opus 4.6 (1M context) --- deploy/dev/flink-session-cluster.yaml | 2 + .../Dockerfile-flink-runner | 12 +++ hoptimator-flink-runner/build.gradle | 2 + hoptimator-flink-runner/python/demo_udfs.py | 8 ++ .../flink/runner/functions/Greet.java | 15 ++++ .../flink/runner/functions/StringLength.java | 15 ++++ .../flink/runner/functions/GreetTest.java | 26 ++++++ .../runner/functions/StringLengthTest.java | 26 ++++++ .../hoptimator/k8s/TestSqlScripts.java | 5 ++ .../src/test/resources/k8s-ddl-udf-demo.id | 85 +++++++++++++++++++ 10 files changed, 196 insertions(+) create mode 100644 hoptimator-flink-runner/python/demo_udfs.py create mode 100644 hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/functions/Greet.java create mode 100644 hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/functions/StringLength.java create mode 100644 hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/functions/GreetTest.java create mode 100644 hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/functions/StringLengthTest.java create mode 100644 hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id diff --git a/deploy/dev/flink-session-cluster.yaml b/deploy/dev/flink-session-cluster.yaml index 40f46dbf..72efc0e3 100644 --- a/deploy/dev/flink-session-cluster.yaml +++ b/deploy/dev/flink-session-cluster.yaml @@ -9,6 +9,8 @@ spec: flinkVersion: v1_18 flinkConfiguration: taskmanager.numberOfTaskSlots: "6" + python.executable: /usr/bin/python3 + python.client.executable: /usr/bin/python3 serviceAccount: flink jobManager: resource: diff --git a/hoptimator-flink-runner/Dockerfile-flink-runner b/hoptimator-flink-runner/Dockerfile-flink-runner index 0f59e377..93ba6348 100644 --- a/hoptimator-flink-runner/Dockerfile-flink-runner +++ b/hoptimator-flink-runner/Dockerfile-flink-runner @@ -1,2 +1,14 @@ FROM flink:1.18.1 + +# Install Python and PyFlink for Python UDF support +RUN apt-get update && \ + apt-get install -y python3 python3-pip && \ + pip3 install apache-flink==1.18.1 && \ + rm -rf /var/lib/apt/lists/* + +# Copy Python UDF modules +COPY python/ /opt/python-udfs/ +ENV PYTHONPATH=/opt/python-udfs + +# Copy runner JAR (includes Java UDFs) COPY ./build/libs/hoptimator-flink-runner-all.jar /opt/hoptimator-flink-runner.jar diff --git a/hoptimator-flink-runner/build.gradle b/hoptimator-flink-runner/build.gradle index c2840a2d..12b44695 100644 --- a/hoptimator-flink-runner/build.gradle +++ b/hoptimator-flink-runner/build.gradle @@ -12,11 +12,13 @@ dependencies { implementation libs.flink.csv //Flink depends should be provided by runtime + compileOnly libs.flink.table.common compileOnly libs.flink.table.planner compileOnly libs.flink.clients compileOnly libs.flink.table.api.java.bridge testImplementation libs.assertj + testImplementation libs.flink.table.common } shadowJar { diff --git a/hoptimator-flink-runner/python/demo_udfs.py b/hoptimator-flink-runner/python/demo_udfs.py new file mode 100644 index 00000000..26b9ac0d --- /dev/null +++ b/hoptimator-flink-runner/python/demo_udfs.py @@ -0,0 +1,8 @@ +from pyflink.table.udf import udf +from pyflink.table import DataTypes + + +@udf(result_type=DataTypes.STRING()) +def reverse_string(s): + """A simple Python UDF that reverses a string.""" + return s[::-1] if s else None diff --git a/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/functions/Greet.java b/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/functions/Greet.java new file mode 100644 index 00000000..23f9983c --- /dev/null +++ b/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/functions/Greet.java @@ -0,0 +1,15 @@ +package com.linkedin.hoptimator.flink.runner.functions; + +import org.apache.flink.table.functions.ScalarFunction; + + +/** A simple scalar UDF that greets a name. */ +public class Greet extends ScalarFunction { + + public String eval(String name) { + if (name == null) { + return null; + } + return "Hello, " + name + "!"; + } +} diff --git a/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/functions/StringLength.java b/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/functions/StringLength.java new file mode 100644 index 00000000..5fa16a95 --- /dev/null +++ b/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/functions/StringLength.java @@ -0,0 +1,15 @@ +package com.linkedin.hoptimator.flink.runner.functions; + +import org.apache.flink.table.functions.ScalarFunction; + + +/** A simple scalar UDF that returns the length of a string. */ +public class StringLength extends ScalarFunction { + + public Integer eval(String s) { + if (s == null) { + return null; + } + return s.length(); + } +} diff --git a/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/functions/GreetTest.java b/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/functions/GreetTest.java new file mode 100644 index 00000000..db0c4430 --- /dev/null +++ b/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/functions/GreetTest.java @@ -0,0 +1,26 @@ +package com.linkedin.hoptimator.flink.runner.functions; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; + + +class GreetTest { + + private final Greet greet = new Greet(); + + @Test + void greetName() { + assertThat(greet.eval("Alice")).isEqualTo("Hello, Alice!"); + } + + @Test + void greetEmptyString() { + assertThat(greet.eval("")).isEqualTo("Hello, !"); + } + + @Test + void greetNull() { + assertThat(greet.eval(null)).isNull(); + } +} diff --git a/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/functions/StringLengthTest.java b/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/functions/StringLengthTest.java new file mode 100644 index 00000000..be88b6b7 --- /dev/null +++ b/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/functions/StringLengthTest.java @@ -0,0 +1,26 @@ +package com.linkedin.hoptimator.flink.runner.functions; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; + + +class StringLengthTest { + + private final StringLength stringLength = new StringLength(); + + @Test + void lengthOfString() { + assertThat(stringLength.eval("hello")).isEqualTo(5); + } + + @Test + void lengthOfEmptyString() { + assertThat(stringLength.eval("")).isEqualTo(0); + } + + @Test + void lengthOfNull() { + assertThat(stringLength.eval(null)).isNull(); + } +} diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java index 116d7630..9d54d441 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java @@ -24,6 +24,11 @@ public void k8sDdlScriptUdf() throws Exception { run("k8s-ddl-udf.id"); } + @Test + public void k8sDdlScriptUdfDemo() throws Exception { + run("k8s-ddl-udf-demo.id"); + } + @Test public void k8sValidationScript() throws Exception { run("k8s-validation.id"); diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id new file mode 100644 index 00000000..831d1488 --- /dev/null +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id @@ -0,0 +1,85 @@ +!set outputformat mysql +!use k8s + +# Register demo Java UDFs baked into the Flink runner image +create function greet as 'com.linkedin.hoptimator.flink.runner.functions.Greet'; +(0 rows modified) + +!update + +create function str_len returns integer as 'com.linkedin.hoptimator.flink.runner.functions.StringLength'; +(0 rows modified) + +!update + +# Register demo Python UDF baked into the Flink runner image +create function reverse_str as 'demo_udfs.reverse_string' language python; +(0 rows modified) + +!update + +# UDFs validate against real data (opaque placeholders return null at JDBC layer) +select greet("FIRST_NAME") as greeting from profile.members; ++----------+ +| GREETING | ++----------+ +| | +| | +| | ++----------+ +(3 rows) + +!ok + +select str_len("FIRST_NAME") as len from profile.members; ++-----+ +| LEN | ++-----+ +| | +| | +| | ++-----+ +(3 rows) + +!ok + +# Pipeline with Java UDF generates correct FlinkSessionJob spec +insert into ads.page_views select greet("FIRST_NAME") as page_urn, "MEMBER_URN" as member_urn from profile.members; +apiVersion: flink.apache.org/v1beta1 +kind: FlinkSessionJob +metadata: + name: ads-database-pageviews +spec: + deploymentName: basic-session-deployment + job: + entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner + args: + - CREATE FUNCTION IF NOT EXISTS `GREET` AS 'com.linkedin.hoptimator.flink.runner.functions.Greet' + - CREATE FUNCTION IF NOT EXISTS `STR_LEN` AS 'com.linkedin.hoptimator.flink.runner.functions.StringLength' + - CREATE FUNCTION IF NOT EXISTS `REVERSE_STR` AS 'demo_udfs.reverse_string' LANGUAGE PYTHON + - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () + - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') + - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `GREET`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` + jarURI: file:///opt/hoptimator-flink-runner.jar + parallelism: 1 + upgradeMode: stateless + state: running +!specify PAGE_VIEWS + +# Clean up +drop function greet; +(0 rows modified) + +!update + +drop function str_len; +(0 rows modified) + +!update + +drop function reverse_str; +(0 rows modified) + +!update From e00c73fab3d3b482bbf434314d015744adde7819 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Mon, 23 Mar 2026 16:46:44 -0500 Subject: [PATCH 04/14] Route UDF delivery through SqlJob CRD for dynamic file delivery Change flink-template.yaml to generate SqlJob instead of FlinkSessionJob directly, so that UDF files (Python code) are bundled into the CRD and can be dynamically delivered to the data plane. - flink-template.yaml now generates SqlJob with sql + files fields - FlinkStreamingSqlJob.yaml.template changed to FlinkSessionJob (session mode) - FlinkStreamingSqlJob encodes files as --file: directives in sql args - FlinkRunner parses --file: directives, writes to /opt/python-udfs/ - FlinkControllerProvider registers FlinkSessionJob API - All integration test expected output updated from FlinkSessionJob to SqlJob Co-Authored-By: Claude Opus 4.6 (1M context) --- deploy/samples/flink-template.yaml | 21 ++- .../catalog/flink/FlinkStreamingSqlJob.java | 21 ++- .../flink/FlinkControllerProvider.java | 1 + .../FlinkStreamingSqlJob.yaml.template | 23 +--- .../hoptimator/flink/runner/FlinkRunner.java | 35 ++++- .../flink/runner/FlinkRunnerTest.java | 49 +++++++ .../src/test/resources/k8s-ddl-udf-demo.id | 34 ++--- .../src/test/resources/k8s-ddl-udf.id | 32 ++--- hoptimator-k8s/src/test/resources/k8s-ddl.id | 130 ++++++++---------- .../src/test/resources/k8s-metadata.id | 12 +- .../test/resources/kafka-ddl-create-table.id | 26 ++-- .../src/test/resources/kafka-ddl.id | 26 ++-- .../src/test/resources/mysql-ddl.id | 30 ++-- .../test/resources/venice-ddl-create-table.id | 26 ++-- .../resources/venice-ddl-insert-partial.id | 26 ++-- 15 files changed, 262 insertions(+), 230 deletions(-) create mode 100644 hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/FlinkRunnerTest.java diff --git a/deploy/samples/flink-template.yaml b/deploy/samples/flink-template.yaml index 1d43a867..d44783eb 100644 --- a/deploy/samples/flink-template.yaml +++ b/deploy/samples/flink-template.yaml @@ -6,18 +6,15 @@ metadata: name: flink-template spec: yaml: | - apiVersion: flink.apache.org/v1beta1 - kind: FlinkSessionJob + apiVersion: hoptimator.linkedin.com/v1alpha1 + kind: SqlJob metadata: name: {{name}} spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - {{flinksql}} - jarURI: file:///opt/{{flink.app.name}}.jar - parallelism: {{flink.parallelism:1}} - upgradeMode: stateless - state: running - {{flink.app.type==SQL}} + dialect: Flink + executionMode: Streaming + sql: + - {{flinksql}} + files: + {{files}} + {{flink.app.type==SQL}} diff --git a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java index f32b0992..bc5a58c0 100644 --- a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java +++ b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java @@ -2,6 +2,8 @@ import com.linkedin.hoptimator.catalog.Resource; +import java.nio.charset.StandardCharsets; +import java.util.Base64; import java.util.Map; @@ -15,9 +17,22 @@ public FlinkStreamingSqlJob(String namespace, String name, String sql) { } public FlinkStreamingSqlJob(String namespace, String name, String sql, Map files) { - this(namespace, name, sql); - if (files != null && !files.isEmpty()) { - export("files", files); + this(namespace, name, prependFiles(sql, files)); + } + + /** Prepend --file directives to the SQL script so FlinkRunner can extract and write them. */ + private static String prependFiles(String sql, Map files) { + if (files == null || files.isEmpty()) { + return sql; + } + StringBuilder sb = new StringBuilder(); + for (Map.Entry entry : files.entrySet()) { + String encoded = Base64.getEncoder().encodeToString( + entry.getValue().getBytes(StandardCharsets.UTF_8)); + sb.append("--file:").append(entry.getKey()).append(":").append(encoded); + sb.append(";\n"); } + sb.append(sql); + return sb.toString(); } } diff --git a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkControllerProvider.java b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkControllerProvider.java index b7f12aca..bc1f3e2e 100644 --- a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkControllerProvider.java +++ b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkControllerProvider.java @@ -19,6 +19,7 @@ public class FlinkControllerProvider implements ControllerProvider { @Override public Collection controllers(Operator operator) { operator.registerApi("FlinkDeployment", "flinkdeployment", "flinkdeployments", "flink.apache.org", "v1beta1"); + operator.registerApi("FlinkSessionJob", "flinksessionjob", "flinksessionjobs", "flink.apache.org", "v1beta1"); operator.registerApi("SqlJob", "sqljob", "sqljobs", "hoptimator.linkedin.com", "v1alpha1", V1alpha1SqlJob.class, V1alpha1SqlJobList.class); diff --git a/hoptimator-flink-adapter/src/main/resources/FlinkStreamingSqlJob.yaml.template b/hoptimator-flink-adapter/src/main/resources/FlinkStreamingSqlJob.yaml.template index 57956b0c..7945fc86 100644 --- a/hoptimator-flink-adapter/src/main/resources/FlinkStreamingSqlJob.yaml.template +++ b/hoptimator-flink-adapter/src/main/resources/FlinkStreamingSqlJob.yaml.template @@ -1,29 +1,14 @@ apiVersion: flink.apache.org/v1beta1 -kind: FlinkDeployment +kind: FlinkSessionJob metadata: - namespace: {{namespace}} - name: {{name}}-flink-job + name: {{name}} spec: - image: docker.io/library/hoptimator-flink-runner - imagePullPolicy: Never - flinkVersion: v1_18 - flinkConfiguration: - taskmanager.numberOfTaskSlots: "1" - serviceAccount: flink - jobManager: - resource: - memory: "2048m" - cpu: 0.1 - taskManager: - resource: - memory: "2048m" - cpu: 0.1 + deploymentName: basic-session-deployment job: entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner args: - {{sql}} - jarURI: local:///opt/hoptimator-flink-runner.jar + jarURI: file:///opt/hoptimator-flink-runner.jar parallelism: 1 upgradeMode: stateless state: running - diff --git a/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java b/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java index 985a9100..50e28e2a 100644 --- a/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java +++ b/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java @@ -6,15 +6,32 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Base64; + /** Runs SQL from command-line args. */ public final class FlinkRunner { private static final Logger logger = LoggerFactory.getLogger(FlinkRunner.class); + static final String FILE_PREFIX = "--file:"; + private static final Path DEFAULT_UDF_DIR = Paths.get("/opt/python-udfs"); private FlinkRunner() { } - public static void main(String[] args) { + public static void main(String[] args) throws IOException { + // Phase 1: Extract and write file directives before executing SQL + for (String arg : args) { + String stmt = arg.replaceAll("\\n", "").trim(); + if (stmt.startsWith(FILE_PREFIX)) { + writeFile(stmt, DEFAULT_UDF_DIR); + } + } + + // Phase 2: Execute SQL statements EnvironmentSettings settings = EnvironmentSettings.newInstance().inStreamingMode().build(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, settings); @@ -32,4 +49,20 @@ public static void main(String[] args) { } } } + + /** Parses a --file:name:base64content directive and writes the file to the given directory. */ + static void writeFile(String directive, Path targetDir) throws IOException { + String payload = directive.substring(FILE_PREFIX.length()); + int colonIdx = payload.indexOf(':'); + if (colonIdx < 0) { + throw new IllegalArgumentException("Invalid file directive (missing ':'): " + directive); + } + String filename = payload.substring(0, colonIdx); + String encoded = payload.substring(colonIdx + 1); + byte[] content = Base64.getDecoder().decode(encoded); + Files.createDirectories(targetDir); + Path filePath = targetDir.resolve(filename); + Files.write(filePath, content); + logger.info("Wrote UDF file: {}", filePath); + } } diff --git a/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/FlinkRunnerTest.java b/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/FlinkRunnerTest.java new file mode 100644 index 00000000..313a569c --- /dev/null +++ b/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/FlinkRunnerTest.java @@ -0,0 +1,49 @@ +package com.linkedin.hoptimator.flink.runner; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Base64; + + +class FlinkRunnerTest { + + @Test + void writeFileDecodesBase64AndWritesToDisk(@TempDir Path tempDir) throws Exception { + String content = "print('hello world')"; + String encoded = Base64.getEncoder().encodeToString(content.getBytes(StandardCharsets.UTF_8)); + String directive = "--file:test.py:" + encoded; + + FlinkRunner.writeFile(directive, tempDir); + + Path written = tempDir.resolve("test.py"); + assertThat(written).exists(); + assertThat(Files.readString(written)).isEqualTo(content); + } + + @Test + void writeFileHandlesMultilineContent(@TempDir Path tempDir) throws Exception { + String content = "from pyflink.table.udf import udf\n\n@udf(result_type=DataTypes.STRING())\ndef reverse(s):\n return s[::-1]\n"; + String encoded = Base64.getEncoder().encodeToString(content.getBytes(StandardCharsets.UTF_8)); + String directive = "--file:my_udfs.py:" + encoded; + + FlinkRunner.writeFile(directive, tempDir); + + Path written = tempDir.resolve("my_udfs.py"); + assertThat(written).exists(); + assertThat(Files.readString(written)).isEqualTo(content); + } + + @Test + void writeFileRejectsMalformedDirective(@TempDir Path tempDir) { + assertThatThrownBy(() -> FlinkRunner.writeFile("--file:no-colon-after-name", tempDir)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("missing ':'"); + } +} diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id index 831d1488..b08edfaf 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id @@ -43,29 +43,25 @@ select str_len("FIRST_NAME") as len from profile.members; !ok -# Pipeline with Java UDF generates correct FlinkSessionJob spec +# Pipeline with Java UDF generates correct SqlJob spec insert into ads.page_views select greet("FIRST_NAME") as page_urn, "MEMBER_URN" as member_urn from profile.members; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: ads-database-pageviews spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE FUNCTION IF NOT EXISTS `GREET` AS 'com.linkedin.hoptimator.flink.runner.functions.Greet' - - CREATE FUNCTION IF NOT EXISTS `STR_LEN` AS 'com.linkedin.hoptimator.flink.runner.functions.StringLength' - - CREATE FUNCTION IF NOT EXISTS `REVERSE_STR` AS 'demo_udfs.reverse_string' LANGUAGE PYTHON - - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () - - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') - - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `GREET`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 1 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE FUNCTION IF NOT EXISTS `GREET` AS 'com.linkedin.hoptimator.flink.runner.functions.Greet' + - CREATE FUNCTION IF NOT EXISTS `STR_LEN` AS 'com.linkedin.hoptimator.flink.runner.functions.StringLength' + - CREATE FUNCTION IF NOT EXISTS `REVERSE_STR` AS 'demo_udfs.reverse_string' LANGUAGE PYTHON + - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () + - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') + - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `GREET`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` + files: {} !specify PAGE_VIEWS # Clean up diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id index 133d4b3e..4fcf89c0 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id @@ -34,27 +34,23 @@ create function py_udf returns varchar as 'my_module.my_func' language python; # Verify pipeline SQL includes CREATE FUNCTION for all registered UDFs before connectors insert into ads.page_views select my_udf("FIRST_NAME") as page_urn, "MEMBER_URN" as member_urn from profile.members; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: ads-database-pageviews spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE FUNCTION IF NOT EXISTS `MY_UDF` AS 'com.example.MyUdf' - - CREATE FUNCTION IF NOT EXISTS `INT_UDF` AS 'com.example.IntUdf' - - CREATE FUNCTION IF NOT EXISTS `PY_UDF` AS 'my_module.my_func' LANGUAGE PYTHON - - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () - - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') - - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `MY_UDF`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 1 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE FUNCTION IF NOT EXISTS `MY_UDF` AS 'com.example.MyUdf' + - CREATE FUNCTION IF NOT EXISTS `INT_UDF` AS 'com.example.IntUdf' + - CREATE FUNCTION IF NOT EXISTS `PY_UDF` AS 'my_module.my_func' LANGUAGE PYTHON + - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () + - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') + - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `MY_UDF`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` + files: {} !specify PAGE_VIEWS # Clean up diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl.id b/hoptimator-k8s/src/test/resources/k8s-ddl.id index 251a8bb5..90b4b62f 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl.id @@ -163,106 +163,86 @@ drop view ads.audience; !update insert into ads.page_views select first_name as page_urn, last_name as member_urn from profile.members where first_name = 'Alice'; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: ads-database-pageviews spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () - - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') - - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `FIRST_NAME` AS `PAGE_URN`, `LAST_NAME` AS `MEMBER_URN` FROM `PROFILE`.`MEMBERS` WHERE `FIRST_NAME` = 'Alice' - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 1 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () + - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') + - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `FIRST_NAME` AS `PAGE_URN`, `LAST_NAME` AS `MEMBER_URN` FROM `PROFILE`.`MEMBERS` WHERE `FIRST_NAME` = 'Alice' + files: {} !specify PAGE_VIEWS insert into ads.ad_clicks select * from ads.ad_clicks; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: ads-database-adclicks spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - - CREATE TABLE IF NOT EXISTS `ADS`.`AD_CLICKS_source` (`CAMPAIGN_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') - - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - - CREATE TABLE IF NOT EXISTS `ADS`.`AD_CLICKS_sink` (`CAMPAIGN_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - - INSERT INTO `ADS`.`AD_CLICKS_sink` (`CAMPAIGN_URN`, `MEMBER_URN`) SELECT `CAMPAIGN_URN`, `MEMBER_URN` FROM `ADS`.`AD_CLICKS_source` - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 1 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`AD_CLICKS_source` (`CAMPAIGN_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`AD_CLICKS_sink` (`CAMPAIGN_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') + - INSERT INTO `ADS`.`AD_CLICKS_sink` (`CAMPAIGN_URN`, `MEMBER_URN`) SELECT `CAMPAIGN_URN`, `MEMBER_URN` FROM `ADS`.`AD_CLICKS_source` + files: {} !specify AD_CLICKS create or replace materialized view ads."PAGE_VIEWS$forward-field-order" as select CAMPAIGN_URN as PAGE_URN, MEMBER_URN as MEMBER_URN from ads.ad_clicks; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: ads-database-pageviews-forward-field-order spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - - CREATE TABLE IF NOT EXISTS `ADS`.`AD_CLICKS` (`CAMPAIGN_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') - - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `CAMPAIGN_URN` AS `PAGE_URN`, `MEMBER_URN` FROM `ADS`.`AD_CLICKS` - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 1 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`AD_CLICKS` (`CAMPAIGN_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') + - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `CAMPAIGN_URN` AS `PAGE_URN`, `MEMBER_URN` FROM `ADS`.`AD_CLICKS` + files: {} !specify PAGE_VIEWS create or replace materialized view ads."PAGE_VIEWS$reverse-field-order" as select MEMBER_URN as MEMBER_URN, CAMPAIGN_URN as PAGE_URN from ads.ad_clicks; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: ads-database-pageviews-reverse-field-order spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - - CREATE TABLE IF NOT EXISTS `ADS`.`AD_CLICKS` (`CAMPAIGN_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') - - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - - INSERT INTO `ADS`.`PAGE_VIEWS` (`MEMBER_URN`, `PAGE_URN`) SELECT `MEMBER_URN`, `CAMPAIGN_URN` AS `PAGE_URN` FROM `ADS`.`AD_CLICKS` - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 1 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`AD_CLICKS` (`CAMPAIGN_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') + - INSERT INTO `ADS`.`PAGE_VIEWS` (`MEMBER_URN`, `PAGE_URN`) SELECT `MEMBER_URN`, `CAMPAIGN_URN` AS `PAGE_URN` FROM `ADS`.`AD_CLICKS` + files: {} !specify PAGE_VIEWS create or replace materialized view PROFILE."MEMBERS$test" AS SELECT "PAGE_URN" AS "COMPANY_URN", "MEMBER_URN" FROM ADS.PAGE_VIEWS; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: profile-database-members-test spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') - - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () - - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='blackhole') - - INSERT INTO `PROFILE`.`MEMBERS` (`COMPANY_URN`, `MEMBER_URN`) SELECT `PAGE_URN` AS `COMPANY_URN`, `MEMBER_URN` FROM `ADS`.`PAGE_VIEWS` - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 1 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') + - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () + - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='blackhole') + - INSERT INTO `PROFILE`.`MEMBERS` (`COMPANY_URN`, `MEMBER_URN`) SELECT `PAGE_URN` AS `COMPANY_URN`, `MEMBER_URN` FROM `ADS`.`PAGE_VIEWS` + files: {} !specify MEMBERS diff --git a/hoptimator-k8s/src/test/resources/k8s-metadata.id b/hoptimator-k8s/src/test/resources/k8s-metadata.id index ef1d492c..a9fb0f98 100644 --- a/hoptimator-k8s/src/test/resources/k8s-metadata.id +++ b/hoptimator-k8s/src/test/resources/k8s-metadata.id @@ -64,8 +64,8 @@ select name, failed from "k8s".pipeline_elements order by name; +---------------------------------------------+--------+ | NAME | FAILED | +---------------------------------------------+--------+ -| FlinkSessionJob/ads-database-audience2 | false | -| FlinkSessionJob/ads-database-pages | false | +| SqlJob/ads-database-audience2 | false | +| SqlJob/ads-database-pages | false | | TableTrigger/ads-database-pageviews-trigger | false | +---------------------------------------------+--------+ (3 rows) @@ -76,8 +76,8 @@ select * from "k8s".pipeline_element_map order by element_name, pipeline_name; +---------------------------------------------+---------------+ | ELEMENT_NAME | PIPELINE_NAME | +---------------------------------------------+---------------+ -| FlinkSessionJob/ads-database-audience2 | ads-audience2 | -| FlinkSessionJob/ads-database-pages | ads-pages | +| SqlJob/ads-database-audience2 | ads-audience2 | +| SqlJob/ads-database-pages | ads-pages | | TableTrigger/ads-database-pageviews-trigger | ads-audience2 | | TableTrigger/ads-database-pageviews-trigger | ads-pages | +---------------------------------------------+---------------+ @@ -90,9 +90,9 @@ name = t2.element_name) pe on pl.name = pe.pipeline_name order by pipeline_name, +---------------+---------------------------------------------+----------------+ | PIPELINE_NAME | ELEMENT_NAME | ELEMENT_FAILED | +---------------+---------------------------------------------+----------------+ -| ads-audience2 | FlinkSessionJob/ads-database-audience2 | false | +| ads-audience2 | SqlJob/ads-database-audience2 | false | | ads-audience2 | TableTrigger/ads-database-pageviews-trigger | false | -| ads-pages | FlinkSessionJob/ads-database-pages | false | +| ads-pages | SqlJob/ads-database-pages | false | | ads-pages | TableTrigger/ads-database-pageviews-trigger | false | +---------------+---------------------------------------------+----------------+ (4 rows) diff --git a/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id b/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id index 7c11d61d..dd41b990 100644 --- a/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id +++ b/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id @@ -19,24 +19,20 @@ create or replace table "KAFKA"."create-table-test" ("KEY" VARCHAR, "VALUE" BINA !describe "KAFKA"."create-table-test" insert into KAFKA."create-table-test" select * from KAFKA."existing-topic-2"; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: kafka-database-create-table-test spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE DATABASE IF NOT EXISTS `KAFKA` WITH () - - CREATE TABLE IF NOT EXISTS `KAFKA`.`existing-topic-2` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='existing-topic-2', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') - - CREATE DATABASE IF NOT EXISTS `KAFKA` WITH () - - CREATE TABLE IF NOT EXISTS `KAFKA`.`create-table-test` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='create-table-test', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') - - INSERT INTO `KAFKA`.`create-table-test` (`KEY`, `VALUE`) SELECT `KEY`, `VALUE` FROM `KAFKA`.`existing-topic-2` - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 1 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE DATABASE IF NOT EXISTS `KAFKA` WITH () + - CREATE TABLE IF NOT EXISTS `KAFKA`.`existing-topic-2` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='existing-topic-2', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') + - CREATE DATABASE IF NOT EXISTS `KAFKA` WITH () + - CREATE TABLE IF NOT EXISTS `KAFKA`.`create-table-test` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='create-table-test', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') + - INSERT INTO `KAFKA`.`create-table-test` (`KEY`, `VALUE`) SELECT `KEY`, `VALUE` FROM `KAFKA`.`existing-topic-2` + files: {} --- apiVersion: kafka.strimzi.io/v1beta2 kind: KafkaTopic diff --git a/hoptimator-kafka/src/test/resources/kafka-ddl.id b/hoptimator-kafka/src/test/resources/kafka-ddl.id index 32b76d33..85778052 100644 --- a/hoptimator-kafka/src/test/resources/kafka-ddl.id +++ b/hoptimator-kafka/src/test/resources/kafka-ddl.id @@ -22,24 +22,20 @@ drop view kafka."new-topic$kafka-test"; !update insert into kafka."existing-topic-1" select * from kafka."existing-topic-2"; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: kafka-database-existing-topic-1 spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE DATABASE IF NOT EXISTS `KAFKA` WITH () - - CREATE TABLE IF NOT EXISTS `KAFKA`.`existing-topic-2` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'k1'='v1', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='existing-topic-2', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') - - CREATE DATABASE IF NOT EXISTS `KAFKA` WITH () - - CREATE TABLE IF NOT EXISTS `KAFKA`.`existing-topic-1` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'k2'='v2', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='existing-topic-1', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') - - INSERT INTO `KAFKA`.`existing-topic-1` (`KEY`, `VALUE`) SELECT `KEY`, `VALUE` FROM `KAFKA`.`existing-topic-2` - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 2 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE DATABASE IF NOT EXISTS `KAFKA` WITH () + - CREATE TABLE IF NOT EXISTS `KAFKA`.`existing-topic-2` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'k1'='v1', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='existing-topic-2', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') + - CREATE DATABASE IF NOT EXISTS `KAFKA` WITH () + - CREATE TABLE IF NOT EXISTS `KAFKA`.`existing-topic-1` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'k2'='v2', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='existing-topic-1', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') + - INSERT INTO `KAFKA`.`existing-topic-1` (`KEY`, `VALUE`) SELECT `KEY`, `VALUE` FROM `KAFKA`.`existing-topic-2` + files: {} --- apiVersion: kafka.strimzi.io/v1beta2 kind: KafkaTopic diff --git a/hoptimator-mysql/src/test/resources/mysql-ddl.id b/hoptimator-mysql/src/test/resources/mysql-ddl.id index 7dfecfd8..ed39c1de 100644 --- a/hoptimator-mysql/src/test/resources/mysql-ddl.id +++ b/hoptimator-mysql/src/test/resources/mysql-ddl.id @@ -21,24 +21,20 @@ drop materialized view "MYSQL"."testdb"."users$partial"; !update insert into mysql."testdb"."users" ("user_id", "username", "email") select "user_id", 'a' as "username", 'b' as "email" from mysql."testdb"."orders"; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: mysql-users spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE CATALOG IF NOT EXISTS `MYSQL` WITH () - - CREATE DATABASE IF NOT EXISTS `MYSQL`.`testdb` WITH () - - CREATE TABLE IF NOT EXISTS `MYSQL`.`testdb`.`orders` (`order_id` INTEGER, `user_id` INTEGER, `product_name` VARCHAR, `quantity` INTEGER, `price` DECIMAL, `order_date` TIMESTAMP, `status` VARCHAR) WITH ('connector'='mysql', 'hostname'='localhost', 'password'='hoptimatorpassword', 'port'='3306', 'tables'='orders', 'username'='hoptimator') - - CREATE CATALOG IF NOT EXISTS `MYSQL` WITH () - - CREATE DATABASE IF NOT EXISTS `MYSQL`.`testdb` WITH () - - CREATE TABLE IF NOT EXISTS `MYSQL`.`testdb`.`users` (`user_id` INTEGER, `username` VARCHAR, `email` VARCHAR, `created_at` TIMESTAMP, `is_active` BOOLEAN) WITH ('connector'='mysql', 'hostname'='localhost', 'password'='hoptimatorpassword', 'port'='3306', 'tables'='users', 'username'='hoptimator') - - INSERT INTO `MYSQL`.`testdb`.`users` (`user_id`, `username`, `email`) SELECT `user_id`, 'a' AS `username`, 'b' AS `email` FROM `MYSQL`.`testdb`.`orders` - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 1 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE CATALOG IF NOT EXISTS `MYSQL` WITH () + - CREATE DATABASE IF NOT EXISTS `MYSQL`.`testdb` WITH () + - CREATE TABLE IF NOT EXISTS `MYSQL`.`testdb`.`orders` (`order_id` INTEGER, `user_id` INTEGER, `product_name` VARCHAR, `quantity` INTEGER, `price` DECIMAL, `order_date` TIMESTAMP, `status` VARCHAR) WITH ('connector'='mysql', 'hostname'='localhost', 'password'='hoptimatorpassword', 'port'='3306', 'tables'='orders', 'username'='hoptimator') + - CREATE CATALOG IF NOT EXISTS `MYSQL` WITH () + - CREATE DATABASE IF NOT EXISTS `MYSQL`.`testdb` WITH () + - CREATE TABLE IF NOT EXISTS `MYSQL`.`testdb`.`users` (`user_id` INTEGER, `username` VARCHAR, `email` VARCHAR, `created_at` TIMESTAMP, `is_active` BOOLEAN) WITH ('connector'='mysql', 'hostname'='localhost', 'password'='hoptimatorpassword', 'port'='3306', 'tables'='users', 'username'='hoptimator') + - INSERT INTO `MYSQL`.`testdb`.`users` (`user_id`, `username`, `email`) SELECT `user_id`, 'a' AS `username`, 'b' AS `email` FROM `MYSQL`.`testdb`.`orders` + files: {} !specify users diff --git a/hoptimator-venice/src/test/resources/venice-ddl-create-table.id b/hoptimator-venice/src/test/resources/venice-ddl-create-table.id index c6f243dc..2879136f 100644 --- a/hoptimator-venice/src/test/resources/venice-ddl-create-table.id +++ b/hoptimator-venice/src/test/resources/venice-ddl-create-table.id @@ -66,24 +66,20 @@ Key schema evolution is not supported in Venice. Store integration-test-store ha # Test insert operation with existing store insert into "VENICE"."integration-test-store" ("KEY_id", "i") select "KEY", "intField" AS "i" from "VENICE"."test-store-primitive"; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: venice-integration-test-store spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE DATABASE IF NOT EXISTS `VENICE` WITH () - - CREATE TABLE IF NOT EXISTS `VENICE`.`test-store-primitive` (`KEY` INTEGER, `intField` INTEGER, `stringField` VARCHAR) WITH ('connector'='venice', 'key.fields'='KEY', 'key.fields-prefix'='', 'key.type'='PRIMITIVE', 'partial-update-mode'='true', 'storeName'='test-store-primitive', 'value.fields-include'='EXCEPT_KEY') - - CREATE DATABASE IF NOT EXISTS `VENICE` WITH () - - CREATE TABLE IF NOT EXISTS `VENICE`.`integration-test-store` (`KEY_id` INTEGER, `i` INTEGER, `s` VARCHAR, `new_field` DOUBLE) WITH ('connector'='venice', 'key.fields'='KEY_id', 'key.fields-prefix'='KEY_', 'key.type'='RECORD', 'partial-update-mode'='true', 'storeName'='integration-test-store', 'value.fields-include'='EXCEPT_KEY') - - INSERT INTO `VENICE`.`integration-test-store` (`KEY_id`, `i`) SELECT `KEY` AS `KEY_id`, `intField` AS `i` FROM `VENICE`.`test-store-primitive` - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 1 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE DATABASE IF NOT EXISTS `VENICE` WITH () + - CREATE TABLE IF NOT EXISTS `VENICE`.`test-store-primitive` (`KEY` INTEGER, `intField` INTEGER, `stringField` VARCHAR) WITH ('connector'='venice', 'key.fields'='KEY', 'key.fields-prefix'='', 'key.type'='PRIMITIVE', 'partial-update-mode'='true', 'storeName'='test-store-primitive', 'value.fields-include'='EXCEPT_KEY') + - CREATE DATABASE IF NOT EXISTS `VENICE` WITH () + - CREATE TABLE IF NOT EXISTS `VENICE`.`integration-test-store` (`KEY_id` INTEGER, `i` INTEGER, `s` VARCHAR, `new_field` DOUBLE) WITH ('connector'='venice', 'key.fields'='KEY_id', 'key.fields-prefix'='KEY_', 'key.type'='RECORD', 'partial-update-mode'='true', 'storeName'='integration-test-store', 'value.fields-include'='EXCEPT_KEY') + - INSERT INTO `VENICE`.`integration-test-store` (`KEY_id`, `i`) SELECT `KEY` AS `KEY_id`, `intField` AS `i` FROM `VENICE`.`test-store-primitive` + files: {} !specify integration-test-store # Clean up - drop tables diff --git a/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id b/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id index 3f07ef60..0ed33a24 100644 --- a/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id +++ b/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id @@ -22,22 +22,18 @@ drop materialized view "VENICE"."test-store$insert-partial"; !update insert into "VENICE"."test-store" ("KEY_id", "intField") select "KEY", "intField" from "VENICE"."test-store-primitive"; -apiVersion: flink.apache.org/v1beta1 -kind: FlinkSessionJob +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob metadata: name: venice-test-store spec: - deploymentName: basic-session-deployment - job: - entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner - args: - - CREATE DATABASE IF NOT EXISTS `VENICE` WITH () - - CREATE TABLE IF NOT EXISTS `VENICE`.`test-store-primitive` (`KEY` INTEGER, `intField` INTEGER, `stringField` VARCHAR) WITH ('connector'='venice', 'key.fields'='KEY', 'key.fields-prefix'='', 'key.type'='PRIMITIVE', 'partial-update-mode'='true', 'storeName'='test-store-primitive', 'value.fields-include'='EXCEPT_KEY') - - CREATE DATABASE IF NOT EXISTS `VENICE` WITH () - - CREATE TABLE IF NOT EXISTS `VENICE`.`test-store` (`KEY_id` INTEGER, `intField` INTEGER, `stringField` VARCHAR) WITH ('connector'='venice', 'key.fields'='KEY_id', 'key.fields-prefix'='KEY_', 'key.type'='RECORD', 'partial-update-mode'='true', 'storeName'='test-store', 'value.fields-include'='EXCEPT_KEY') - - INSERT INTO `VENICE`.`test-store` (`KEY_id`, `intField`) SELECT `KEY` AS `KEY_id`, `intField` FROM `VENICE`.`test-store-primitive` - jarURI: file:///opt/hoptimator-flink-runner.jar - parallelism: 1 - upgradeMode: stateless - state: running + dialect: Flink + executionMode: Streaming + sql: + - CREATE DATABASE IF NOT EXISTS `VENICE` WITH () + - CREATE TABLE IF NOT EXISTS `VENICE`.`test-store-primitive` (`KEY` INTEGER, `intField` INTEGER, `stringField` VARCHAR) WITH ('connector'='venice', 'key.fields'='KEY', 'key.fields-prefix'='', 'key.type'='PRIMITIVE', 'partial-update-mode'='true', 'storeName'='test-store-primitive', 'value.fields-include'='EXCEPT_KEY') + - CREATE DATABASE IF NOT EXISTS `VENICE` WITH () + - CREATE TABLE IF NOT EXISTS `VENICE`.`test-store` (`KEY_id` INTEGER, `intField` INTEGER, `stringField` VARCHAR) WITH ('connector'='venice', 'key.fields'='KEY_id', 'key.fields-prefix'='KEY_', 'key.type'='RECORD', 'partial-update-mode'='true', 'storeName'='test-store', 'value.fields-include'='EXCEPT_KEY') + - INSERT INTO `VENICE`.`test-store` (`KEY_id`, `intField`) SELECT `KEY` AS `KEY_id`, `intField` FROM `VENICE`.`test-store-primitive` + files: {} !specify test-store From 851d7c0e60c0b0e642981c28797f5fc613792e71 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Mon, 23 Mar 2026 17:12:50 -0500 Subject: [PATCH 05/14] Revive hoptimator-flink-adapter: SqlJob controller in the build The flink-adapter module was orphaned from the build (not in settings.gradle). Revive it so the SqlJob -> FlinkSessionJob reconciler is compiled, packaged, and deployed with the operator. - Add hoptimator-flink-adapter to settings.gradle - Add as runtimeOnly dependency in hoptimator-operator-integration (discovered via SPI ControllerProvider) - Rewrite FlinkControllerProvider and FlinkStreamingSqlJobReconciler to use current K8sContext/K8sApi pattern (was using old Operator API) - Fix build.gradle dependency aliases (libs.kubernetes.client) - Add hoptimator-util dependency for Api interface Co-Authored-By: Claude Opus 4.6 (1M context) --- hoptimator-flink-adapter/build.gradle | 5 +- .../flink/FlinkControllerProvider.java | 20 +++-- .../flink/FlinkStreamingSqlJobReconciler.java | 83 +++++++++---------- hoptimator-operator-integration/build.gradle | 1 + settings.gradle | 1 + 5 files changed, 53 insertions(+), 57 deletions(-) diff --git a/hoptimator-flink-adapter/build.gradle b/hoptimator-flink-adapter/build.gradle index 1e7a7ddb..a51a921d 100644 --- a/hoptimator-flink-adapter/build.gradle +++ b/hoptimator-flink-adapter/build.gradle @@ -8,8 +8,9 @@ dependencies { implementation project(':hoptimator-catalog') implementation project(':hoptimator-k8s') implementation project(':hoptimator-operator') - implementation libs.kubernetesClient - implementation libs.kubernetesExtendedClient + implementation project(':hoptimator-util') + implementation libs.kubernetes.client + implementation libs.kubernetes.extended.client testImplementation libs.assertj } diff --git a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkControllerProvider.java b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkControllerProvider.java index bc1f3e2e..daf0d38e 100644 --- a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkControllerProvider.java +++ b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkControllerProvider.java @@ -1,14 +1,16 @@ package com.linkedin.hoptimator.operator.flink; +import com.linkedin.hoptimator.k8s.K8sApiEndpoint; +import com.linkedin.hoptimator.k8s.K8sContext; import com.linkedin.hoptimator.k8s.models.V1alpha1SqlJob; import com.linkedin.hoptimator.k8s.models.V1alpha1SqlJobList; import com.linkedin.hoptimator.operator.ControllerProvider; -import com.linkedin.hoptimator.operator.Operator; import io.kubernetes.client.extended.controller.Controller; import io.kubernetes.client.extended.controller.builder.ControllerBuilder; import io.kubernetes.client.extended.controller.reconciler.Reconciler; +import java.time.Duration; import java.util.Collection; import java.util.Collections; @@ -16,16 +18,16 @@ /** A bridge to flink-kubernetes-operator */ public class FlinkControllerProvider implements ControllerProvider { - @Override - public Collection controllers(Operator operator) { - operator.registerApi("FlinkDeployment", "flinkdeployment", "flinkdeployments", "flink.apache.org", "v1beta1"); - operator.registerApi("FlinkSessionJob", "flinksessionjob", "flinksessionjobs", "flink.apache.org", "v1beta1"); + public static final K8sApiEndpoint SQL_JOBS = + new K8sApiEndpoint<>("SqlJob", "hoptimator.linkedin.com", "v1alpha1", "sqljobs", false, + V1alpha1SqlJob.class, V1alpha1SqlJobList.class); - operator.registerApi("SqlJob", "sqljob", "sqljobs", "hoptimator.linkedin.com", "v1alpha1", V1alpha1SqlJob.class, - V1alpha1SqlJobList.class); + @Override + public Collection controllers(K8sContext context) { + context.registerInformer(SQL_JOBS, Duration.ofMinutes(5)); - Reconciler reconciler = new FlinkStreamingSqlJobReconciler(operator); - Controller controller = ControllerBuilder.defaultBuilder(operator.informerFactory()) + Reconciler reconciler = new FlinkStreamingSqlJobReconciler(context); + Controller controller = ControllerBuilder.defaultBuilder(context.informerFactory()) .withReconciler(reconciler) .withName("flink-streaming-sql-job-controller") .withWorkerCount(1) diff --git a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java index 086e7290..63272e55 100644 --- a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java +++ b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java @@ -2,36 +2,44 @@ import com.linkedin.hoptimator.catalog.Resource; import com.linkedin.hoptimator.catalog.flink.FlinkStreamingSqlJob; +import com.linkedin.hoptimator.k8s.K8sApi; +import com.linkedin.hoptimator.k8s.K8sContext; +import com.linkedin.hoptimator.k8s.K8sYamlApi; import com.linkedin.hoptimator.k8s.models.V1alpha1SqlJob; +import com.linkedin.hoptimator.k8s.models.V1alpha1SqlJobList; import com.linkedin.hoptimator.k8s.models.V1alpha1SqlJobSpec.DialectEnum; import com.linkedin.hoptimator.k8s.models.V1alpha1SqlJobSpec.ExecutionModeEnum; import com.linkedin.hoptimator.k8s.models.V1alpha1SqlJobStatus; -import com.linkedin.hoptimator.operator.Operator; import io.kubernetes.client.extended.controller.reconciler.Reconciler; import io.kubernetes.client.extended.controller.reconciler.Request; import io.kubernetes.client.extended.controller.reconciler.Result; +import io.kubernetes.client.util.generic.dynamic.DynamicKubernetesObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.sql.SQLException; +import java.time.Duration; import java.util.List; import java.util.Map; import java.util.stream.Collectors; /** - * Manifests streaming SqlJobs as Flink jobs. - * + * Manifests streaming SqlJobs as Flink session jobs. */ public class FlinkStreamingSqlJobReconciler implements Reconciler { private static final Logger log = LoggerFactory.getLogger(FlinkStreamingSqlJobReconciler.class); - private static final String SQLJOB = "hoptimator.linkedin.com/v1alpha1/SqlJob"; - private final Operator operator; + private final K8sContext context; + private final K8sApi sqlJobApi; + private final K8sYamlApi yamlApi; - public FlinkStreamingSqlJobReconciler(Operator operator) { - this.operator = operator; + public FlinkStreamingSqlJobReconciler(K8sContext context) { + this.context = context; + this.sqlJobApi = new K8sApi<>(context, FlinkControllerProvider.SQL_JOBS); + this.yamlApi = new K8sYamlApi(context); } @Override @@ -39,14 +47,17 @@ public Result reconcile(Request request) { log.info("Reconciling request {}", request); String name = request.getName(); String namespace = request.getNamespace(); - Result result = new Result(true, operator.pendingRetryDuration()); try { - V1alpha1SqlJob object = operator.fetch(SQLJOB, namespace, name); - - if (object == null) { - log.info("Object {}/{} deleted. Skipping."); - return new Result(false); + V1alpha1SqlJob object; + try { + object = sqlJobApi.get(namespace, name); + } catch (SQLException e) { + if (e.getErrorCode() == 404) { + log.info("Object {} deleted. Skipping.", name); + return new Result(false); + } + throw e; } V1alpha1SqlJobStatus status = object.getStatus(); @@ -55,9 +66,6 @@ public Result reconcile(Request request) { object.setStatus(status); } - List sql = object.getSpec().getSql(); - String script = sql.stream().collect(Collectors.joining(";\n")); - DialectEnum dialect = object.getSpec().getDialect(); if (!DialectEnum.FLINK.equals(dialect)) { log.info("Not Flink SQL. Skipping."); @@ -70,43 +78,26 @@ public Result reconcile(Request request) { return new Result(false); } - Resource.TemplateFactory templateFactory = new Resource.SimpleTemplateFactory(Resource.Environment.EMPTY); + List sql = object.getSpec().getSql(); + String script = sql.stream().collect(Collectors.joining(";\n")); Map files = object.getSpec().getFiles(); - Resource sqlJob = new FlinkStreamingSqlJob(namespace, name, script, files); - boolean allReady = true; - boolean anyFailed = false; - for (String yaml : sqlJob.render(templateFactory)) { - operator.apply(yaml, object); - if (!operator.isReady(yaml)) { - allReady = false; - } - if (operator.isFailed(yaml)) { - anyFailed = true; - allReady = false; - } - } - object.getStatus().setReady(allReady); - object.getStatus().setFailed(anyFailed); + Resource.TemplateFactory templateFactory = new Resource.SimpleTemplateFactory(Resource.Environment.EMPTY); + Resource sqlJob = new FlinkStreamingSqlJob(namespace, name, script, files); - if (allReady) { - object.getStatus().setMessage("Ready."); - result = new Result(false); // done - } - if (anyFailed) { - object.getStatus().setMessage("Failed."); - result = new Result(false); // done + for (String yaml : sqlJob.render(templateFactory)) { + DynamicKubernetesObject obj = yamlApi.objFromYaml(yaml); + context.own(obj); + yamlApi.update(obj); } - operator.apiFor(SQLJOB) - .updateStatus(object, x -> object.getStatus()) - .onFailure((x, y) -> log.error("Failed to update status of SqlJob {}: {}.", name, y.getMessage())) - .throwsApiException(); + object.getStatus().setReady(true); + object.getStatus().setMessage("Ready."); + sqlJobApi.updateStatus(object, object.getStatus()); } catch (Exception e) { log.error("Encountered exception while reconciling Flink streaming SqlJob {}/{}", namespace, name, e); - return new Result(true, operator.failureRetryDuration()); + return new Result(true, Duration.ofMinutes(5)); } - return result; + return new Result(false); } } - diff --git a/hoptimator-operator-integration/build.gradle b/hoptimator-operator-integration/build.gradle index 92211ab6..8fb8c952 100644 --- a/hoptimator-operator-integration/build.gradle +++ b/hoptimator-operator-integration/build.gradle @@ -6,6 +6,7 @@ plugins { dependencies { implementation project(':hoptimator-operator') + runtimeOnly project(':hoptimator-flink-adapter') implementation libs.slf4j.simple testImplementation libs.assertj diff --git a/settings.gradle b/settings.gradle index 326a82ef..883d4a18 100644 --- a/settings.gradle +++ b/settings.gradle @@ -6,6 +6,7 @@ include 'hoptimator-avro' include 'hoptimator-catalog' // <-- marked for deletion include 'hoptimator-cli' include 'hoptimator-demodb' +include 'hoptimator-flink-adapter' include 'hoptimator-flink-runner' include 'hoptimator-jdbc' include 'hoptimator-jdbc-driver' From fd4c18a692b147c1e7c1443e0314f60219fe5fda Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Mon, 23 Mar 2026 17:27:29 -0500 Subject: [PATCH 06/14] Fix SpotBugs: make UDF directory configurable via system property Replace hardcoded absolute path with System.getProperty fallback to satisfy SpotBugs DMI_HARDCODED_ABSOLUTE_FILENAME check. Configurable via -Dhoptimator.udf.dir, defaults to /opt/python-udfs. Co-Authored-By: Claude Opus 4.6 (1M context) --- .../com/linkedin/hoptimator/flink/runner/FlinkRunner.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java b/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java index 50e28e2a..dc7a819c 100644 --- a/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java +++ b/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java @@ -17,17 +17,19 @@ public final class FlinkRunner { private static final Logger logger = LoggerFactory.getLogger(FlinkRunner.class); static final String FILE_PREFIX = "--file:"; - private static final Path DEFAULT_UDF_DIR = Paths.get("/opt/python-udfs"); + private static final String UDF_DIR_PROPERTY = "hoptimator.udf.dir"; + private static final String UDF_DIR_DEFAULT = "/opt/python-udfs"; private FlinkRunner() { } public static void main(String[] args) throws IOException { // Phase 1: Extract and write file directives before executing SQL + Path udfDir = Paths.get(System.getProperty(UDF_DIR_PROPERTY, UDF_DIR_DEFAULT)); for (String arg : args) { String stmt = arg.replaceAll("\\n", "").trim(); if (stmt.startsWith(FILE_PREFIX)) { - writeFile(stmt, DEFAULT_UDF_DIR); + writeFile(stmt, udfDir); } } From bd404b7f949dbc2f80320b05ee05f17b552aef36 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Mon, 23 Mar 2026 21:03:40 -0500 Subject: [PATCH 07/14] Fix integration test expected output: empty files renders as blank The template engine renders an empty map as blank string, not {}. Co-Authored-By: Claude Opus 4.6 (1M context) --- hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id | 2 +- hoptimator-k8s/src/test/resources/k8s-ddl-udf.id | 2 +- hoptimator-k8s/src/test/resources/k8s-ddl.id | 10 +++++----- .../src/test/resources/kafka-ddl-create-table.id | 2 +- hoptimator-kafka/src/test/resources/kafka-ddl.id | 2 +- hoptimator-mysql/src/test/resources/mysql-ddl.id | 2 +- .../src/test/resources/venice-ddl-create-table.id | 2 +- .../src/test/resources/venice-ddl-insert-partial.id | 2 +- 8 files changed, 12 insertions(+), 12 deletions(-) diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id index b08edfaf..6d1b2cea 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id @@ -61,7 +61,7 @@ spec: - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `GREET`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` - files: {} + files: !specify PAGE_VIEWS # Clean up diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id index 4fcf89c0..876d39d7 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id @@ -50,7 +50,7 @@ spec: - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `MY_UDF`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` - files: {} + files: !specify PAGE_VIEWS # Clean up diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl.id b/hoptimator-k8s/src/test/resources/k8s-ddl.id index 90b4b62f..316d795d 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl.id @@ -176,7 +176,7 @@ spec: - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `FIRST_NAME` AS `PAGE_URN`, `LAST_NAME` AS `MEMBER_URN` FROM `PROFILE`.`MEMBERS` WHERE `FIRST_NAME` = 'Alice' - files: {} + files: !specify PAGE_VIEWS insert into ads.ad_clicks select * from ads.ad_clicks; @@ -193,7 +193,7 @@ spec: - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - CREATE TABLE IF NOT EXISTS `ADS`.`AD_CLICKS_sink` (`CAMPAIGN_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`AD_CLICKS_sink` (`CAMPAIGN_URN`, `MEMBER_URN`) SELECT `CAMPAIGN_URN`, `MEMBER_URN` FROM `ADS`.`AD_CLICKS_source` - files: {} + files: !specify AD_CLICKS create or replace materialized view ads."PAGE_VIEWS$forward-field-order" as select CAMPAIGN_URN as PAGE_URN, MEMBER_URN as MEMBER_URN from ads.ad_clicks; @@ -210,7 +210,7 @@ spec: - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `CAMPAIGN_URN` AS `PAGE_URN`, `MEMBER_URN` FROM `ADS`.`AD_CLICKS` - files: {} + files: !specify PAGE_VIEWS create or replace materialized view ads."PAGE_VIEWS$reverse-field-order" as select MEMBER_URN as MEMBER_URN, CAMPAIGN_URN as PAGE_URN from ads.ad_clicks; @@ -227,7 +227,7 @@ spec: - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`PAGE_VIEWS` (`MEMBER_URN`, `PAGE_URN`) SELECT `MEMBER_URN`, `CAMPAIGN_URN` AS `PAGE_URN` FROM `ADS`.`AD_CLICKS` - files: {} + files: !specify PAGE_VIEWS create or replace materialized view PROFILE."MEMBERS$test" AS SELECT "PAGE_URN" AS "COMPANY_URN", "MEMBER_URN" FROM ADS.PAGE_VIEWS; @@ -244,5 +244,5 @@ spec: - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `PROFILE`.`MEMBERS` (`COMPANY_URN`, `MEMBER_URN`) SELECT `PAGE_URN` AS `COMPANY_URN`, `MEMBER_URN` FROM `ADS`.`PAGE_VIEWS` - files: {} + files: !specify MEMBERS diff --git a/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id b/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id index dd41b990..348a9184 100644 --- a/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id +++ b/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id @@ -32,7 +32,7 @@ spec: - CREATE DATABASE IF NOT EXISTS `KAFKA` WITH () - CREATE TABLE IF NOT EXISTS `KAFKA`.`create-table-test` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='create-table-test', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') - INSERT INTO `KAFKA`.`create-table-test` (`KEY`, `VALUE`) SELECT `KEY`, `VALUE` FROM `KAFKA`.`existing-topic-2` - files: {} + files: --- apiVersion: kafka.strimzi.io/v1beta2 kind: KafkaTopic diff --git a/hoptimator-kafka/src/test/resources/kafka-ddl.id b/hoptimator-kafka/src/test/resources/kafka-ddl.id index 85778052..5b7a289c 100644 --- a/hoptimator-kafka/src/test/resources/kafka-ddl.id +++ b/hoptimator-kafka/src/test/resources/kafka-ddl.id @@ -35,7 +35,7 @@ spec: - CREATE DATABASE IF NOT EXISTS `KAFKA` WITH () - CREATE TABLE IF NOT EXISTS `KAFKA`.`existing-topic-1` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'k2'='v2', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='existing-topic-1', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') - INSERT INTO `KAFKA`.`existing-topic-1` (`KEY`, `VALUE`) SELECT `KEY`, `VALUE` FROM `KAFKA`.`existing-topic-2` - files: {} + files: --- apiVersion: kafka.strimzi.io/v1beta2 kind: KafkaTopic diff --git a/hoptimator-mysql/src/test/resources/mysql-ddl.id b/hoptimator-mysql/src/test/resources/mysql-ddl.id index ed39c1de..33de54de 100644 --- a/hoptimator-mysql/src/test/resources/mysql-ddl.id +++ b/hoptimator-mysql/src/test/resources/mysql-ddl.id @@ -36,5 +36,5 @@ spec: - CREATE DATABASE IF NOT EXISTS `MYSQL`.`testdb` WITH () - CREATE TABLE IF NOT EXISTS `MYSQL`.`testdb`.`users` (`user_id` INTEGER, `username` VARCHAR, `email` VARCHAR, `created_at` TIMESTAMP, `is_active` BOOLEAN) WITH ('connector'='mysql', 'hostname'='localhost', 'password'='hoptimatorpassword', 'port'='3306', 'tables'='users', 'username'='hoptimator') - INSERT INTO `MYSQL`.`testdb`.`users` (`user_id`, `username`, `email`) SELECT `user_id`, 'a' AS `username`, 'b' AS `email` FROM `MYSQL`.`testdb`.`orders` - files: {} + files: !specify users diff --git a/hoptimator-venice/src/test/resources/venice-ddl-create-table.id b/hoptimator-venice/src/test/resources/venice-ddl-create-table.id index 2879136f..2f24a193 100644 --- a/hoptimator-venice/src/test/resources/venice-ddl-create-table.id +++ b/hoptimator-venice/src/test/resources/venice-ddl-create-table.id @@ -79,7 +79,7 @@ spec: - CREATE DATABASE IF NOT EXISTS `VENICE` WITH () - CREATE TABLE IF NOT EXISTS `VENICE`.`integration-test-store` (`KEY_id` INTEGER, `i` INTEGER, `s` VARCHAR, `new_field` DOUBLE) WITH ('connector'='venice', 'key.fields'='KEY_id', 'key.fields-prefix'='KEY_', 'key.type'='RECORD', 'partial-update-mode'='true', 'storeName'='integration-test-store', 'value.fields-include'='EXCEPT_KEY') - INSERT INTO `VENICE`.`integration-test-store` (`KEY_id`, `i`) SELECT `KEY` AS `KEY_id`, `intField` AS `i` FROM `VENICE`.`test-store-primitive` - files: {} + files: !specify integration-test-store # Clean up - drop tables diff --git a/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id b/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id index f80403d5..e3394785 100644 --- a/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id +++ b/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id @@ -36,5 +36,5 @@ spec: - CREATE DATABASE IF NOT EXISTS `VENICE` WITH () - CREATE TABLE IF NOT EXISTS `VENICE`.`test-store` (`KEY_name` VARCHAR, `KEY_age` INTEGER, `intField` INTEGER, `stringField` VARCHAR) WITH ('connector'='venice', 'key.fields'='KEY_name;KEY_age', 'key.fields-prefix'='KEY_', 'key.type'='RECORD', 'partial-update-mode'='true', 'storeName'='test-store', 'value.fields-include'='EXCEPT_KEY') - INSERT INTO `VENICE`.`test-store` (`KEY_name`, `KEY_age`, `intField`) SELECT `stringField` AS `KEY_name`, `KEY` AS `KEY_age`, `intField` FROM `VENICE`.`test-store-primitive` - files: {} + files: !specify test-store From 768ec267eb9a53e83d493f1391c5e866938b4fe7 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Tue, 24 Mar 2026 11:21:42 -0500 Subject: [PATCH 08/14] Fix integration test: empty files map renders as indented {} SnakeYAML dumps an empty map as "{}\n", which the template engine renders as an indented {} on a separate line after "files:". Co-Authored-By: Claude Opus 4.6 (1M context) --- hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id | 1 + hoptimator-k8s/src/test/resources/k8s-ddl-udf.id | 1 + hoptimator-k8s/src/test/resources/k8s-ddl.id | 5 +++++ .../src/test/resources/kafka-ddl-create-table.id | 1 + hoptimator-kafka/src/test/resources/kafka-ddl.id | 1 + hoptimator-mysql/src/test/resources/mysql-ddl.id | 1 + .../src/test/resources/venice-ddl-create-table.id | 1 + .../src/test/resources/venice-ddl-insert-partial.id | 1 + 8 files changed, 12 insertions(+) diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id index 6d1b2cea..ba0ae6a8 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id @@ -62,6 +62,7 @@ spec: - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `GREET`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` files: + {} !specify PAGE_VIEWS # Clean up diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id index 876d39d7..80e6f24d 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id @@ -51,6 +51,7 @@ spec: - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `MY_UDF`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` files: + {} !specify PAGE_VIEWS # Clean up diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl.id b/hoptimator-k8s/src/test/resources/k8s-ddl.id index 316d795d..2fb5277d 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl.id @@ -177,6 +177,7 @@ spec: - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `FIRST_NAME` AS `PAGE_URN`, `LAST_NAME` AS `MEMBER_URN` FROM `PROFILE`.`MEMBERS` WHERE `FIRST_NAME` = 'Alice' files: + {} !specify PAGE_VIEWS insert into ads.ad_clicks select * from ads.ad_clicks; @@ -194,6 +195,7 @@ spec: - CREATE TABLE IF NOT EXISTS `ADS`.`AD_CLICKS_sink` (`CAMPAIGN_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`AD_CLICKS_sink` (`CAMPAIGN_URN`, `MEMBER_URN`) SELECT `CAMPAIGN_URN`, `MEMBER_URN` FROM `ADS`.`AD_CLICKS_source` files: + {} !specify AD_CLICKS create or replace materialized view ads."PAGE_VIEWS$forward-field-order" as select CAMPAIGN_URN as PAGE_URN, MEMBER_URN as MEMBER_URN from ads.ad_clicks; @@ -211,6 +213,7 @@ spec: - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `CAMPAIGN_URN` AS `PAGE_URN`, `MEMBER_URN` FROM `ADS`.`AD_CLICKS` files: + {} !specify PAGE_VIEWS create or replace materialized view ads."PAGE_VIEWS$reverse-field-order" as select MEMBER_URN as MEMBER_URN, CAMPAIGN_URN as PAGE_URN from ads.ad_clicks; @@ -228,6 +231,7 @@ spec: - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`PAGE_VIEWS` (`MEMBER_URN`, `PAGE_URN`) SELECT `MEMBER_URN`, `CAMPAIGN_URN` AS `PAGE_URN` FROM `ADS`.`AD_CLICKS` files: + {} !specify PAGE_VIEWS create or replace materialized view PROFILE."MEMBERS$test" AS SELECT "PAGE_URN" AS "COMPANY_URN", "MEMBER_URN" FROM ADS.PAGE_VIEWS; @@ -245,4 +249,5 @@ spec: - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `PROFILE`.`MEMBERS` (`COMPANY_URN`, `MEMBER_URN`) SELECT `PAGE_URN` AS `COMPANY_URN`, `MEMBER_URN` FROM `ADS`.`PAGE_VIEWS` files: + {} !specify MEMBERS diff --git a/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id b/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id index 348a9184..e4c7ff62 100644 --- a/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id +++ b/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id @@ -33,6 +33,7 @@ spec: - CREATE TABLE IF NOT EXISTS `KAFKA`.`create-table-test` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='create-table-test', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') - INSERT INTO `KAFKA`.`create-table-test` (`KEY`, `VALUE`) SELECT `KEY`, `VALUE` FROM `KAFKA`.`existing-topic-2` files: + {} --- apiVersion: kafka.strimzi.io/v1beta2 kind: KafkaTopic diff --git a/hoptimator-kafka/src/test/resources/kafka-ddl.id b/hoptimator-kafka/src/test/resources/kafka-ddl.id index 5b7a289c..3c84c004 100644 --- a/hoptimator-kafka/src/test/resources/kafka-ddl.id +++ b/hoptimator-kafka/src/test/resources/kafka-ddl.id @@ -36,6 +36,7 @@ spec: - CREATE TABLE IF NOT EXISTS `KAFKA`.`existing-topic-1` (`KEY` VARCHAR, `VALUE` BINARY) WITH ('connector'='kafka', 'k2'='v2', 'key.fields'='KEY', 'key.format'='raw', 'properties.bootstrap.servers'='one-kafka-bootstrap.kafka.svc.cluster.local:9094', 'scan.startup.mode'='earliest-offset', 'topic'='existing-topic-1', 'value.fields-include'='EXCEPT_KEY', 'value.format'='json') - INSERT INTO `KAFKA`.`existing-topic-1` (`KEY`, `VALUE`) SELECT `KEY`, `VALUE` FROM `KAFKA`.`existing-topic-2` files: + {} --- apiVersion: kafka.strimzi.io/v1beta2 kind: KafkaTopic diff --git a/hoptimator-mysql/src/test/resources/mysql-ddl.id b/hoptimator-mysql/src/test/resources/mysql-ddl.id index 33de54de..29f8f646 100644 --- a/hoptimator-mysql/src/test/resources/mysql-ddl.id +++ b/hoptimator-mysql/src/test/resources/mysql-ddl.id @@ -37,4 +37,5 @@ spec: - CREATE TABLE IF NOT EXISTS `MYSQL`.`testdb`.`users` (`user_id` INTEGER, `username` VARCHAR, `email` VARCHAR, `created_at` TIMESTAMP, `is_active` BOOLEAN) WITH ('connector'='mysql', 'hostname'='localhost', 'password'='hoptimatorpassword', 'port'='3306', 'tables'='users', 'username'='hoptimator') - INSERT INTO `MYSQL`.`testdb`.`users` (`user_id`, `username`, `email`) SELECT `user_id`, 'a' AS `username`, 'b' AS `email` FROM `MYSQL`.`testdb`.`orders` files: + {} !specify users diff --git a/hoptimator-venice/src/test/resources/venice-ddl-create-table.id b/hoptimator-venice/src/test/resources/venice-ddl-create-table.id index 2f24a193..86d77320 100644 --- a/hoptimator-venice/src/test/resources/venice-ddl-create-table.id +++ b/hoptimator-venice/src/test/resources/venice-ddl-create-table.id @@ -80,6 +80,7 @@ spec: - CREATE TABLE IF NOT EXISTS `VENICE`.`integration-test-store` (`KEY_id` INTEGER, `i` INTEGER, `s` VARCHAR, `new_field` DOUBLE) WITH ('connector'='venice', 'key.fields'='KEY_id', 'key.fields-prefix'='KEY_', 'key.type'='RECORD', 'partial-update-mode'='true', 'storeName'='integration-test-store', 'value.fields-include'='EXCEPT_KEY') - INSERT INTO `VENICE`.`integration-test-store` (`KEY_id`, `i`) SELECT `KEY` AS `KEY_id`, `intField` AS `i` FROM `VENICE`.`test-store-primitive` files: + {} !specify integration-test-store # Clean up - drop tables diff --git a/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id b/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id index e3394785..76a9fd74 100644 --- a/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id +++ b/hoptimator-venice/src/test/resources/venice-ddl-insert-partial.id @@ -37,4 +37,5 @@ spec: - CREATE TABLE IF NOT EXISTS `VENICE`.`test-store` (`KEY_name` VARCHAR, `KEY_age` INTEGER, `intField` INTEGER, `stringField` VARCHAR) WITH ('connector'='venice', 'key.fields'='KEY_name;KEY_age', 'key.fields-prefix'='KEY_', 'key.type'='RECORD', 'partial-update-mode'='true', 'storeName'='test-store', 'value.fields-include'='EXCEPT_KEY') - INSERT INTO `VENICE`.`test-store` (`KEY_name`, `KEY_age`, `intField`) SELECT `stringField` AS `KEY_name`, `KEY` AS `KEY_age`, `intField` FROM `VENICE`.`test-store-primitive` files: + {} !specify test-store From 6436bb909d6a056d80305fc4c992871070a8c4e5 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Tue, 24 Mar 2026 12:33:24 -0500 Subject: [PATCH 09/14] Trim trailing newline from SnakeYAML map dump in template engine SnakeYAML's dump() appends a trailing newline to its output (e.g., "{}\n" for an empty map). The template engine's multiline expansion converts this into a spurious whitespace-only line. Trimming the output fixes the rendering of {{files}} and other map variables. Co-Authored-By: Claude Opus 4.6 (1M context) --- .../src/main/java/com/linkedin/hoptimator/util/Template.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/Template.java b/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/Template.java index 57e00c81..433fa49a 100644 --- a/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/Template.java +++ b/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/Template.java @@ -121,7 +121,7 @@ public String getOrDefault(String key, ThrowingSupplier f) throws SQLExc } private String formatMapAsString(Map configMap) { - return new Yaml().dump(configMap); + return new Yaml().dump(configMap).trim(); } private String formatPropertiesAsString(Properties props) { From 6d2a7ac665d395734a01499d9857c065e4878a8c Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Tue, 24 Mar 2026 21:22:31 -0500 Subject: [PATCH 10/14] FlinkRunner fetches SQL and files from SqlJob via K8s API Replace the --file: encoding mechanism with the production pattern: FlinkRunner receives --sqljob=namespace/name and fetches the SqlJob CR directly from the K8s API to get SQL statements and UDF files. - FlinkRunner uses DynamicKubernetesApi to fetch SqlJob CR - Extracts spec.sql (statements) and spec.files (UDF code) - Writes files to UDF directory, then executes SQL - Falls back to SQL-from-args for backward compatibility - Reconciler simplified: just passes SqlJob reference to template - FlinkStreamingSqlJob reduced to namespace+name export - RBAC added for Flink SA to read SqlJob CRs Co-Authored-By: Claude Opus 4.6 (1M context) --- Makefile | 1 + deploy/dev/flink-sqljob-rbac.yaml | 24 ++++ .../catalog/flink/FlinkStreamingSqlJob.java | 27 +--- .../flink/FlinkStreamingSqlJobReconciler.java | 12 +- .../FlinkStreamingSqlJob.yaml.template | 2 +- hoptimator-flink-runner/build.gradle | 1 + .../hoptimator/flink/runner/FlinkRunner.java | 125 +++++++++++++----- .../flink/runner/FlinkRunnerTest.java | 36 +++-- 8 files changed, 142 insertions(+), 86 deletions(-) create mode 100644 deploy/dev/flink-sqljob-rbac.yaml diff --git a/Makefile b/Makefile index 771dbf89..996cdd78 100644 --- a/Makefile +++ b/Makefile @@ -56,6 +56,7 @@ deploy-flink: deploy helm upgrade --install --atomic --set webhook.create=false,image.pullPolicy=Never,image.repository=docker.io/library/hoptimator-flink-operator,image.tag=latest --set-json='watchNamespaces=["default","flink"]' flink-kubernetes-operator flink-operator-repo/flink-kubernetes-operator kubectl apply -f ./deploy/dev/flink-session-cluster.yaml kubectl apply -f ./deploy/dev/flink-sql-gateway.yaml + kubectl apply -f ./deploy/dev/flink-sqljob-rbac.yaml kubectl apply -f ./deploy/samples/flink-template.yaml kubectl apply -f ./deploy/samples/flink-beam-template.yaml diff --git a/deploy/dev/flink-sqljob-rbac.yaml b/deploy/dev/flink-sqljob-rbac.yaml new file mode 100644 index 00000000..57105801 --- /dev/null +++ b/deploy/dev/flink-sqljob-rbac.yaml @@ -0,0 +1,24 @@ +## Grants the Flink service account read access to SqlJob CRs, +## so FlinkRunner can fetch SQL and files at runtime. + +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + name: sqljob-reader +rules: +- apiGroups: ["hoptimator.linkedin.com"] + resources: ["sqljobs"] + verbs: ["get"] +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRoleBinding +metadata: + name: flink-sqljob-reader +roleRef: + apiGroup: rbac.authorization.k8s.io + kind: ClusterRole + name: sqljob-reader +subjects: +- kind: ServiceAccount + name: flink + namespace: flink diff --git a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java index bc5a58c0..73116c64 100644 --- a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java +++ b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/catalog/flink/FlinkStreamingSqlJob.java @@ -2,37 +2,12 @@ import com.linkedin.hoptimator.catalog.Resource; -import java.nio.charset.StandardCharsets; -import java.util.Base64; -import java.util.Map; - public class FlinkStreamingSqlJob extends Resource { - public FlinkStreamingSqlJob(String namespace, String name, String sql) { + public FlinkStreamingSqlJob(String namespace, String name) { super("FlinkStreamingSqlJob"); export("namespace", namespace); export("name", name); - export("sql", sql); - } - - public FlinkStreamingSqlJob(String namespace, String name, String sql, Map files) { - this(namespace, name, prependFiles(sql, files)); - } - - /** Prepend --file directives to the SQL script so FlinkRunner can extract and write them. */ - private static String prependFiles(String sql, Map files) { - if (files == null || files.isEmpty()) { - return sql; - } - StringBuilder sb = new StringBuilder(); - for (Map.Entry entry : files.entrySet()) { - String encoded = Base64.getEncoder().encodeToString( - entry.getValue().getBytes(StandardCharsets.UTF_8)); - sb.append("--file:").append(entry.getKey()).append(":").append(encoded); - sb.append(";\n"); - } - sb.append(sql); - return sb.toString(); } } diff --git a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java index 63272e55..b5a20f4b 100644 --- a/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java +++ b/hoptimator-flink-adapter/src/main/java/com/linkedin/hoptimator/operator/flink/FlinkStreamingSqlJobReconciler.java @@ -21,13 +21,13 @@ import java.sql.SQLException; import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; /** * Manifests streaming SqlJobs as Flink session jobs. + * + *

The reconciler creates a FlinkSessionJob that references the SqlJob by name. + * The FlinkRunner fetches SQL and files directly from the SqlJob CR at runtime. */ public class FlinkStreamingSqlJobReconciler implements Reconciler { private static final Logger log = LoggerFactory.getLogger(FlinkStreamingSqlJobReconciler.class); @@ -78,12 +78,8 @@ public Result reconcile(Request request) { return new Result(false); } - List sql = object.getSpec().getSql(); - String script = sql.stream().collect(Collectors.joining(";\n")); - Map files = object.getSpec().getFiles(); - Resource.TemplateFactory templateFactory = new Resource.SimpleTemplateFactory(Resource.Environment.EMPTY); - Resource sqlJob = new FlinkStreamingSqlJob(namespace, name, script, files); + Resource sqlJob = new FlinkStreamingSqlJob(namespace, name); for (String yaml : sqlJob.render(templateFactory)) { DynamicKubernetesObject obj = yamlApi.objFromYaml(yaml); diff --git a/hoptimator-flink-adapter/src/main/resources/FlinkStreamingSqlJob.yaml.template b/hoptimator-flink-adapter/src/main/resources/FlinkStreamingSqlJob.yaml.template index 7945fc86..d16674ba 100644 --- a/hoptimator-flink-adapter/src/main/resources/FlinkStreamingSqlJob.yaml.template +++ b/hoptimator-flink-adapter/src/main/resources/FlinkStreamingSqlJob.yaml.template @@ -7,7 +7,7 @@ spec: job: entryClass: com.linkedin.hoptimator.flink.runner.FlinkRunner args: - - {{sql}} + - --sqljob={{namespace}}/{{name}} jarURI: file:///opt/hoptimator-flink-runner.jar parallelism: 1 upgradeMode: stateless diff --git a/hoptimator-flink-runner/build.gradle b/hoptimator-flink-runner/build.gradle index 12b44695..db4d0e0f 100644 --- a/hoptimator-flink-runner/build.gradle +++ b/hoptimator-flink-runner/build.gradle @@ -10,6 +10,7 @@ dependencies { implementation libs.flink.connector.kafka implementation libs.flink.connector.mysql.cdc implementation libs.flink.csv + implementation libs.kubernetes.client //Flink depends should be provided by runtime compileOnly libs.flink.table.common diff --git a/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java b/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java index dc7a819c..4868c294 100644 --- a/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java +++ b/hoptimator-flink-runner/src/main/java/com/linkedin/hoptimator/flink/runner/FlinkRunner.java @@ -6,65 +6,128 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import io.kubernetes.client.openapi.ApiClient; +import io.kubernetes.client.util.Config; +import io.kubernetes.client.util.generic.dynamic.DynamicKubernetesApi; +import io.kubernetes.client.util.generic.dynamic.DynamicKubernetesObject; + import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; -import java.util.Base64; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; -/** Runs SQL from command-line args. */ +/** Runs SQL from a SqlJob CR or from command-line args. */ public final class FlinkRunner { private static final Logger logger = LoggerFactory.getLogger(FlinkRunner.class); - static final String FILE_PREFIX = "--file:"; + private static final String SQLJOB_PREFIX = "--sqljob="; private static final String UDF_DIR_PROPERTY = "hoptimator.udf.dir"; private static final String UDF_DIR_DEFAULT = "/opt/python-udfs"; private FlinkRunner() { } - public static void main(String[] args) throws IOException { - // Phase 1: Extract and write file directives before executing SQL - Path udfDir = Paths.get(System.getProperty(UDF_DIR_PROPERTY, UDF_DIR_DEFAULT)); - for (String arg : args) { - String stmt = arg.replaceAll("\\n", "").trim(); - if (stmt.startsWith(FILE_PREFIX)) { - writeFile(stmt, udfDir); + public static void main(String[] args) throws Exception { + List sqlStatements; + + // Check for --sqljob argument to fetch SQL and files from a SqlJob CR + String sqlJobRef = findSqlJobRef(args); + if (sqlJobRef != null) { + logger.info("Fetching SqlJob: {}", sqlJobRef); + sqlStatements = loadFromSqlJob(sqlJobRef); + } else { + // Backward compatibility: SQL from command-line args + logger.info("No --sqljob argument found. Using SQL from command-line args."); + sqlStatements = new ArrayList<>(); + for (String arg : args) { + String stmt = arg.replaceAll("\\n", "").trim(); + if (!stmt.isEmpty() && !stmt.startsWith("--")) { + sqlStatements.add(stmt); + } } } - // Phase 2: Execute SQL statements + // Execute SQL statements EnvironmentSettings settings = EnvironmentSettings.newInstance().inStreamingMode().build(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, settings); - for (int i = 0; i < args.length; i++) { - String stmt = args[i].replaceAll("\\n", "").trim(); - if (!stmt.isEmpty() && !stmt.startsWith("--")) { - logger.info("Executing statement #{}: {}", i, stmt); - try { - tEnv.executeSql(stmt); - } catch (Exception e) { - logger.error("Error executing SQL statement #{}: `{}`", i, stmt, e); - throw e; - } + for (int i = 0; i < sqlStatements.size(); i++) { + String stmt = sqlStatements.get(i); + logger.info("Executing statement #{}: {}", i, stmt); + try { + tEnv.executeSql(stmt); + } catch (Exception e) { + logger.error("Error executing SQL statement #{}: `{}`", i, stmt, e); + throw e; + } + } + } + + /** Finds the --sqljob=namespace/name argument, or null if not present. */ + static String findSqlJobRef(String[] args) { + for (String arg : args) { + if (arg.startsWith(SQLJOB_PREFIX)) { + return arg.substring(SQLJOB_PREFIX.length()); } } + return null; } - /** Parses a --file:name:base64content directive and writes the file to the given directory. */ - static void writeFile(String directive, Path targetDir) throws IOException { - String payload = directive.substring(FILE_PREFIX.length()); - int colonIdx = payload.indexOf(':'); - if (colonIdx < 0) { - throw new IllegalArgumentException("Invalid file directive (missing ':'): " + directive); + /** Fetches a SqlJob CR and returns the SQL statements, writing any files to the UDF directory. */ + @SuppressWarnings("unchecked") + static List loadFromSqlJob(String ref) throws Exception { + String[] parts = ref.split("/", 2); + if (parts.length != 2) { + throw new IllegalArgumentException("SqlJob reference must be namespace/name, got: " + ref); } - String filename = payload.substring(0, colonIdx); - String encoded = payload.substring(colonIdx + 1); - byte[] content = Base64.getDecoder().decode(encoded); + String namespace = parts[0]; + String name = parts[1]; + + ApiClient client = Config.defaultClient(); + DynamicKubernetesApi api = new DynamicKubernetesApi("hoptimator.linkedin.com", "v1alpha1", "sqljobs", client); + DynamicKubernetesObject obj = api.get(namespace, name).throwsApiException().getObject(); + + Map spec = (Map) obj.getRaw().get("spec"); + if (spec == null) { + throw new IllegalStateException("SqlJob " + ref + " has no spec"); + } + + // Extract and execute files + Map files = (Map) spec.get("files"); + if (files != null && !files.isEmpty()) { + Path udfDir = Paths.get(System.getProperty(UDF_DIR_PROPERTY, UDF_DIR_DEFAULT)); + for (Map.Entry entry : files.entrySet()) { + writeFile(entry.getKey(), entry.getValue(), udfDir); + } + } + + // Extract SQL statements + List sql = (List) spec.get("sql"); + if (sql == null) { + return Collections.emptyList(); + } + + List statements = new ArrayList<>(); + for (String stmt : sql) { + String trimmed = stmt.replaceAll("\\n", "").trim(); + if (!trimmed.isEmpty()) { + statements.add(trimmed); + } + } + return statements; + } + + /** Writes a file to the given directory. */ + static void writeFile(String filename, String content, Path targetDir) throws IOException { Files.createDirectories(targetDir); Path filePath = targetDir.resolve(filename); - Files.write(filePath, content); + Files.write(filePath, content.getBytes(StandardCharsets.UTF_8)); logger.info("Wrote UDF file: {}", filePath); } } diff --git a/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/FlinkRunnerTest.java b/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/FlinkRunnerTest.java index 313a569c..2ce352d0 100644 --- a/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/FlinkRunnerTest.java +++ b/hoptimator-flink-runner/src/test/java/com/linkedin/hoptimator/flink/runner/FlinkRunnerTest.java @@ -1,7 +1,6 @@ package com.linkedin.hoptimator.flink.runner; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -9,41 +8,38 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; -import java.util.Base64; class FlinkRunnerTest { @Test - void writeFileDecodesBase64AndWritesToDisk(@TempDir Path tempDir) throws Exception { - String content = "print('hello world')"; - String encoded = Base64.getEncoder().encodeToString(content.getBytes(StandardCharsets.UTF_8)); - String directive = "--file:test.py:" + encoded; + void findSqlJobRefPresent() { + String[] args = {"--sqljob=default/my-job", "CREATE TABLE t (x INT)"}; + assertThat(FlinkRunner.findSqlJobRef(args)).isEqualTo("default/my-job"); + } - FlinkRunner.writeFile(directive, tempDir); + @Test + void findSqlJobRefAbsent() { + String[] args = {"CREATE TABLE t (x INT)", "INSERT INTO t SELECT 1"}; + assertThat(FlinkRunner.findSqlJobRef(args)).isNull(); + } + + @Test + void writeFileCreatesFileOnDisk(@TempDir Path tempDir) throws Exception { + FlinkRunner.writeFile("test.py", "print('hello')", tempDir); Path written = tempDir.resolve("test.py"); assertThat(written).exists(); - assertThat(Files.readString(written)).isEqualTo(content); + assertThat(Files.readString(written, StandardCharsets.UTF_8)).isEqualTo("print('hello')"); } @Test void writeFileHandlesMultilineContent(@TempDir Path tempDir) throws Exception { String content = "from pyflink.table.udf import udf\n\n@udf(result_type=DataTypes.STRING())\ndef reverse(s):\n return s[::-1]\n"; - String encoded = Base64.getEncoder().encodeToString(content.getBytes(StandardCharsets.UTF_8)); - String directive = "--file:my_udfs.py:" + encoded; - - FlinkRunner.writeFile(directive, tempDir); + FlinkRunner.writeFile("my_udfs.py", content, tempDir); Path written = tempDir.resolve("my_udfs.py"); assertThat(written).exists(); - assertThat(Files.readString(written)).isEqualTo(content); - } - - @Test - void writeFileRejectsMalformedDirective(@TempDir Path tempDir) { - assertThatThrownBy(() -> FlinkRunner.writeFile("--file:no-colon-after-name", tempDir)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("missing ':'"); + assertThat(Files.readString(written, StandardCharsets.UTF_8)).isEqualTo(content); } } From d864f5cb8dca19b5bab429de09806b16bb43339e Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Wed, 25 Mar 2026 08:37:05 -0500 Subject: [PATCH 11/14] Add integration test for SqlJob with inline UDF files Tests that CREATE FUNCTION with CODE option produces a SqlJob containing the files map. The inline Python code is extracted from the CODE option and mapped to a filename derived from the AS clause (e.g., 'my_udf.transform' -> 'my_udf.py'). Co-Authored-By: Claude Opus 4.6 (1M context) --- .../hoptimator/k8s/TestSqlScripts.java | 5 ++ .../src/test/resources/k8s-ddl-udf-files.id | 47 +++++++++++++++++++ 2 files changed, 52 insertions(+) create mode 100644 hoptimator-k8s/src/test/resources/k8s-ddl-udf-files.id diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java index 9d54d441..4eb7722f 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java @@ -29,6 +29,11 @@ public void k8sDdlScriptUdfDemo() throws Exception { run("k8s-ddl-udf-demo.id"); } + @Test + public void k8sDdlScriptUdfFiles() throws Exception { + run("k8s-ddl-udf-files.id"); + } + @Test public void k8sValidationScript() throws Exception { run("k8s-validation.id"); diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-files.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-files.id new file mode 100644 index 00000000..5509596e --- /dev/null +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-files.id @@ -0,0 +1,47 @@ +!set outputformat mysql +!use k8s + +# Register a Python UDF with inline CODE that should appear in SqlJob files +create function my_py_udf as 'my_udf.transform' language python with (CODE='x = 1'); +(0 rows modified) + +!update + +# UDF validates against real data (opaque placeholder returns null) +select my_py_udf("FIRST_NAME") as val from profile.members; ++-----+ +| VAL | ++-----+ +| | +| | +| | ++-----+ +(3 rows) + +!ok + +# Pipeline generates SqlJob with files containing the inline CODE +insert into ads.page_views select my_py_udf("FIRST_NAME") as page_urn, "MEMBER_URN" as member_urn from profile.members; +apiVersion: hoptimator.linkedin.com/v1alpha1 +kind: SqlJob +metadata: + name: ads-database-pageviews +spec: + dialect: Flink + executionMode: Streaming + sql: + - CREATE FUNCTION IF NOT EXISTS `MY_PY_UDF` AS 'my_udf.transform' LANGUAGE PYTHON + - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () + - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') + - CREATE DATABASE IF NOT EXISTS `ADS` WITH () + - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') + - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `MY_PY_UDF`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` + files: + {my_udf.py: x = 1} +!specify PAGE_VIEWS + +# Clean up +drop function my_py_udf; +(0 rows modified) + +!update From b82d83c912d2ef1a37b0eac7ee7e5a964aaef415 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Wed, 25 Mar 2026 09:27:14 -0500 Subject: [PATCH 12/14] Clean CREATE FUNCTION syntax with dollar-quoting and AS/IN semantics New syntax: CREATE FUNCTION foo [RETURNS type] [LANGUAGE lang] AS 'callable' [IN ] - LANGUAGE moves before AS for natural reading - AS names the callable entry point (class for Java, function for Python) - IN provides the source: module name or $$-delimited inline code - $$...$$ dollar-quoting avoids escaping issues for code blocks Implementation: - DollarQuoting preprocessor converts $$...$$ to single-quoted strings before Calcite parses the SQL (hooked into PARSER_FACTORY) - Grammar updated in parserImpls.ftl and generated parser - DDL executor: when LANGUAGE + IN present, auto-detects inline code (whitespace) vs module reference, derives module.function path Examples: -- Java class reference (unchanged) CREATE FUNCTION greet AS 'com.example.Greet'; -- Python module reference CREATE FUNCTION foo LANGUAGE PYTHON AS 'func' IN 'module'; -- Python inline code with dollar-quoting CREATE FUNCTION foo LANGUAGE PYTHON AS 'foo' IN $$ def foo(s): return s[::-1] $$; Co-Authored-By: Claude Opus 4.6 (1M context) --- .../src/main/codegen/includes/parserImpls.ftl | 2 +- .../hoptimator/jdbc/DollarQuoting.java | 69 +++++++++++++++++++ .../jdbc/HoptimatorDdlExecutor.java | 24 +++++-- .../jdbc/ddl/HoptimatorDdlParserImpl.java | 10 +-- .../src/test/resources/create-function-ddl.id | 25 ++++++- .../src/test/resources/k8s-ddl-udf-demo.id | 2 +- .../src/test/resources/k8s-ddl-udf-files.id | 10 +-- .../src/test/resources/k8s-ddl-udf.id | 2 +- 8 files changed, 125 insertions(+), 19 deletions(-) create mode 100644 hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/DollarQuoting.java diff --git a/hoptimator-jdbc/src/main/codegen/includes/parserImpls.ftl b/hoptimator-jdbc/src/main/codegen/includes/parserImpls.ftl index db572cb0..b4840416 100644 --- a/hoptimator-jdbc/src/main/codegen/includes/parserImpls.ftl +++ b/hoptimator-jdbc/src/main/codegen/includes/parserImpls.ftl @@ -341,9 +341,9 @@ SqlCreate SqlCreateFunction(Span s, boolean replace) : ifNotExists = IfNotExistsOpt() id = CompoundIdentifier() [ returnType = DataType() ] + [ language = SimpleIdentifier() ] template = StringLiteral() [ namespace = StringLiteral() ] - [ language = SimpleIdentifier() ] [ optionList = Options() ] { return new SqlCreateFunction(s.end(this), replace, ifNotExists, id, template, diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/DollarQuoting.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/DollarQuoting.java new file mode 100644 index 00000000..90dfb83b --- /dev/null +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/DollarQuoting.java @@ -0,0 +1,69 @@ +package com.linkedin.hoptimator.jdbc; + +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.io.UncheckedIOException; + + +/** + * Preprocesses SQL to convert $$-delimited strings to standard single-quoted strings. + * + *

This enables PostgreSQL-style dollar-quoting for inline code blocks, e.g.: + *

{@code
+ * CREATE FUNCTION foo LANGUAGE PYTHON AS 'foo' IN $$
+ * def foo(s):
+ *     return s[::-1]
+ * $$;
+ * }
+ * + *

The preprocessor converts {@code $$...$$} to {@code '...'}, escaping any + * internal single quotes by doubling them ({@code '} → {@code ''}). + */ +final class DollarQuoting { + + private DollarQuoting() { + } + + /** Preprocesses a Reader, converting $$-delimited strings to single-quoted SQL strings. */ + static Reader preprocess(Reader reader) { + try { + StringBuilder sb = new StringBuilder(); + char[] buf = new char[4096]; + int n; + while ((n = reader.read(buf)) != -1) { + sb.append(buf, 0, n); + } + return new StringReader(preprocess(sb.toString())); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + /** Converts $$-delimited strings to single-quoted SQL strings. */ + static String preprocess(String sql) { + int idx = sql.indexOf("$$"); + if (idx < 0) { + return sql; + } + StringBuilder result = new StringBuilder(); + int i = 0; + while (i < sql.length()) { + if (i + 1 < sql.length() && sql.charAt(i) == '$' && sql.charAt(i + 1) == '$') { + int end = sql.indexOf("$$", i + 2); + if (end < 0) { + throw new IllegalArgumentException("Unclosed $$ delimiter in SQL"); + } + String content = sql.substring(i + 2, end); + result.append('\''); + result.append(content.replace("'", "''")); + result.append('\''); + i = end + 2; + } else { + result.append(sql.charAt(i)); + i++; + } + } + return result.toString(); + } +} diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java index f53afc74..800ef1f8 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java @@ -115,7 +115,8 @@ public HoptimatorDdlExecutor(HoptimatorConnection connection) { public static final SqlParserImplFactory PARSER_FACTORY = new SqlParserImplFactory() { @Override public SqlAbstractParserImpl getParser(Reader stream) { - SqlAbstractParserImpl parser = HoptimatorDdlParserImpl.FACTORY.getParser(stream); + Reader preprocessed = DollarQuoting.preprocess(stream); + SqlAbstractParserImpl parser = HoptimatorDdlParserImpl.FACTORY.getParser(preprocessed); parser.setConformance(SqlConformanceEnum.BABEL); return parser; } @@ -383,16 +384,31 @@ public void execute(SqlCreateFunction create, CalcitePrepare.Context context) { String name = create.name.names.get(create.name.names.size() - 1); String as = ((SqlLiteral) create.job).getValueAs(String.class); - String namespace = create.namespace != null + String inClause = create.namespace != null ? ((SqlLiteral) create.namespace).getValueAs(String.class) : null; Map options = HoptimatorDdlUtils.options(create.options); - // Extract LANGUAGE from the dedicated clause (elevated from WITH options) + // Extract LANGUAGE from the dedicated clause if (create.language != null) { options.put("LANGUAGE", create.language.getSimple()); } - UserFunction userFunction = new UserFunction(name, as, namespace, options); + // When LANGUAGE is present with an IN clause, AS names the callable and IN provides the source. + // If IN contains whitespace, it's inline code; otherwise it's a module reference. + if (create.language != null && inClause != null) { + if (inClause.contains(" ") || inClause.contains("\n")) { + // Inline code: derive module.function reference from function name + String funcName = as; + String moduleName = name.toLowerCase(java.util.Locale.ROOT); + as = moduleName + "." + funcName; + options.put("CODE", inClause); + } else { + // Module reference: IN names the module, AS names the function within it + as = inClause + "." + as; + } + } + + UserFunction userFunction = new UserFunction(name, as, null, options); // Determine return type from RETURNS clause (default: VARCHAR) String returnsName = create.returnType != null diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/HoptimatorDdlParserImpl.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/HoptimatorDdlParserImpl.java index 4ea70433..f29e6a97 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/HoptimatorDdlParserImpl.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ddl/HoptimatorDdlParserImpl.java @@ -6378,6 +6378,11 @@ final public SqlCreate SqlCreateFunction(Span s, boolean replace) throws ParseEx jj_consume_token(RETURNS); returnType = DataType(); } + // optional: LANGUAGE (before AS) + if (jj_ntk == LANGUAGE || (jj_ntk == -1 && jj_ntk() == LANGUAGE)) { + jj_consume_token(LANGUAGE); + language = SimpleIdentifier(); + } jj_consume_token(AS); template = StringLiteral(); switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { @@ -6389,11 +6394,6 @@ final public SqlCreate SqlCreateFunction(Span s, boolean replace) throws ParseEx jj_la1[59] = jj_gen; ; } - // optional: LANGUAGE - if (jj_ntk == LANGUAGE || (jj_ntk == -1 && jj_ntk() == LANGUAGE)) { - jj_consume_token(LANGUAGE); - language = SimpleIdentifier(); - } switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { case WITH: optionList = Options(); diff --git a/hoptimator-jdbc/src/test/resources/create-function-ddl.id b/hoptimator-jdbc/src/test/resources/create-function-ddl.id index 9cd01669..413cbfc6 100644 --- a/hoptimator-jdbc/src/test/resources/create-function-ddl.id +++ b/hoptimator-jdbc/src/test/resources/create-function-ddl.id @@ -55,8 +55,8 @@ select bool_udf(x) as val from (values ('a')) as t(x); !ok -# Test CREATE FUNCTION with LANGUAGE clause -create function py_udf returns varchar as 'my_module.my_func' language python; +# Test CREATE FUNCTION with LANGUAGE clause (LANGUAGE before AS) +create function py_udf returns varchar language python as 'my_func' in 'my_module'; (0 rows modified) !update @@ -73,6 +73,22 @@ select py_udf(x, y) as val from (values ('a', 'b'), ('c', 'd')) as t(x, y); !ok +# Test CREATE FUNCTION with inline code via dollar-quoting +create function inline_udf returns varchar language python as 'inline_udf' in $$x = 1$$; +(0 rows modified) + +!update + +select inline_udf(x) as val from (values ('a')) as t(x); ++-----+ +| VAL | ++-----+ +| | ++-----+ +(1 row) + +!ok + # Clean up drop function my_udf; (0 rows modified) @@ -93,3 +109,8 @@ drop function py_udf; (0 rows modified) !update + +drop function inline_udf; +(0 rows modified) + +!update diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id index ba0ae6a8..fa2f53d1 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-demo.id @@ -13,7 +13,7 @@ create function str_len returns integer as 'com.linkedin.hoptimator.flink.runner !update # Register demo Python UDF baked into the Flink runner image -create function reverse_str as 'demo_udfs.reverse_string' language python; +create function reverse_str language python as 'reverse_string' in 'demo_udfs'; (0 rows modified) !update diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-files.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-files.id index 5509596e..becb3d64 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl-udf-files.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf-files.id @@ -1,8 +1,8 @@ !set outputformat mysql !use k8s -# Register a Python UDF with inline CODE that should appear in SqlJob files -create function my_py_udf as 'my_udf.transform' language python with (CODE='x = 1'); +# Register a Python UDF with inline code using dollar-quoting +create function my_py_udf returns varchar language python as 'my_py_udf' in $$x = 1$$; (0 rows modified) !update @@ -20,7 +20,7 @@ select my_py_udf("FIRST_NAME") as val from profile.members; !ok -# Pipeline generates SqlJob with files containing the inline CODE +# Pipeline generates SqlJob with files containing the inline code insert into ads.page_views select my_py_udf("FIRST_NAME") as page_urn, "MEMBER_URN" as member_urn from profile.members; apiVersion: hoptimator.linkedin.com/v1alpha1 kind: SqlJob @@ -30,14 +30,14 @@ spec: dialect: Flink executionMode: Streaming sql: - - CREATE FUNCTION IF NOT EXISTS `MY_PY_UDF` AS 'my_udf.transform' LANGUAGE PYTHON + - CREATE FUNCTION IF NOT EXISTS `MY_PY_UDF` AS 'my_py_udf.my_py_udf' LANGUAGE PYTHON - CREATE DATABASE IF NOT EXISTS `PROFILE` WITH () - CREATE TABLE IF NOT EXISTS `PROFILE`.`MEMBERS` (`FIRST_NAME` VARCHAR, `LAST_NAME` VARCHAR, `MEMBER_URN` VARCHAR, `COMPANY_URN` VARCHAR) WITH ('connector'='datagen', 'number-of-rows'='10') - CREATE DATABASE IF NOT EXISTS `ADS` WITH () - CREATE TABLE IF NOT EXISTS `ADS`.`PAGE_VIEWS` (`PAGE_URN` VARCHAR, `MEMBER_URN` VARCHAR) WITH ('connector'='blackhole') - INSERT INTO `ADS`.`PAGE_VIEWS` (`PAGE_URN`, `MEMBER_URN`) SELECT `MY_PY_UDF`(`FIRST_NAME`) AS `PAGE_URN`, `MEMBER_URN` FROM `PROFILE`.`MEMBERS` files: - {my_udf.py: x = 1} + {my_py_udf.py: x = 1} !specify PAGE_VIEWS # Clean up diff --git a/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id index 80e6f24d..1f5ea198 100644 --- a/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id +++ b/hoptimator-k8s/src/test/resources/k8s-ddl-udf.id @@ -27,7 +27,7 @@ create function int_udf returns integer as 'com.example.IntUdf'; !update # Register a Python UDF with LANGUAGE clause -create function py_udf returns varchar as 'my_module.my_func' language python; +create function py_udf returns varchar language python as 'my_func' in 'my_module'; (0 rows modified) !update From 5099bbf48c5a7adc9b2b912395e5aa014b4d15d9 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Wed, 25 Mar 2026 09:32:47 -0500 Subject: [PATCH 13/14] Support file paths and URLs in CREATE FUNCTION IN clause The IN clause now accepts three forms: - Inline code: IN $$def foo(s): ...$$ - File path: IN '/path/to/udf.py' - URL: IN 'file:///path/to/udf.py' File paths and URLs are detected by the presence of '/' or ':\', read at DDL execution time, and stored as CODE in the function options. The module.function reference is derived from the function name, same as inline code. Examples: CREATE FUNCTION foo LANGUAGE PYTHON AS 'foo' IN '/tmp/my_udfs.py'; CREATE FUNCTION foo LANGUAGE PYTHON AS 'foo' IN 'file:///tmp/my_udfs.py'; Co-Authored-By: Claude Opus 4.6 (1M context) --- .../jdbc/HoptimatorDdlExecutor.java | 28 ++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java index 800ef1f8..a0a8eb7c 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java @@ -394,7 +394,10 @@ public void execute(SqlCreateFunction create, CalcitePrepare.Context context) { } // When LANGUAGE is present with an IN clause, AS names the callable and IN provides the source. - // If IN contains whitespace, it's inline code; otherwise it's a module reference. + // The IN value can be: + // - Inline code (contains whitespace, e.g. from $$...$$) + // - A file path or URL (contains '/' or ':', e.g. '/path/to/udf.py' or 'file:///udf.py') + // - A module reference (simple name, e.g. 'demo_udfs') if (create.language != null && inClause != null) { if (inClause.contains(" ") || inClause.contains("\n")) { // Inline code: derive module.function reference from function name @@ -402,6 +405,13 @@ public void execute(SqlCreateFunction create, CalcitePrepare.Context context) { String moduleName = name.toLowerCase(java.util.Locale.ROOT); as = moduleName + "." + funcName; options.put("CODE", inClause); + } else if (inClause.contains("/") || inClause.contains(":\\")) { + // File path or URL: read content and treat as inline code + String code = readSource(inClause); + String funcName = as; + String moduleName = name.toLowerCase(java.util.Locale.ROOT); + as = moduleName + "." + funcName; + options.put("CODE", code); } else { // Module reference: IN names the module, AS names the function within it as = inClause + "." + as; @@ -864,4 +874,20 @@ String databaseName() { return databaseName; } } + + /** Reads source code from a file path or URL. */ + private static String readSource(String location) { + try { + java.net.URI uri; + if (location.contains("://")) { + uri = new java.net.URI(location); + } else { + uri = java.nio.file.Paths.get(location).toUri(); + } + return new String(java.nio.file.Files.readAllBytes(java.nio.file.Paths.get(uri)), + java.nio.charset.StandardCharsets.UTF_8); + } catch (Exception e) { + throw new RuntimeException("Failed to read source from " + location, e); + } + } } From 01d08143428371690553c106254d555ca1da6eab Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Wed, 25 Mar 2026 09:47:59 -0500 Subject: [PATCH 14/14] Add USING JAR support for Java UDF archives When a function's IN clause points to a JAR archive (detected by .jar extension), the JAR location is stored in the function options and emitted as USING JAR in the generated Flink SQL: CREATE FUNCTION greet AS 'com.example.Greet' USING JAR '/path/to/udfs.jar'; This enables Flink to dynamically load UDF classes from external JARs. JAR references are also included in the SqlJob files map so the data plane can fetch them. Example DDL: CREATE FUNCTION greet AS 'com.example.Greet' IN 'https://artifactory/udfs-1.0.jar'; CREATE FUNCTION greet AS 'com.example.Greet' IN '/opt/libs/udfs.jar'; Co-Authored-By: Claude Opus 4.6 (1M context) --- .../hoptimator/jdbc/HoptimatorDdlExecutor.java | 17 ++++++++++------- .../hoptimator/util/planner/PipelineRel.java | 8 +++++++- .../util/planner/ScriptImplementor.java | 5 +++++ 3 files changed, 22 insertions(+), 8 deletions(-) diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java index a0a8eb7c..639cd284 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java @@ -393,13 +393,16 @@ public void execute(SqlCreateFunction create, CalcitePrepare.Context context) { options.put("LANGUAGE", create.language.getSimple()); } - // When LANGUAGE is present with an IN clause, AS names the callable and IN provides the source. - // The IN value can be: - // - Inline code (contains whitespace, e.g. from $$...$$) - // - A file path or URL (contains '/' or ':', e.g. '/path/to/udf.py' or 'file:///udf.py') - // - A module reference (simple name, e.g. 'demo_udfs') - if (create.language != null && inClause != null) { - if (inClause.contains(" ") || inClause.contains("\n")) { + // The IN clause provides the source for the function. Its interpretation depends on context: + // - JAR archive (.jar extension or URL): stored as JAR option for USING JAR emission + // - Inline code (contains whitespace, e.g. from $$...$$): stored as CODE option + // - File path or URL (contains '/'): read content and stored as CODE option + // - Module reference (simple name, e.g. 'demo_udfs'): combined with AS as module.function + if (inClause != null) { + if (inClause.endsWith(".jar")) { + // JAR archive: AS is the class, IN is the JAR location + options.put("JAR", inClause); + } else if (inClause.contains(" ") || inClause.contains("\n")) { // Inline code: derive module.function reference from function name String funcName = as; String moduleName = name.toLowerCase(java.util.Locale.ROOT); diff --git a/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/PipelineRel.java b/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/PipelineRel.java index e73e07aa..ac3263ba 100644 --- a/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/PipelineRel.java +++ b/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/PipelineRel.java @@ -116,7 +116,7 @@ public Pipeline pipeline(String name, Connection connection) throws SQLException templateEvals.put("query", query(connection)); templateEvals.put("fieldMap", fieldMap()); - // Collect inline file content from functions with CODE option + // Collect inline file content and JAR references from functions Map files = new HashMap<>(); for (UserFunction func : functions) { String code = func.options().get("CODE"); @@ -127,6 +127,12 @@ public Pipeline pipeline(String name, Connection connection) throws SQLException String filename = (dotIdx >= 0 ? module.substring(0, dotIdx) : module) + ".py"; files.put(filename, code); } + String jar = func.options().get("JAR"); + if (jar != null) { + // Store JAR reference as URL value (data plane fetches it) + String jarName = jar.contains("/") ? jar.substring(jar.lastIndexOf('/') + 1) : jar; + files.put(jarName, jar); + } } Job job = new Job(name, sources.keySet(), sink, templateEvals, files); diff --git a/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/ScriptImplementor.java b/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/ScriptImplementor.java index 874f7703..0ff5f451 100644 --- a/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/ScriptImplementor.java +++ b/hoptimator-util/src/main/java/com/linkedin/hoptimator/util/planner/ScriptImplementor.java @@ -550,6 +550,11 @@ public void implement(SqlWriter w) { w.keyword("LANGUAGE"); w.literal(language); } + String jar = options.get("JAR"); + if (jar != null) { + w.keyword("USING JAR"); + w.literal("'" + jar + "'"); + } w.literal(";"); } }