From 6e41c4c6239493adde7bd8639fd1afef4799b48a Mon Sep 17 00:00:00 2001 From: Ferenc Csaky Date: Fri, 5 Jun 2026 19:19:27 +0200 Subject: [PATCH 1/3] feat: Resolve available, non-secret environment variables during `compile` --- sqrl-planner/pom.xml | 4 + .../planner/parser/SqrlStatementParser.java | 71 +++++----- .../planner/util/NonSecretEnvVarResolver.java | 61 +++++++++ .../com/datasqrl/util/ConfigLoaderUtils.java | 37 ++++-- .../util/NonSecretEnvVarResolverTest.java | 124 ++++++++++++++++++ .../resources/usecases/duckdb/package.json | 2 +- 6 files changed, 259 insertions(+), 40 deletions(-) create mode 100644 sqrl-planner/src/main/java/com/datasqrl/planner/util/NonSecretEnvVarResolver.java create mode 100644 sqrl-planner/src/test/java/com/datasqrl/planner/util/NonSecretEnvVarResolverTest.java diff --git a/sqrl-planner/pom.xml b/sqrl-planner/pom.xml index bf3bcebfb..a80b340dc 100644 --- a/sqrl-planner/pom.xml +++ b/sqrl-planner/pom.xml @@ -133,6 +133,10 @@ flink-avro + + com.datasqrl.flinkrunner + env-utils + com.datasqrl.flinkrunner stdlib-utils diff --git a/sqrl-planner/src/main/java/com/datasqrl/planner/parser/SqrlStatementParser.java b/sqrl-planner/src/main/java/com/datasqrl/planner/parser/SqrlStatementParser.java index 5b630ace1..fd455604d 100644 --- a/sqrl-planner/src/main/java/com/datasqrl/planner/parser/SqrlStatementParser.java +++ b/sqrl-planner/src/main/java/com/datasqrl/planner/parser/SqrlStatementParser.java @@ -24,6 +24,7 @@ import com.datasqrl.error.ErrorLocation.FileLocation; import com.datasqrl.planner.hint.PlannerHints; import com.datasqrl.planner.parser.SqrlTableFunctionStatement.ParsedArgument; +import com.datasqrl.planner.util.NonSecretEnvVarResolver; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -122,22 +123,26 @@ public class SqrlStatementParser { * @return */ public List parseScript(String script, ErrorCollector scriptErrors) { - List sqlStatements = new ArrayList<>(); + var parsedStatements = new ArrayList(); + + var envVarResolver = NonSecretEnvVarResolver.builder().strict(false).build(); var localErrors = scriptErrors; + try { var statements = SqlScriptStatementSplitter.splitStatements(script); - for (ParsedObject statement : statements) { - localErrors = scriptErrors.atFile(statement.getFileLocation()); - sqlStatements.add( - new ParsedStatement( - new ParsedObject<>( - parseStatement(statement.getObject()), statement.getFileLocation()), - statement.get())); + for (var stmt : statements) { + localErrors = scriptErrors.atFile(stmt.getFileLocation()); + + var sqlStmt = parseStatement(stmt.getObject(), envVarResolver); + var parsedObj = new ParsedObject<>(sqlStmt, stmt.getFileLocation()); + + parsedStatements.add(new ParsedStatement(parsedObj, stmt.get())); } } catch (StatementParserException e) { throw localErrors.handle(e); } - return sqlStatements; + + return parsedStatements; } /** @@ -146,28 +151,30 @@ public List parseScript(String script, ErrorCollector scriptErr * @param statement * @return */ - public SQLStatement parseStatement(String statement) { - var importExportMatcher = IMPORT_EXPORT.matcher(statement); + SQLStatement parseStatement(String statement, NonSecretEnvVarResolver envVarResolver) { + var resolvedStmt = envVarResolver.resolve(statement); + var importExportMatcher = IMPORT_EXPORT.matcher(resolvedStmt); // #1: Imports and Exports if (importExportMatcher.find()) { // it's an import or export statement var directive = importExportMatcher.group("fullmatch"); var commentEnd = importExportMatcher.end() - directive.length(); - var comment = SqrlComments.parse(parse(statement.substring(0, commentEnd), statement, 0)); - var body = statement.substring(importExportMatcher.end()).trim(); + var comment = + SqrlComments.parse(parse(resolvedStmt.substring(0, commentEnd), resolvedStmt, 0)); + var body = resolvedStmt.substring(importExportMatcher.end()).trim(); if (directive.equalsIgnoreCase("import")) { var subMatcher = IMPORT_PARSER.matcher(body); checkFatal(subMatcher.find(), ErrorCode.INVALID_IMPORT, "Could not parse IMPORT statement"); - var packageIdentifier = parseNamePath(subMatcher, "package", statement); - var alias = parseNamePath(subMatcher, "identifier", statement); + var packageIdentifier = parseNamePath(subMatcher, "package", resolvedStmt); + var alias = parseNamePath(subMatcher, "identifier", resolvedStmt); checkFatal(packageIdentifier.isPresent(), ErrorCode.INVALID_IMPORT, "Missing package path"); return new SqrlImportStatement(packageIdentifier, alias, comment); } else if (directive.equalsIgnoreCase("export")) { var subMatcher = EXPORT_PARSER.matcher(body); checkFatal(subMatcher.find(), ErrorCode.INVALID_IMPORT, "Could not parse IMPORT statement"); - var packageIdentifier = parseNamePath(subMatcher, "package", statement); - var tableName = parseNamePath(subMatcher, "identifier", statement); + var packageIdentifier = parseNamePath(subMatcher, "package", resolvedStmt); + var tableName = parseNamePath(subMatcher, "identifier", resolvedStmt); checkFatal(packageIdentifier.isPresent(), ErrorCode.INVALID_EXPORT, "Missing package path"); checkFatal(packageIdentifier.isPresent(), ErrorCode.INVALID_EXPORT, "Missing table"); return new SqrlExportStatement(tableName, packageIdentifier, comment); @@ -178,11 +185,11 @@ public SQLStatement parseStatement(String statement) { } // #2: SQRL Table Definitions - var sqrlDefinition = SQRL_DEFINITION.matcher(statement); + var sqrlDefinition = SQRL_DEFINITION.matcher(resolvedStmt); if (sqrlDefinition.find()) { // it's a SQRL definition var tableName = parseNamePath( - sqrlDefinition.group("tablename"), statement, sqrlDefinition.start("tablename")); + sqrlDefinition.group("tablename"), resolvedStmt, sqrlDefinition.start("tablename")); checkFatal( tableName.isPresent(), tableName.getFileLocation(), @@ -196,16 +203,17 @@ public SQLStatement parseStatement(String statement) { tableName.get()); var isRelationship = tableName.get().size() > 1; var commentEnd = sqrlDefinition.start("fullmatch"); - var comment = SqrlComments.parse(parse(statement.substring(0, commentEnd), statement, 0)); + var comment = + SqrlComments.parse(parse(resolvedStmt.substring(0, commentEnd), resolvedStmt, 0)); var definitionBody = - parse(statement.substring(sqrlDefinition.end()), statement, sqrlDefinition.end()); + parse(resolvedStmt.substring(sqrlDefinition.end()), resolvedStmt, sqrlDefinition.end()); var access = AccessModifier.QUERY; ParsedObject arguments = - parse(sqrlDefinition, "arguments", statement).map(str -> str.isBlank() ? null : str); + parse(sqrlDefinition, "arguments", resolvedStmt).map(str -> str.isBlank() ? null : str); ParsedObject returnType = - parse(sqrlDefinition, "returntype", statement).map(str -> str.isBlank() ? null : str); + parse(sqrlDefinition, "returntype", resolvedStmt).map(str -> str.isBlank() ? null : str); // Identify SQL keyword var sqlKeywordPattern = Pattern.compile("^\\s*(\\w+)"); var keywordMatcher = sqlKeywordPattern.matcher(definitionBody.get()); @@ -229,7 +237,7 @@ public SQLStatement parseStatement(String statement) { access = AccessModifier.SUBSCRIPTION; var newDefinitionStart = sqrlDefinition.end() + keywordEnd; definitionBody = - parse(statement.substring(newDefinitionStart), statement, newDefinitionStart); + parse(resolvedStmt.substring(newDefinitionStart), resolvedStmt, newDefinitionStart); } if (arguments.isEmpty() && returnType.isEmpty() && !isRelationship) { definition = new SqrlTableDefinition(tableName, definitionBody, access, comment); @@ -289,26 +297,27 @@ public SQLStatement parseStatement(String statement) { } // #3: Create Table - var createTable = CREATE_TABLE.matcher(statement); + var createTable = CREATE_TABLE.matcher(resolvedStmt); if (createTable.find()) { var createTableStmt = parse( - statement.substring(createTable.start("fullmatch")), - statement, + resolvedStmt.substring(createTable.start("fullmatch")), + resolvedStmt, createTable.start("fullmatch")); var commentEnd = createTable.start("fullmatch"); - var comment = SqrlComments.parse(parse(statement.substring(0, commentEnd), statement, 0)); + var comment = + SqrlComments.parse(parse(resolvedStmt.substring(0, commentEnd), resolvedStmt, 0)); return new SqrlCreateTableStatement(createTableStmt, comment); } // #4: Next Batch - var nextBatch = NEXT_BATCH_PARSER.matcher(statement); + var nextBatch = NEXT_BATCH_PARSER.matcher(resolvedStmt); if (nextBatch.find()) { - return new SqrlNextBatch(new ParsedObject<>(statement, FileLocation.START)); + return new SqrlNextBatch(new ParsedObject<>(resolvedStmt, FileLocation.START)); } // #5: If none of the regex match, we assume it's a Flink SQL statement - return new FlinkSQLStatement(new ParsedObject<>(statement, FileLocation.START)); + return new FlinkSQLStatement(new ParsedObject<>(resolvedStmt, FileLocation.START)); } /** diff --git a/sqrl-planner/src/main/java/com/datasqrl/planner/util/NonSecretEnvVarResolver.java b/sqrl-planner/src/main/java/com/datasqrl/planner/util/NonSecretEnvVarResolver.java new file mode 100644 index 000000000..3b26de662 --- /dev/null +++ b/sqrl-planner/src/main/java/com/datasqrl/planner/util/NonSecretEnvVarResolver.java @@ -0,0 +1,61 @@ +/* + * Copyright © 2021 DataSQRL (contact@datasqrl.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.datasqrl.planner.util; + +import com.datasqrl.flinkrunner.utils.EnvVarResolver; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import lombok.experimental.SuperBuilder; + +/** Resolves regular environment variables and converts secret templates to standard templates. */ +@SuperBuilder +public class NonSecretEnvVarResolver extends EnvVarResolver { + + private static final Pattern SECRET_ENVIRONMENT_VARIABLE_PATTERN = + Pattern.compile("\\$\\{\\{(.*?)\\}\\}"); + + private static final Pattern SECRET_ENVIRONMENT_VARIABLE_NAME_PATTERN = + Pattern.compile("[A-Za-z_][A-Za-z0-9_]*"); + + @Override + public String resolve(String src) { + if (src == null || src.isBlank()) { + return src; + } + + var resolvedRegularEnvVars = super.resolve(src); + + return resolveSecretEnvVars(resolvedRegularEnvVars); + } + + private String resolveSecretEnvVars(String src) { + var res = new StringBuilder(); + var matcher = SECRET_ENVIRONMENT_VARIABLE_PATTERN.matcher(src); + while (matcher.find()) { + var key = matcher.group(1); + if (!SECRET_ENVIRONMENT_VARIABLE_NAME_PATTERN.matcher(key).matches()) { + throw new IllegalArgumentException( + "Secret environment variable templates only support variable names, but found: %s" + .formatted(matcher.group())); + } + + matcher.appendReplacement(res, Matcher.quoteReplacement(String.format("${%s}", key))); + } + matcher.appendTail(res); + + return res.toString(); + } +} diff --git a/sqrl-planner/src/main/java/com/datasqrl/util/ConfigLoaderUtils.java b/sqrl-planner/src/main/java/com/datasqrl/util/ConfigLoaderUtils.java index 92a738110..6435362ae 100644 --- a/sqrl-planner/src/main/java/com/datasqrl/util/ConfigLoaderUtils.java +++ b/sqrl-planner/src/main/java/com/datasqrl/util/ConfigLoaderUtils.java @@ -26,6 +26,7 @@ import com.datasqrl.error.ErrorMessage; import com.datasqrl.error.ResourceFileUtil; import com.datasqrl.planner.dag.plan.MutationDatabase; +import com.datasqrl.planner.util.NonSecretEnvVarResolver; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; @@ -101,7 +102,9 @@ static PackageJson loadResolvedConfigFromFile( Files.isRegularFile(packageJson), String.format("Failed to load %s, it is not a regular file", packageJson)); - var objectNode = convertFileToObjectNode(errors, Either.Left(packageJson)); + // This is called by 'run', 'test', and 'exec', so env var resolution will be done by + // DatasqrlRun. + var objectNode = convertFileToObjectNode(errors, Either.Left(packageJson), false); return SqrlConfig.loadResolvedConfig(errors, objectNode, packageSchemaPath); } @@ -294,14 +297,14 @@ static PackageJson loadUnresolvedConfig( if (url == null) { throw errors.withConfig(defaultPath).exception("Default configuration not found"); } - var defaultJson = convertFileToObjectNode(errors, Either.Right(url)); + var defaultJson = convertFileToObjectNode(errors, Either.Right(url), false); valid &= isValidJson(errors, defaultJson, PACKAGE_SCHEMA_PATH); jsons.add(defaultJson); } // Convert, validate, and add files for (Path file : files) { - var json = convertFileToObjectNode(errors, Either.Left(file)); + var json = convertFileToObjectNode(errors, Either.Left(file), true); valid &= isValidJson(errors, json, PACKAGE_SCHEMA_PATH); jsons.add(json); } @@ -326,15 +329,33 @@ private static void validatePlanDir(Path planDir) { Files.isDirectory(planDir), "Failed to load Flink config, plan dir does not exist."); } - private static ObjectNode convertFileToObjectNode(ErrorCollector errors, Either file) { - var local = errors.withConfig(file.isLeft() ? file.left().toString() : file.right().toString()); + private static ObjectNode convertFileToObjectNode( + ErrorCollector errors, Either file, boolean resolveEnvVars) { + var filename = file.isLeft() ? file.left().toString() : file.right().toString(); + var localErr = errors.withConfig(filename); + + ObjectMapper mapper; + if (resolveEnvVars) { + var resolver = NonSecretEnvVarResolver.builder().strict(false).build(); + mapper = resolver.initObjectMapper(MAPPER); + } else { + mapper = MAPPER; + } + try { - var jsonNode = - file.isLeft() ? MAPPER.readTree(file.left().toFile()) : MAPPER.readTree(file.right()); + JsonNode jsonNode; + + if (file.isLeft()) { + jsonNode = mapper.readTree(file.left().toFile()); + + } else { + jsonNode = mapper.readTree(file.right()); + } + return (ObjectNode) jsonNode; } catch (IOException e) { - throw local.exception("Could not parse JSON file [%s]: %s", file, e.toString()); + throw localErr.exception("Could not parse JSON file [%s]: %s", file, e.toString()); } } } diff --git a/sqrl-planner/src/test/java/com/datasqrl/planner/util/NonSecretEnvVarResolverTest.java b/sqrl-planner/src/test/java/com/datasqrl/planner/util/NonSecretEnvVarResolverTest.java new file mode 100644 index 000000000..3a1e7d21f --- /dev/null +++ b/sqrl-planner/src/test/java/com/datasqrl/planner/util/NonSecretEnvVarResolverTest.java @@ -0,0 +1,124 @@ +/* + * Copyright © 2021 DataSQRL (contact@datasqrl.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.datasqrl.planner.util; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.util.Map; +import java.util.stream.Stream; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; + +class NonSecretEnvVarResolverTest { + + private NonSecretEnvVarResolver resolver; + + @ParameterizedTest + @MethodSource("secretEnvTemplates") + void givenSecretEnvTemplate_whenResolve_thenConvertToStandardEnvTemplate( + String command, String expected) { + resolver = NonSecretEnvVarResolver.builder().envVars(Map.of("HOST", "db.example.com")).build(); + + var result = resolver.resolve(command); + + assertThat(result).isEqualTo(expected); + } + + private static Stream secretEnvTemplates() { + return Stream.of( + arguments("${{SECRET}}", "${SECRET}"), + arguments("Token=${{SECRET}}", "Token=${SECRET}"), + arguments( + "Host=${HOST}, password=${{DB_PASSWORD}}", + "Host=db.example.com, password=${DB_PASSWORD}")); + } + + @ParameterizedTest + @ValueSource( + strings = { + "${{TOKEN:-default}}", + "${{TOKEN:=default}}", + "${{TOKEN:default}}", + "${{TOKEN-NAME}}", + "${{}}" + }) + void givenUnsupportedSecretEnvTemplate_whenResolve_thenThrowException(String command) { + resolver = NonSecretEnvVarResolver.builder().envVars(Map.of()).strict(false).build(); + + assertThatThrownBy(() -> resolver.resolve(command)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Secret environment variable templates only support variable names"); + } + + @ParameterizedTest + @MethodSource("regularTemplatesProducingSecretTemplates") + void givenRegularResolutionProducesSecretTemplate_whenResolve_thenConvertToStandardEnvTemplate( + String command, String expected) { + resolver = + NonSecretEnvVarResolver.builder().envVars(Map.of("SECRET_TEMPLATE", "${{SECRET}}")).build(); + + var result = resolver.resolve(command); + + assertThat(result).isEqualTo(expected); + } + + private static Stream regularTemplatesProducingSecretTemplates() { + return Stream.of( + arguments("${SECRET_TEMPLATE}", "${SECRET}"), + arguments("${MISSING:-${{SECRET}}}", "${SECRET}")); + } + + @ParameterizedTest + @MethodSource("regularEnvTemplates") + void givenRegularEnvTemplate_whenResolve_thenDelegateToEnvVarResolver( + String command, String expected) { + resolver = + NonSecretEnvVarResolver.builder().envVars(Map.of("USER", "John")).strict(false).build(); + + var result = resolver.resolve(command); + + assertThat(result).isEqualTo(expected); + } + + private static Stream regularEnvTemplates() { + return Stream.of( + arguments("${USER}", "John"), + arguments("${MISSING:-default}", "default"), + arguments("${MISSING}", "${MISSING}")); + } + + @ParameterizedTest + @MethodSource("jsonSources") + void givenJsonSource_whenResolveInJson_thenResolveSecretStringLeafNodes( + String command, String expected) throws IOException { + resolver = NonSecretEnvVarResolver.builder().envVars(Map.of()).build(); + + var result = resolver.resolveInJson(command); + var json = new ObjectMapper().readTree(result); + + assertThat(json.get("secret").asText()).isEqualTo(expected); + } + + private static Stream jsonSources() { + return Stream.of(arguments("{\"secret\":\"${{SECRET}}\"}", "${SECRET}")); + } +} diff --git a/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/duckdb/package.json b/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/duckdb/package.json index 62cf8d43d..818e8fdb1 100644 --- a/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/duckdb/package.json +++ b/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/duckdb/package.json @@ -23,7 +23,7 @@ }, "connectors": { "iceberg": { - "warehouse": "/tmp/duckdb" + "warehouse": "${TEST_WH}" } }, "test-runner": { From d72b76ae0554a45b1b7ced0dacf4a5cc85b20e4f Mon Sep 17 00:00:00 2001 From: Ferenc Csaky Date: Tue, 9 Jun 2026 18:34:27 +0200 Subject: [PATCH 2/3] rework script and pkg json resolution logic --- pom.xml | 7 ++ .../flink/plan/FlinkSqlNodeFactory.java | 53 ++++++++++++-- .../planner/Sqrl2FlinkSQLTranslator.java | 4 +- .../planner/parser/SqrlStatementParser.java | 71 ++++++++----------- .../planner/tables/FlinkTableBuilder.java | 2 +- .../com/datasqrl/util/ConfigLoaderUtils.java | 52 ++++++++++---- sqrl-testing/sqrl-testing-integration/pom.xml | 5 ++ .../java/com/datasqrl/UseCaseCompileTest.java | 13 +++- .../flink-only-compile-package.txt | 3 +- .../iceberg-export-compile-package.txt | 2 +- .../resources/usecases/duckdb/package.json | 2 +- .../flink-only-compile/flink-only.sqrl | 16 ++--- .../iceberg/iceberg-sink.table.sql | 4 +- 13 files changed, 159 insertions(+), 75 deletions(-) diff --git a/pom.xml b/pom.xml index 2ba89f552..addb1ab6b 100644 --- a/pom.xml +++ b/pom.xml @@ -167,6 +167,7 @@ 4.2.0 2.6 2.2.50 + 2.1.8 2.0.5 5.0.12 @@ -656,6 +657,12 @@ ${testcontainers.version} test + + uk.org.webcompere + system-stubs-jupiter + ${system-stubs.version} + test + diff --git a/sqrl-planner/src/main/java/com/datasqrl/engine/stream/flink/plan/FlinkSqlNodeFactory.java b/sqrl-planner/src/main/java/com/datasqrl/engine/stream/flink/plan/FlinkSqlNodeFactory.java index 38e8d5ece..d0f9cbd70 100644 --- a/sqrl-planner/src/main/java/com/datasqrl/engine/stream/flink/plan/FlinkSqlNodeFactory.java +++ b/sqrl-planner/src/main/java/com/datasqrl/engine/stream/flink/plan/FlinkSqlNodeFactory.java @@ -16,10 +16,11 @@ package com.datasqrl.engine.stream.flink.plan; import com.datasqrl.calcite.schema.sql.SqlDataTypeSpecBuilder; +import com.datasqrl.planner.util.NonSecretEnvVarResolver; import com.datasqrl.sql.SqlCallRewriter; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -42,6 +43,7 @@ import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.flink.sql.parser.ddl.SqlCreateFunction; import org.apache.flink.sql.parser.ddl.SqlCreateTable; +import org.apache.flink.sql.parser.ddl.SqlCreateTableLike; import org.apache.flink.sql.parser.ddl.SqlCreateView; import org.apache.flink.sql.parser.ddl.SqlDistribution; import org.apache.flink.sql.parser.ddl.SqlTableColumn; @@ -152,6 +154,40 @@ public static SqlTableConstraint createPrimaryKeyConstraint(List primary SqlParserPos.ZERO); } + public static SqlCreateTable resolveTableProperties(SqlCreateTable createTable) { + var resolvedPropMap = resolvePropertiesToMap(createTable.getPropertyList()); + var resolvedPropList = createProperties(resolvedPropMap); + + if (createTable instanceof SqlCreateTableLike likeTable) { + return new SqlCreateTableLike( + likeTable.getParserPosition(), + likeTable.getTableName(), + likeTable.getColumnList(), + likeTable.getTableConstraints(), + resolvedPropList, + likeTable.getDistribution(), + likeTable.getPartitionKeyList(), + likeTable.getWatermark().orElse(null), + likeTable.getComment().orElse(null), + likeTable.getTableLike(), + likeTable.isTemporary(), + likeTable.ifNotExists); + } + + return new SqlCreateTable( + createTable.getParserPosition(), + createTable.getTableName(), + createTable.getColumnList(), + createTable.getTableConstraints(), + resolvedPropList, + createTable.getDistribution(), + createTable.getPartitionKeyList(), + createTable.getWatermark().orElse(null), + createTable.getComment().orElse(null), + createTable.isTemporary(), + createTable.ifNotExists); + } + public static SqlNodeList createProperties(Map options) { List props = options.entrySet().stream() @@ -167,15 +203,20 @@ public static SqlNodeList createProperties(Map options) { return new SqlNodeList(props, SqlParserPos.ZERO); } - public static Map propertiesToMap(SqlNodeList nodeList) { - Map result = new HashMap<>(); - for (SqlNode node : nodeList) { + public static Map resolvePropertiesToMap(SqlNodeList nodeList) { + var res = new LinkedHashMap(); + var resolver = NonSecretEnvVarResolver.builder().strict(false).build(); + + for (var node : nodeList) { var option = (SqlTableOption) node; var keyLiteral = (SqlLiteral) option.getKey(); var valueLiteral = (SqlLiteral) option.getValue(); - result.put(keyLiteral.toValue(), valueLiteral.toValue()); + var resolvedVal = resolver.resolve(valueLiteral.toValue()); + + res.put(keyLiteral.toValue(), resolvedVal); } - return result; + + return res; } public static SqlNodeList createPartitionKeys(List partitionKeys) { diff --git a/sqrl-planner/src/main/java/com/datasqrl/planner/Sqrl2FlinkSQLTranslator.java b/sqrl-planner/src/main/java/com/datasqrl/planner/Sqrl2FlinkSQLTranslator.java index 069ade2c9..125eb1d5a 100644 --- a/sqrl-planner/src/main/java/com/datasqrl/planner/Sqrl2FlinkSQLTranslator.java +++ b/sqrl-planner/src/main/java/com/datasqrl/planner/Sqrl2FlinkSQLTranslator.java @@ -828,7 +828,7 @@ private AddTableResult addTable( Optional mutationBuilder) { var tableSqlNode = parseSQL(createTableSql); checkArgument(tableSqlNode instanceof SqlCreateTable, "Expected CREATE TABLE statement"); - var tableDefinition = (SqlCreateTable) tableSqlNode; + var tableDefinition = FlinkSqlNodeFactory.resolveTableProperties((SqlCreateTable) tableSqlNode); var fullTable = tableDefinition; var origTableName = fullTable.getTableName().getSimple(); final var finalTableName = tableNameModifier.apply(origTableName); @@ -837,7 +837,7 @@ private AddTableResult addTable( // Check if the LIKE clause is referencing an external schema SqlTableLike likeClause = likeTable.getTableLike(); var likeTableName = likeClause.getSourceTable().toString(); - var likeTableProps = FlinkSqlNodeFactory.propertiesToMap(likeTable.getPropertyList()); + var likeTableProps = FlinkSqlNodeFactory.resolvePropertiesToMap(likeTable.getPropertyList()); Optional schema = schemaLoader.loadSchema(finalTableName, likeTableName, likeTableProps); if (schema.isPresent()) { diff --git a/sqrl-planner/src/main/java/com/datasqrl/planner/parser/SqrlStatementParser.java b/sqrl-planner/src/main/java/com/datasqrl/planner/parser/SqrlStatementParser.java index fd455604d..5b630ace1 100644 --- a/sqrl-planner/src/main/java/com/datasqrl/planner/parser/SqrlStatementParser.java +++ b/sqrl-planner/src/main/java/com/datasqrl/planner/parser/SqrlStatementParser.java @@ -24,7 +24,6 @@ import com.datasqrl.error.ErrorLocation.FileLocation; import com.datasqrl.planner.hint.PlannerHints; import com.datasqrl.planner.parser.SqrlTableFunctionStatement.ParsedArgument; -import com.datasqrl.planner.util.NonSecretEnvVarResolver; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -123,26 +122,22 @@ public class SqrlStatementParser { * @return */ public List parseScript(String script, ErrorCollector scriptErrors) { - var parsedStatements = new ArrayList(); - - var envVarResolver = NonSecretEnvVarResolver.builder().strict(false).build(); + List sqlStatements = new ArrayList<>(); var localErrors = scriptErrors; - try { var statements = SqlScriptStatementSplitter.splitStatements(script); - for (var stmt : statements) { - localErrors = scriptErrors.atFile(stmt.getFileLocation()); - - var sqlStmt = parseStatement(stmt.getObject(), envVarResolver); - var parsedObj = new ParsedObject<>(sqlStmt, stmt.getFileLocation()); - - parsedStatements.add(new ParsedStatement(parsedObj, stmt.get())); + for (ParsedObject statement : statements) { + localErrors = scriptErrors.atFile(statement.getFileLocation()); + sqlStatements.add( + new ParsedStatement( + new ParsedObject<>( + parseStatement(statement.getObject()), statement.getFileLocation()), + statement.get())); } } catch (StatementParserException e) { throw localErrors.handle(e); } - - return parsedStatements; + return sqlStatements; } /** @@ -151,30 +146,28 @@ public List parseScript(String script, ErrorCollector scriptErr * @param statement * @return */ - SQLStatement parseStatement(String statement, NonSecretEnvVarResolver envVarResolver) { - var resolvedStmt = envVarResolver.resolve(statement); - var importExportMatcher = IMPORT_EXPORT.matcher(resolvedStmt); + public SQLStatement parseStatement(String statement) { + var importExportMatcher = IMPORT_EXPORT.matcher(statement); // #1: Imports and Exports if (importExportMatcher.find()) { // it's an import or export statement var directive = importExportMatcher.group("fullmatch"); var commentEnd = importExportMatcher.end() - directive.length(); - var comment = - SqrlComments.parse(parse(resolvedStmt.substring(0, commentEnd), resolvedStmt, 0)); - var body = resolvedStmt.substring(importExportMatcher.end()).trim(); + var comment = SqrlComments.parse(parse(statement.substring(0, commentEnd), statement, 0)); + var body = statement.substring(importExportMatcher.end()).trim(); if (directive.equalsIgnoreCase("import")) { var subMatcher = IMPORT_PARSER.matcher(body); checkFatal(subMatcher.find(), ErrorCode.INVALID_IMPORT, "Could not parse IMPORT statement"); - var packageIdentifier = parseNamePath(subMatcher, "package", resolvedStmt); - var alias = parseNamePath(subMatcher, "identifier", resolvedStmt); + var packageIdentifier = parseNamePath(subMatcher, "package", statement); + var alias = parseNamePath(subMatcher, "identifier", statement); checkFatal(packageIdentifier.isPresent(), ErrorCode.INVALID_IMPORT, "Missing package path"); return new SqrlImportStatement(packageIdentifier, alias, comment); } else if (directive.equalsIgnoreCase("export")) { var subMatcher = EXPORT_PARSER.matcher(body); checkFatal(subMatcher.find(), ErrorCode.INVALID_IMPORT, "Could not parse IMPORT statement"); - var packageIdentifier = parseNamePath(subMatcher, "package", resolvedStmt); - var tableName = parseNamePath(subMatcher, "identifier", resolvedStmt); + var packageIdentifier = parseNamePath(subMatcher, "package", statement); + var tableName = parseNamePath(subMatcher, "identifier", statement); checkFatal(packageIdentifier.isPresent(), ErrorCode.INVALID_EXPORT, "Missing package path"); checkFatal(packageIdentifier.isPresent(), ErrorCode.INVALID_EXPORT, "Missing table"); return new SqrlExportStatement(tableName, packageIdentifier, comment); @@ -185,11 +178,11 @@ SQLStatement parseStatement(String statement, NonSecretEnvVarResolver envVarReso } // #2: SQRL Table Definitions - var sqrlDefinition = SQRL_DEFINITION.matcher(resolvedStmt); + var sqrlDefinition = SQRL_DEFINITION.matcher(statement); if (sqrlDefinition.find()) { // it's a SQRL definition var tableName = parseNamePath( - sqrlDefinition.group("tablename"), resolvedStmt, sqrlDefinition.start("tablename")); + sqrlDefinition.group("tablename"), statement, sqrlDefinition.start("tablename")); checkFatal( tableName.isPresent(), tableName.getFileLocation(), @@ -203,17 +196,16 @@ SQLStatement parseStatement(String statement, NonSecretEnvVarResolver envVarReso tableName.get()); var isRelationship = tableName.get().size() > 1; var commentEnd = sqrlDefinition.start("fullmatch"); - var comment = - SqrlComments.parse(parse(resolvedStmt.substring(0, commentEnd), resolvedStmt, 0)); + var comment = SqrlComments.parse(parse(statement.substring(0, commentEnd), statement, 0)); var definitionBody = - parse(resolvedStmt.substring(sqrlDefinition.end()), resolvedStmt, sqrlDefinition.end()); + parse(statement.substring(sqrlDefinition.end()), statement, sqrlDefinition.end()); var access = AccessModifier.QUERY; ParsedObject arguments = - parse(sqrlDefinition, "arguments", resolvedStmt).map(str -> str.isBlank() ? null : str); + parse(sqrlDefinition, "arguments", statement).map(str -> str.isBlank() ? null : str); ParsedObject returnType = - parse(sqrlDefinition, "returntype", resolvedStmt).map(str -> str.isBlank() ? null : str); + parse(sqrlDefinition, "returntype", statement).map(str -> str.isBlank() ? null : str); // Identify SQL keyword var sqlKeywordPattern = Pattern.compile("^\\s*(\\w+)"); var keywordMatcher = sqlKeywordPattern.matcher(definitionBody.get()); @@ -237,7 +229,7 @@ SQLStatement parseStatement(String statement, NonSecretEnvVarResolver envVarReso access = AccessModifier.SUBSCRIPTION; var newDefinitionStart = sqrlDefinition.end() + keywordEnd; definitionBody = - parse(resolvedStmt.substring(newDefinitionStart), resolvedStmt, newDefinitionStart); + parse(statement.substring(newDefinitionStart), statement, newDefinitionStart); } if (arguments.isEmpty() && returnType.isEmpty() && !isRelationship) { definition = new SqrlTableDefinition(tableName, definitionBody, access, comment); @@ -297,27 +289,26 @@ SQLStatement parseStatement(String statement, NonSecretEnvVarResolver envVarReso } // #3: Create Table - var createTable = CREATE_TABLE.matcher(resolvedStmt); + var createTable = CREATE_TABLE.matcher(statement); if (createTable.find()) { var createTableStmt = parse( - resolvedStmt.substring(createTable.start("fullmatch")), - resolvedStmt, + statement.substring(createTable.start("fullmatch")), + statement, createTable.start("fullmatch")); var commentEnd = createTable.start("fullmatch"); - var comment = - SqrlComments.parse(parse(resolvedStmt.substring(0, commentEnd), resolvedStmt, 0)); + var comment = SqrlComments.parse(parse(statement.substring(0, commentEnd), statement, 0)); return new SqrlCreateTableStatement(createTableStmt, comment); } // #4: Next Batch - var nextBatch = NEXT_BATCH_PARSER.matcher(resolvedStmt); + var nextBatch = NEXT_BATCH_PARSER.matcher(statement); if (nextBatch.find()) { - return new SqrlNextBatch(new ParsedObject<>(resolvedStmt, FileLocation.START)); + return new SqrlNextBatch(new ParsedObject<>(statement, FileLocation.START)); } // #5: If none of the regex match, we assume it's a Flink SQL statement - return new FlinkSQLStatement(new ParsedObject<>(resolvedStmt, FileLocation.START)); + return new FlinkSQLStatement(new ParsedObject<>(statement, FileLocation.START)); } /** diff --git a/sqrl-planner/src/main/java/com/datasqrl/planner/tables/FlinkTableBuilder.java b/sqrl-planner/src/main/java/com/datasqrl/planner/tables/FlinkTableBuilder.java index 188624f53..c089570c5 100644 --- a/sqrl-planner/src/main/java/com/datasqrl/planner/tables/FlinkTableBuilder.java +++ b/sqrl-planner/src/main/java/com/datasqrl/planner/tables/FlinkTableBuilder.java @@ -130,7 +130,7 @@ public FlinkTableBuilder setConnectorOptions(Map options) { } public Map getConnectorOptions() { - return FlinkSqlNodeFactory.propertiesToMap(getPropertyList()); + return FlinkSqlNodeFactory.resolvePropertiesToMap(getPropertyList()); } public FlinkTableBuilder setDummyConnector() { diff --git a/sqrl-planner/src/main/java/com/datasqrl/util/ConfigLoaderUtils.java b/sqrl-planner/src/main/java/com/datasqrl/util/ConfigLoaderUtils.java index 6435362ae..97034f184 100644 --- a/sqrl-planner/src/main/java/com/datasqrl/util/ConfigLoaderUtils.java +++ b/sqrl-planner/src/main/java/com/datasqrl/util/ConfigLoaderUtils.java @@ -29,6 +29,7 @@ import com.datasqrl.planner.util.NonSecretEnvVarResolver; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.annotations.VisibleForTesting; import com.networknt.schema.SchemaRegistry; @@ -334,22 +335,17 @@ private static ObjectNode convertFileToObjectNode( var filename = file.isLeft() ? file.left().toString() : file.right().toString(); var localErr = errors.withConfig(filename); - ObjectMapper mapper; - if (resolveEnvVars) { - var resolver = NonSecretEnvVarResolver.builder().strict(false).build(); - mapper = resolver.initObjectMapper(MAPPER); - } else { - mapper = MAPPER; - } - try { JsonNode jsonNode; - if (file.isLeft()) { - jsonNode = mapper.readTree(file.left().toFile()); - + jsonNode = MAPPER.readTree(file.left().toFile()); } else { - jsonNode = mapper.readTree(file.right()); + jsonNode = MAPPER.readTree(file.right().openStream()); + } + + if (resolveEnvVars) { + var resolver = NonSecretEnvVarResolver.builder().strict(false).build(); + resolveConnectorEnvVars(jsonNode.get("connectors"), resolver); } return (ObjectNode) jsonNode; @@ -358,4 +354,36 @@ private static ObjectNode convertFileToObjectNode( throw localErr.exception("Could not parse JSON file [%s]: %s", file, e.toString()); } } + + private static void resolveConnectorEnvVars(JsonNode node, NonSecretEnvVarResolver resolver) { + if (node == null) { + return; + } + + if (node.isObject()) { + var objectNode = (ObjectNode) node; + for (var field : objectNode.properties()) { + var value = field.getValue(); + + if (value.isContainerNode()) { + resolveConnectorEnvVars(value, resolver); + + } else if (value.isTextual()) { + objectNode.put(field.getKey(), resolver.resolve(value.asText())); + } + } + } else if (node.isArray()) { + var arrayNode = (ArrayNode) node; + for (var i = 0; i < arrayNode.size(); i++) { + var value = arrayNode.get(i); + + if (value.isContainerNode()) { + resolveConnectorEnvVars(value, resolver); + + } else if (value.isTextual()) { + arrayNode.set(i, MAPPER.getNodeFactory().textNode(resolver.resolve(value.asText()))); + } + } + } + } } diff --git a/sqrl-testing/sqrl-testing-integration/pom.xml b/sqrl-testing/sqrl-testing-integration/pom.xml index 78cf0ca41..62ffab140 100644 --- a/sqrl-testing/sqrl-testing-integration/pom.xml +++ b/sqrl-testing/sqrl-testing-integration/pom.xml @@ -139,5 +139,10 @@ mockito-core test + + uk.org.webcompere + system-stubs-jupiter + test + diff --git a/sqrl-testing/sqrl-testing-integration/src/test/java/com/datasqrl/UseCaseCompileTest.java b/sqrl-testing/sqrl-testing-integration/src/test/java/com/datasqrl/UseCaseCompileTest.java index 8945ecbfe..1a6e47a0c 100644 --- a/sqrl-testing/sqrl-testing-integration/src/test/java/com/datasqrl/UseCaseCompileTest.java +++ b/sqrl-testing/sqrl-testing-integration/src/test/java/com/datasqrl/UseCaseCompileTest.java @@ -21,9 +21,13 @@ import java.nio.file.Path; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import uk.org.webcompere.systemstubs.environment.EnvironmentVariables; +import uk.org.webcompere.systemstubs.jupiter.SystemStub; +import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension; /** * Parametrized Test for parsing and planning of SQRL scripts in resources/usecases. Add entire SQRL @@ -34,10 +38,17 @@ * `-warn` are expected to produce warnings which are snapshotted. SQRL scripts ending in -disabled` * are ignored. */ +@ExtendWith(SystemStubsExtension.class) public class UseCaseCompileTest { private static final Path USECASE_DIR = getResourcesDirectory("usecases"); + @SuppressWarnings("unused") + @SystemStub + private final EnvironmentVariables environmentVariables = + new EnvironmentVariables( + "DATAGEN_CONNECTOR", "datagen", "ICEBERG_TEST_WAREHOUSE", "/tmp/test_iceberg_wh"); + @RegisterExtension final CliCompileTestExtension snapshotExtension = new CliCompileTestExtension(); @@ -55,7 +66,7 @@ void testUseCase(Path packageFile) { @Test @Disabled("Intended for manual usage") void runTestCaseByName() { - var pkg = USECASE_DIR.resolve("function-translation/duckdb").resolve("package.json"); + var pkg = USECASE_DIR.resolve("flink-only-compile").resolve("package.json"); UseCaseTestHelper.testUseCase( snapshotExtension, getClass(), diff --git a/sqrl-testing/sqrl-testing-integration/src/test/resources/snapshots/com/datasqrl/UseCaseCompileTest/flink-only-compile-package.txt b/sqrl-testing/sqrl-testing-integration/src/test/resources/snapshots/com/datasqrl/UseCaseCompileTest/flink-only-compile-package.txt index ec89f1fe5..f4e421cae 100644 --- a/sqrl-testing/sqrl-testing-integration/src/test/resources/snapshots/com/datasqrl/UseCaseCompileTest/flink-only-compile-package.txt +++ b/sqrl-testing/sqrl-testing-integration/src/test/resources/snapshots/com/datasqrl/UseCaseCompileTest/flink-only-compile-package.txt @@ -23,7 +23,8 @@ CREATE TABLE IF NOT EXISTS `customer_id_count` ( PRIMARY KEY (`customerid`) NOT ENFORCED ) WITH ( - 'connector' = 'blackhole' + 'connector' = 'print', + 'print-identifier' = '${SUPER_SECRET}' ); EXECUTE STATEMENT SET BEGIN INSERT INTO `customer_id_count` diff --git a/sqrl-testing/sqrl-testing-integration/src/test/resources/snapshots/com/datasqrl/UseCaseCompileTest/iceberg-export-compile-package.txt b/sqrl-testing/sqrl-testing-integration/src/test/resources/snapshots/com/datasqrl/UseCaseCompileTest/iceberg-export-compile-package.txt index c6447e36b..2508b2ab2 100644 --- a/sqrl-testing/sqrl-testing-integration/src/test/resources/snapshots/com/datasqrl/UseCaseCompileTest/iceberg-export-compile-package.txt +++ b/sqrl-testing/sqrl-testing-integration/src/test/resources/snapshots/com/datasqrl/UseCaseCompileTest/iceberg-export-compile-package.txt @@ -140,7 +140,7 @@ CREATE TABLE `Iceberg_sink_ex1` ( WITH ( 'connector' = 'iceberg', 'catalog-table' = 'my-table', - 'warehouse' = '/tmp/duckdb', + 'warehouse' = '/tmp/test_iceberg_wh', 'catalog-type' = 'hadoop', 'catalog-name' = 'mydatabase' ) diff --git a/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/duckdb/package.json b/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/duckdb/package.json index 818e8fdb1..62cf8d43d 100644 --- a/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/duckdb/package.json +++ b/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/duckdb/package.json @@ -23,7 +23,7 @@ }, "connectors": { "iceberg": { - "warehouse": "${TEST_WH}" + "warehouse": "/tmp/duckdb" } }, "test-runner": { diff --git a/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/flink-only-compile/flink-only.sqrl b/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/flink-only-compile/flink-only.sqrl index 8247cccd5..ba4ca5ab5 100644 --- a/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/flink-only-compile/flink-only.sqrl +++ b/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/flink-only-compile/flink-only.sqrl @@ -5,7 +5,7 @@ CREATE TABLE Customer ( lastUpdated AS NOW(), WATERMARK FOR `lastUpdated` AS `lastUpdated` - INTERVAL '0.001' SECOND ) WITH ( - 'connector' = 'datagen', + 'connector' = '${DATAGEN_CONNECTOR}', -- Generate customerid sequentially 'fields.customerid.kind' = 'sequence', @@ -23,13 +23,13 @@ CREATE TABLE Customer ( ); CREATE TABLE IF NOT EXISTS `customer_id_count` ( - customerid BIGINT, - numEntries BIGINT, - PRIMARY KEY (customerid) NOT ENFORCED - ) - WITH ( - 'connector' = 'blackhole' - ); + customerid BIGINT, + numEntries BIGINT, + PRIMARY KEY (customerid) NOT ENFORCED +) WITH ( + 'connector' = 'print', + 'print-identifier' = '${{SUPER_SECRET}}' +); INSERT INTO `customer_id_count` (SELECT diff --git a/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/iceberg-export-compile/iceberg/iceberg-sink.table.sql b/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/iceberg-export-compile/iceberg/iceberg-sink.table.sql index 553c1715f..e8e7e8b82 100644 --- a/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/iceberg-export-compile/iceberg/iceberg-sink.table.sql +++ b/sqrl-testing/sqrl-testing-integration/src/test/resources/usecases/iceberg-export-compile/iceberg/iceberg-sink.table.sql @@ -3,7 +3,7 @@ CREATE TABLE Iceberg_sink ( ) WITH ( 'connector' = 'iceberg', 'catalog-table' = 'my-table', - 'warehouse' = '/tmp/duckdb', + 'warehouse' = '${ICEBERG_TEST_WAREHOUSE}', 'catalog-type' = 'hadoop', 'catalog-name' = 'mydatabase' -) LIKE `.`; \ No newline at end of file +) LIKE `.`; From 450a7b421af8f076ac9d74e0dec6dafae2b395f3 Mon Sep 17 00:00:00 2001 From: Ferenc Csaky Date: Tue, 9 Jun 2026 21:08:46 +0200 Subject: [PATCH 3/3] update docs --- documentation/docs/configuration-default.md | 5 +---- documentation/docs/configuration.md | 11 ++++++++++- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/documentation/docs/configuration-default.md b/documentation/docs/configuration-default.md index 73e0054ed..0c62855bb 100644 --- a/documentation/docs/configuration-default.md +++ b/documentation/docs/configuration-default.md @@ -108,10 +108,7 @@ The connector templates configured under `connectors` can use environment variab ### Environment Variables -You can reference environment variables using the `${VAR_NAME}` placeholder syntax, for example `${POSTGRES_PASSWORD}`. -At runtime, these placeholders are automatically resolved using the environment variables defined in the system or deployment environment. - -This can help decouple security credentials or add flexibility across different deployment environments. +See [Environment Variables](configuration#environment-variables-var) for supported placeholder syntax and resolution behavior. ### SQRL Variables diff --git a/documentation/docs/configuration.md b/documentation/docs/configuration.md index 4d5a4cc0a..b23755499 100644 --- a/documentation/docs/configuration.md +++ b/documentation/docs/configuration.md @@ -164,7 +164,16 @@ Refer to the individual engine configuration for connector configuration options ## Environment Variables (`${VAR}`) -Environment variables (e.g. `${POSTGRES_PASSWORD}`) can be referenced inside the configuration files and SQRL scripts. Those are dynamically resolved by the DataSQRL runner when the pipeline is launched. If an environment variable is not configured, it is not replaced. +Environment variables can be referenced with two placeholder types: +- `${VAR_NAME}` references a non-secret environment variable, for example `${POSTGRES_HOST}`. DataSQRL treats this syntax as non-secret, even if the variable name contains words like `PASSWORD` or `TOKEN`. +- `${{VAR_NAME}}` references a secret environment variable, for example `${{POSTGRES_PASSWORD}}`. Secret placeholders are not resolved during compile and are converted to `${VAR_NAME}` in generated artifacts so the actual value is resolved only when the pipeline runs. +DataSQRL resolves environment variables at two different times: +- During `sqrl compile`, DataSQRL resolves available, non-secret environment variables in user-provided `package.json` [connector configuration](configuration-default) and in Flink `CREATE TABLE ... WITH (...)` options in SQRL scripts. +- If a `${VAR_NAME}` placeholder is not available during `sqrl compile`, it is left unchanged in the generated build artifacts so it can still be supplied later. +- During `sqrl run` or `sqrl test`, DataSQRL first compiles the project with the same compile-time rules, then launches the generated artifacts and resolves remaining `${VAR_NAME}` placeholders defined anywhere in the SQRL scripts from the runtime environment. +- If a required runtime placeholder is still unresolved when the generated artifact is launched, the run fails with an error identifying the missing variable. + +Compile-time values are written into the generated artifacts under `build/` and `build/deploy/`. Use the secret placeholder syntax for values that must not be resolved or written during compile.