diff --git a/.github/trigger_files/beam_PostCommit_SQL.json b/.github/trigger_files/beam_PostCommit_SQL.json index 833fd9b0d174..5df3841d2363 100644 --- a/.github/trigger_files/beam_PostCommit_SQL.json +++ b/.github/trigger_files/beam_PostCommit_SQL.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run ", - "modification": 2 + "modification": 3 } diff --git a/.github/trigger_files/beam_PostCommit_XVR_Direct.json b/.github/trigger_files/beam_PostCommit_XVR_Direct.json index 8e1c80dd873d..cccbad0b12df 100644 --- a/.github/trigger_files/beam_PostCommit_XVR_Direct.json +++ b/.github/trigger_files/beam_PostCommit_XVR_Direct.json @@ -1,4 +1,3 @@ { - "https://github.com/apache/beam/pull/32648": "testing Flink 1.19 support", - "modification": 3 + "modification": 4 } diff --git a/CHANGES.md b/CHANGES.md index a8b160736ef2..8b3d95561ca0 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -77,7 +77,11 @@ ## Breaking Changes * X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). -* Previously deprecated Beam ZetaSQL component has been removed from new releases ([#34423](https://github.com/apache/beam/issues/34423)). +* Previously deprecated Beam ZetaSQL component has been removed ([#34423](https://github.com/apache/beam/issues/34423)). + ZetaSQL users could migrate to Calcite SQL with BigQuery dialect enabled. +* Upgraded Beam vendored Calcite to 1.40.0 for Beam SQL ([#35483](https://github.com/apache/beam/issues/35483)), which + improves support for BigQuery and other SQL dialects. Note: Minor behavior changes are observed such as output + significant digits related to casting. ## Deprecations diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 1562fbf8fcec..a7f2b99b9a1f 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -907,7 +907,7 @@ class BeamModulePlugin implements Plugin { threetenbp : "org.threeten:threetenbp:1.6.8", vendored_grpc_1_69_0 : "org.apache.beam:beam-vendor-grpc-1_69_0:0.1", vendored_guava_32_1_2_jre : "org.apache.beam:beam-vendor-guava-32_1_2-jre:0.1", - vendored_calcite_1_28_0 : "org.apache.beam:beam-vendor-calcite-1_28_0:0.2", + vendored_calcite_1_40_0 : "org.apache.beam:beam-vendor-calcite-1_40_0:0.1", woodstox_core_asl : "org.codehaus.woodstox:woodstox-core-asl:4.4.1", zstd_jni : "com.github.luben:zstd-jni:1.5.6-3", quickcheck_core : "com.pholser:junit-quickcheck-core:$quickcheck_version", @@ -1832,13 +1832,12 @@ class BeamModulePlugin implements Plugin { project.ext.includeInJavaBom = configuration.publish project.ext.exportJavadoc = configuration.exportJavadoc - boolean publishEnabledByCommand = isRelease(project) || project.hasProperty('publishing') if (forkJavaVersion == '') { // project needs newer version and not served. // If not publishing ,disable the project. Otherwise, fail the build def msg = "project ${project.name} needs newer Java version to compile. Consider set -Pjava${project.javaVersion}Home" - if (publishEnabledByCommand) { - throw new GradleException("Publish enabled but " + msg + ".") + if (isRelease(project)) { + throw new GradleException("Release enabled but " + msg + ".") } else { logger.config(msg + " if needed.") project.tasks.each { @@ -1846,7 +1845,7 @@ class BeamModulePlugin implements Plugin { } } } - if (publishEnabledByCommand && configuration.publish) { + if ((isRelease(project) || project.hasProperty('publishing')) && configuration.publish) { project.apply plugin: "maven-publish" // plugin to support repository authentication via ~/.m2/settings.xml diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle index 259402aa16fb..af8b6cba1742 100644 --- a/sdks/java/extensions/sql/build.gradle +++ b/sdks/java/extensions/sql/build.gradle @@ -72,7 +72,7 @@ dependencies { javacc "net.java.dev.javacc:javacc:4.0" fmppTask "com.googlecode.fmpp-maven-plugin:fmpp-maven-plugin:1.0" fmppTask "org.freemarker:freemarker:2.3.31" - fmppTemplates library.java.vendored_calcite_1_28_0 + fmppTemplates library.java.vendored_calcite_1_40_0 implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":sdks:java:managed") implementation project(":sdks:java:io:iceberg") @@ -87,7 +87,7 @@ dependencies { implementation library.java.commons_csv implementation library.java.jackson_databind implementation library.java.joda_time - implementation library.java.vendored_calcite_1_28_0 + implementation library.java.vendored_calcite_1_40_0 implementation "org.codehaus.janino:janino:3.0.11" implementation "org.codehaus.janino:commons-compiler:3.0.11" implementation library.java.jackson_core @@ -117,7 +117,7 @@ dependencies { testImplementation "org.apache.iceberg:iceberg-api:1.6.1" testImplementation "org.apache.iceberg:iceberg-core:1.6.1" - testImplementation library.java.vendored_calcite_1_28_0 + testImplementation library.java.vendored_calcite_1_40_0 testImplementation library.java.vendored_guava_32_1_2_jre testImplementation library.java.junit testImplementation library.java.quickcheck_core @@ -164,11 +164,15 @@ task copyFmppTemplatesFromCalciteCore(type: Copy) { into "${project.buildDir}/templates-fmpp" filter{ line -> - line.replace('import org.apache.calcite.', 'import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.') + line.replace('import org.apache.calcite.', 'import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.') } filter{ line -> - line.replace('import static org.apache.calcite.', 'import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.') + line.replace('import static org.apache.calcite.', 'import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.') + } + filter{ + line -> + line.replace('import com.google.common.', 'import org.apache.beam.vendor.calcite.v1_40_0.com.google.common.') } } diff --git a/sdks/java/extensions/sql/jdbc/build.gradle b/sdks/java/extensions/sql/jdbc/build.gradle index 1dae8d5c5a2d..91c48b635d50 100644 --- a/sdks/java/extensions/sql/jdbc/build.gradle +++ b/sdks/java/extensions/sql/jdbc/build.gradle @@ -38,8 +38,8 @@ dependencies { implementation "jline:jline:2.14.6" permitUnusedDeclared "jline:jline:2.14.6" // BEAM-11761 implementation "sqlline:sqlline:1.4.0" - implementation library.java.vendored_calcite_1_28_0 - permitUnusedDeclared library.java.vendored_calcite_1_28_0 + implementation library.java.vendored_calcite_1_40_0 + permitUnusedDeclared library.java.vendored_calcite_1_40_0 testImplementation project(path: ":sdks:java:core", configuration: "shadow") testImplementation project(path: ":sdks:java:io:google-cloud-platform") testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core") diff --git a/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java index c57c5a530c9e..7df39d2ed625 100644 --- a/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java +++ b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java @@ -129,9 +129,9 @@ public void testSelectFromPubsub() throws Exception { assertThat( Arrays.asList( - Arrays.asList("2018-07-01 21:25:20", "enroute", "40.702", "-74.001"), - Arrays.asList("2018-07-01 21:26:06", "enroute", "40.703", "-74.002"), - Arrays.asList("2018-07-02 13:26:06", "enroute", "30.0", "-72.32324")), + Arrays.asList("2018-07-01 21:25:20.000000", "enroute", "40.702", "-74.001"), + Arrays.asList("2018-07-01 21:26:06.000000", "enroute", "40.703", "-74.002"), + Arrays.asList("2018-07-02 13:26:06.000000", "enroute", "30.0", "-72.32324")), everyItem(IsIn.isOneOf(expectedResult.get(30, TimeUnit.SECONDS).toArray()))); } @@ -170,8 +170,8 @@ public void testFilterForSouthManhattan() throws Exception { assertThat( Arrays.asList( - Arrays.asList("2018-07-01 21:25:20", "enroute", "40.701", "-74.001"), - Arrays.asList("2018-07-01 21:26:06", "enroute", "40.702", "-74.002")), + Arrays.asList("2018-07-01 21:25:20.000000", "enroute", "40.701", "-74.001"), + Arrays.asList("2018-07-01 21:26:06.000000", "enroute", "40.702", "-74.002")), everyItem(IsIn.isOneOf(expectedResult.get(30, TimeUnit.SECONDS).toArray()))); } diff --git a/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineTest.java b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineTest.java index e308fa52d6e4..367336e1e7a0 100644 --- a/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineTest.java +++ b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineTest.java @@ -168,7 +168,8 @@ public void testSqlLine_fixedWindow() throws Exception { List> lines = toLines(byteArrayOutputStream); assertThat( Arrays.asList( - Arrays.asList("2018-07-01 21:26:06", "1"), Arrays.asList("2018-07-01 21:26:07", "1")), + Arrays.asList("2018-07-01 21:26:06.000000", "1"), + Arrays.asList("2018-07-01 21:26:07.000000", "1")), everyItem(is(oneOf(lines.toArray())))); } @@ -190,11 +191,11 @@ public void testSqlLine_slidingWindow() throws Exception { List> lines = toLines(byteArrayOutputStream); assertThat( Arrays.asList( - Arrays.asList("2018-07-01 21:26:07", "1"), - Arrays.asList("2018-07-01 21:26:08", "2"), - Arrays.asList("2018-07-01 21:26:09", "2"), - Arrays.asList("2018-07-01 21:26:10", "2"), - Arrays.asList("2018-07-01 21:26:11", "1")), + Arrays.asList("2018-07-01 21:26:07.000000", "1"), + Arrays.asList("2018-07-01 21:26:08.000000", "2"), + Arrays.asList("2018-07-01 21:26:09.000000", "2"), + Arrays.asList("2018-07-01 21:26:10.000000", "2"), + Arrays.asList("2018-07-01 21:26:11.000000", "1")), everyItem(is(oneOf(lines.toArray())))); } } diff --git a/sdks/java/extensions/sql/perf-tests/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryIOPushDownIT.java b/sdks/java/extensions/sql/perf-tests/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryIOPushDownIT.java index ff3e62f551bd..224dc1ad92c1 100644 --- a/sdks/java/extensions/sql/perf-tests/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryIOPushDownIT.java +++ b/sdks/java/extensions/sql/perf-tests/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryIOPushDownIT.java @@ -45,9 +45,9 @@ import org.apache.beam.sdk.testutils.metrics.TimeMonitor; import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSets; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Before; import org.junit.BeforeClass; diff --git a/sdks/java/extensions/sql/src/main/codegen/config.fmpp b/sdks/java/extensions/sql/src/main/codegen/config.fmpp index 7914cf6fcb79..77772c5858e3 100644 --- a/sdks/java/extensions/sql/src/main/codegen/config.fmpp +++ b/sdks/java/extensions/sql/src/main/codegen/config.fmpp @@ -21,10 +21,11 @@ data: { # List of import statements. imports: [ - "org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.ColumnStrategy" - "org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCreate" - "org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlDrop" - "org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName" + "org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.ColumnStrategy" + "org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlCreate" + "org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlDrop" + "org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName" + "org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.NlsString" "org.apache.beam.sdk.extensions.sql.impl.parser.SqlCreateCatalog" "org.apache.beam.sdk.extensions.sql.impl.parser.SqlCreateDatabase" "org.apache.beam.sdk.extensions.sql.impl.parser.SqlCreateExternalTable" @@ -64,6 +65,8 @@ data: { "AFTER" "ALWAYS" "APPLY" + "ARRAY_AGG" + "ARRAY_CONCAT_AGG" "ASC" "ASSERTION" "ASSIGNMENT" @@ -102,12 +105,19 @@ data: { "CONSTRAINTS" "CONSTRAINT_SCHEMA" "CONSTRUCTOR" + "CONTAINS_SUBSTR" "CONTINUE" "CURSOR_NAME" "DATA" "DATABASE" + "DATE_DIFF" + "DATE_TRUNC" + "DATETIME_DIFF" "DATETIME_INTERVAL_CODE" "DATETIME_INTERVAL_PRECISION" + "DATETIME_TRUNC" + "DAYOFWEEK" + "DAYOFYEAR" "DAYS" "DECADE" "DEFAULTS" @@ -126,6 +136,7 @@ data: { "DOMAIN" "DOW" "DOY" + "DOT_FORMAT" "DYNAMIC_FUNCTION" "DYNAMIC_FUNCTION_CODE" "ENCODING" @@ -148,13 +159,16 @@ data: { "GO" "GOTO" "GRANTED" + "GROUP_CONCAT" "HIERARCHY" "HOP" "HOURS" "IGNORE" + "ILIKE" "IMMEDIATE" "IMMEDIATELY" "IMPLEMENTATION" + "INCLUDE" "INCLUDING" "INCREMENT" "INITIALLY" @@ -222,6 +236,7 @@ data: { "PASSTHROUGH" "PAST" "PATH" + "PIVOT" "PLACING" "PLAN" "PLI" @@ -231,6 +246,7 @@ data: { "PRIVILEGES" "PUBLIC" "QUARTER" + "QUARTERS" "READ" "RELATIVE" "REPEATABLE" @@ -243,6 +259,7 @@ data: { "RETURNED_OCTET_LENGTH" "RETURNED_SQLSTATE" "RETURNING" + "RLIKE" "ROLE" "ROUTINE" "ROUTINE_CATALOG" @@ -260,6 +277,7 @@ data: { "SECTION" "SECURITY" "SELF" + "SEPARATOR" "SEQUENCE" "SERIALIZABLE" "SERVER" @@ -322,6 +340,7 @@ data: { "SQL_VARCHAR" "STATE" "STATEMENT" + "STRING_AGG" "STRUCTURE" "STYLE" "SUBCLASS_ORIGIN" @@ -329,8 +348,12 @@ data: { "TABLE_NAME" "TEMPORARY" "TIES" + "TIME_DIFF" + "TIME_TRUNC" "TIMESTAMPADD" "TIMESTAMPDIFF" + "TIMESTAMP_DIFF" + "TIMESTAMP_TRUNC" "TOP_LEVEL_COUNT" "TRANSACTION" "TRANSACTIONS_ACTIVE" @@ -347,6 +370,7 @@ data: { "UNCOMMITTED" "UNCONDITIONAL" "UNDER" + "UNPIVOT" "UNNAMED" "USAGE" "USER_DEFINED_TYPE_CATALOG" @@ -359,6 +383,7 @@ data: { "VERSION" "VIEW" "WEEK" + "WEEKS" "WORK" "WRAPPER" "WRITE" @@ -445,6 +470,12 @@ data: { "SqlDropCatalog" ] + # List of methods for parsing extensions to "TRUNCATE" calls. + # Each must accept arguments "(SqlParserPos pos)". + # Example: "SqlTruncate". + truncateStatementParserMethods: [ + ] + # Binary operators tokens binaryOperatorsTokens: [ ] @@ -461,11 +492,12 @@ data: { "parserImpls.ftl" ] + setOptionParserMethod: "SqlSetOption" includePosixOperators: false includeCompoundIdentifier: true includeBraces: true includeAdditionalDeclarations: false - + includeParsingStringLiteralAsArrayLiteral: false } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableNameExtractionUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableNameExtractionUtils.java index 0f50499926b6..efd3d8dfc353 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableNameExtractionUtils.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableNameExtractionUtils.java @@ -23,13 +23,13 @@ import java.util.Collections; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.TableName; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlAsOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlJoin; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSelect; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSetOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlAsOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlJoin; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSelect; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSetOperator; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java index 0b015c567cda..d684c72b2e69 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java @@ -28,13 +28,13 @@ import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expression; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelProtoDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaVersion; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schemas; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Expression; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaVersion; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schemas; import org.checkerframework.checker.nullness.qual.Nullable; /** Adapter from {@link TableProvider} to {@link Schema}. */ @@ -120,7 +120,7 @@ public Set getTypeNames() { } @Override - public org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Table getTable( + public org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Table getTable( String name) { Table table = resolveMetastore().getTable(name); if (table == null) { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java index 7692aadfb158..ce25610422c1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java @@ -30,15 +30,15 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteConnection; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expression; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelProtoDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaVersion; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Table; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteConnection; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Expression; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaVersion; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Table; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java index bd968ad2cb06..eb2c384b1e6f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java @@ -27,20 +27,20 @@ import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.java.AbstractQueryableTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.QueryProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.Queryable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.prepare.Prepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.TableModify; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.ModifiableTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.TranslatableTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.java.AbstractQueryableTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.QueryProvider; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.Queryable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.prepare.Prepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.TableModify; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.ModifiableTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.TranslatableTable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** Adapter from {@link BeamSqlTable} to a calcite Table. */ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java index 1edb22ac105f..73193f58f131 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java @@ -45,11 +45,11 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamTableStatistics.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamTableStatistics.java index 7aaaedd5e3a7..6f2c15223700 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamTableStatistics.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamTableStatistics.java @@ -21,12 +21,12 @@ import java.util.Collections; import java.util.List; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelDistribution; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelDistributionTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelReferentialConstraint; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Statistic; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.ImmutableBitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelCollation; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelDistribution; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelDistributionTraitDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelReferentialConstraint; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Statistic; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.ImmutableBitSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** This class stores row count statistics. */ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteConnectionWrapper.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteConnectionWrapper.java index f9161f37143f..333a5f4cdeda 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteConnectionWrapper.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteConnectionWrapper.java @@ -35,14 +35,15 @@ import java.util.Map; import java.util.Properties; import java.util.concurrent.Executor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.java.JavaTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.CalciteConnectionConfig; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteConnection; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.Enumerator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.Queryable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expression; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.config.CalciteConnectionConfig; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteConnection; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.Enumerator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.Queryable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Expression; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaPlus; +import org.checkerframework.checker.nullness.qual.Nullable; /** * Abstract wrapper for {@link CalciteConnection} to simplify extension. @@ -333,8 +334,10 @@ public void setSchema(String schema) throws SQLException { connection.setSchema(schema); } + // CalciteConnection.getSchema() marked nullable but Connection.getSchema() does not + @SuppressWarnings("override.return") @Override - public String getSchema() throws SQLException { + public @Nullable String getSchema() throws SQLException { return connection.getSchema(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteFactoryWrapper.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteFactoryWrapper.java index dcce4d1237a0..dd5c0d232bca 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteFactoryWrapper.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteFactoryWrapper.java @@ -21,18 +21,18 @@ import java.sql.SQLException; import java.util.Properties; import java.util.TimeZone; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.java.JavaTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.AvaticaConnection; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.AvaticaFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.AvaticaPreparedStatement; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.AvaticaResultSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.AvaticaSpecificDatabaseMetaData; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.AvaticaStatement; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.Meta; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.QueryState; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.UnregisteredDriver; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.AvaticaConnection; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.AvaticaFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.AvaticaPreparedStatement; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.AvaticaResultSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.AvaticaSpecificDatabaseMetaData; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.AvaticaStatement; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.Meta; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.QueryState; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.UnregisteredDriver; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; import org.checkerframework.checker.nullness.qual.Nullable; /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java index bd0e7ac00895..606a3c5f71a2 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java @@ -31,41 +31,42 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.impl.udf.BeamBuiltinFunctionProvider; -import org.apache.beam.vendor.calcite.v1_28_0.com.google.common.collect.Table; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.CalciteConnectionConfig; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Contexts; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.ConventionTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCost; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner.CannotPlanException; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.prepare.CalciteCatalogReader; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelRoot; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.BuiltInMetadata; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.ChainedRelMetadataProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.JaninoRelMetadataProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.MetadataDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.MetadataHandler; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperatorTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParseException; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParser; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserImplFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.util.SqlOperatorTables; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.FrameworkConfig; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.Frameworks; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.Planner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelConversionException; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.ValidationException; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.BuiltInMethod; +import org.apache.beam.vendor.calcite.v1_40_0.com.google.common.collect.Table; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.config.CalciteConnectionConfig; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Contexts; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.ConventionTraitDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCost; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner.CannotPlanException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.prepare.CalciteCatalogReader; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelRoot; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.BuiltInMetadata; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.ChainedRelMetadataProvider; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.JaninoRelMetadataProvider; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.MetadataDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.MetadataHandler; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataProvider; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperatorTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParseException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParser; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserImplFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.util.SqlOperatorTables; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql2rel.SqlToRelConverter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.FrameworkConfig; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.Frameworks; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.Planner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RelConversionException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.ValidationException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.BuiltInMethod; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.slf4j.Logger; @@ -141,6 +142,9 @@ public FrameworkConfig defaultConfig(JdbcConnection connection, Collection> costKeys = + List> costKeys = bmq.map.cellSet().stream() .filter(entry -> entry.getValue() instanceof BeamCostModel) .filter(entry -> ((BeamCostModel) entry.getValue()).isInfinite()) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JavaUdfLoader.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JavaUdfLoader.java index ab4b86660a6e..1e584ecdef40 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JavaUdfLoader.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JavaUdfLoader.java @@ -40,7 +40,7 @@ import org.apache.beam.sdk.extensions.sql.udf.UdfProvider; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.commons.codec.digest.DigestUtils; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.commons.codec.digest.DigestUtils; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java index 972674df9f91..f9d7eddbc687 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java @@ -25,9 +25,9 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteConnection; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteConnection; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaPlus; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java index 484708ee4559..b23dee607cc1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.extensions.sql.impl; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA_FACTORY; -import static org.apache.beam.vendor.calcite.v1_28_0.org.codehaus.commons.compiler.CompilerFactoryFactory.getDefaultCompilerFactory; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA_FACTORY; +import static org.apache.beam.vendor.calcite.v1_40_0.org.codehaus.commons.compiler.CompilerFactoryFactory.getDefaultCompilerFactory; import com.google.auto.service.AutoService; import java.sql.Connection; @@ -33,18 +33,18 @@ import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.EnumerableRules; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.AvaticaFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteConnection; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.Driver; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollationTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CoreRules; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.Hook; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.EnumerableRules; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.AvaticaFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteConnection; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.Driver; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelCollationTraitDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.CoreRules; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.Hook; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSet; /** * Calcite JDBC driver with Beam defaults. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcFactory.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcFactory.java index 166c6720864b..d6b7273110b0 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcFactory.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcFactory.java @@ -18,26 +18,26 @@ package org.apache.beam.sdk.extensions.sql.impl; import static org.apache.beam.sdk.extensions.sql.impl.JdbcDriver.TOP_LEVEL_BEAM_SCHEMA; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.BuiltInConnectionProperty.TIME_ZONE; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.CalciteConnectionProperty.LEX; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.CalciteConnectionProperty.PARSER_FACTORY; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA_FACTORY; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.CalciteConnectionProperty.TYPE_SYSTEM; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.BuiltInConnectionProperty.TIME_ZONE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.config.CalciteConnectionProperty.LEX; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.config.CalciteConnectionProperty.PARSER_FACTORY; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA_FACTORY; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.config.CalciteConnectionProperty.TYPE_SYSTEM; import java.util.Properties; import org.apache.beam.sdk.extensions.sql.impl.parser.BeamSqlParser; import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRelDataTypeSystem; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.util.ReleaseInfo; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.java.JavaTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.AvaticaConnection; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.AvaticaFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.ConnectionProperty; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.UnregisteredDriver; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.Lex; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.AvaticaConnection; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.AvaticaFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.ConnectionProperty; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.UnregisteredDriver; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.config.Lex; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; import org.checkerframework.checker.nullness.qual.Nullable; /** @@ -46,9 +46,9 @@ * *

The purpose of this class is to intercept the connection creation and force a cache-less root * schema ({@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.SimpleCalciteSchema}). Otherwise + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.SimpleCalciteSchema}). Otherwise * Calcite uses {@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CachingCalciteSchema} that eagerly + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CachingCalciteSchema} that eagerly * caches table information. This behavior does not work well for dynamic table providers. */ class JdbcFactory extends CalciteFactoryWrapper { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/QueryPlanner.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/QueryPlanner.java index a6e6405e1c3d..0f0f8970a3ab 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/QueryPlanner.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/QueryPlanner.java @@ -22,8 +22,8 @@ import java.util.List; import java.util.Map; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java index b87635efcdaa..b92e2ce12556 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.impl; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import java.lang.reflect.Constructor; import java.lang.reflect.Method; @@ -27,29 +27,29 @@ import java.util.Arrays; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.CallImplementor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.NullPolicy; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.ReflectiveCallNotNullImplementor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.RexImpTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.RexToLixTranslator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.ByteString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.function.SemiStrict; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.function.Strict; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expression; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expressions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.ImplementableFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.ScalarFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperatorBinding; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.CallImplementor; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.NullPolicy; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.ReflectiveCallNotNullImplementor; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.RexImpTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.RexToLixTranslator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.util.ByteString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.function.SemiStrict; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.function.Strict; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Expression; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Expressions; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.ImplementableFunction; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.ScalarFunction; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperatorBinding; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMultimap; /** * Beam-customized version from {@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.impl.ScalarFunctionImpl} , to + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.impl.ScalarFunctionImpl} , to * address BEAM-5921. */ @SuppressWarnings({ @@ -81,7 +81,7 @@ public String getJarPath() { } /** - * Creates {@link org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function} for + * Creates {@link org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function} for * each method in a given class. */ public static ImmutableMultimap createAll(Class clazz) { @@ -100,7 +100,7 @@ public static ImmutableMultimap createAll(Class clazz) { } /** - * Creates {@link org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function} from + * Creates {@link org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function} from * given method. When {@code eval} method does not suit, {@code null} is returned. * * @param method method that is used to implement the function diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java index b8081d0c3312..3196667cb8cb 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java @@ -28,8 +28,8 @@ import org.apache.beam.sdk.extensions.sql.TableNameExtractionUtils; import org.apache.beam.sdk.extensions.sql.meta.CustomTableResolver; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** Utils to wire up the custom table resolution into Calcite's planner. */ @@ -168,7 +168,7 @@ private static List tablesForSchema( */ private static class SchemaWithName { String name; - org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema schema; + org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema schema; static SchemaWithName create(JdbcConnection connection, String name) { SchemaWithName schemaWithName = new SchemaWithName(); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java index bb19f0eca6fc..3b387a8d1a30 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java @@ -27,12 +27,12 @@ import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.AggImplementor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.AggregateFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.FunctionParameter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.ImplementableAggFunction; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.AggImplementor; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.AggregateFunction; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.FunctionParameter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.ImplementableAggFunction; /** Implement {@link AggregateFunction} to take a {@link CombineFn} as UDAF. */ @Internal diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java index cdb70fb0d03b..95fddf680279 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java @@ -18,9 +18,9 @@ package org.apache.beam.sdk.extensions.sql.impl; import java.lang.reflect.Method; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.TranslatableTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.impl.TableMacroImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.TranslatableTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.impl.TableMacroImpl; /** Beam-customized facade behind {@link Function} to address BEAM-5921. */ @SuppressWarnings({ @@ -31,7 +31,7 @@ class UdfImpl { private UdfImpl() {} /** - * Creates {@link org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function} from + * Creates {@link org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function} from * given class. * *

If a method of the given name is not found or it does not suit, returns {@code null}. @@ -49,7 +49,7 @@ public static Function create(Class clazz, String methodName) { } /** - * Creates {@link org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function} from + * Creates {@link org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function} from * given method. * * @param method method that is used to implement the function diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java index a37a5b69eaf9..fa6293bbb175 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java @@ -24,12 +24,12 @@ import java.util.ArrayList; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.FunctionParameter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.impl.ReflectiveFunctionBase; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.ReflectUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.FunctionParameter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.impl.ReflectiveFunctionBase; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.ReflectUtil; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** Beam-customized version from {@link ReflectiveFunctionBase}, to address BEAM-5921. */ @@ -101,7 +101,7 @@ public static ParameterListBuilder builder() { /** * Helps build lists of {@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.FunctionParameter}. + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.FunctionParameter}. */ public static class ParameterListBuilder { final List builder = new ArrayList<>(); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ZetaSqlUserDefinedSQLNativeTableValuedFunction.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ZetaSqlUserDefinedSQLNativeTableValuedFunction.java index 7beb33a16f76..6295027c9939 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ZetaSqlUserDefinedSQLNativeTableValuedFunction.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ZetaSqlUserDefinedSQLNativeTableValuedFunction.java @@ -19,13 +19,13 @@ import java.util.List; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlOperandTypeChecker; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlOperandTypeInference; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlOperandTypeChecker; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlOperandTypeInference; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlReturnTypeInference; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction; /** This is a class to indicate that a TVF is a ZetaSQL SQL native UDTVF. */ @Internal diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPCall.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPCall.java index 64689d219786..228c2e892a97 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPCall.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPCall.java @@ -19,11 +19,11 @@ import java.util.ArrayList; import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexPatternFieldRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexPatternFieldRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; /** * A {@code CEPCall} instance represents an operation (node) that contains an operator and a list of diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPFieldRef.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPFieldRef.java index 26274e23318a..78d15b8f9fbf 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPFieldRef.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPFieldRef.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.impl.cep; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexPatternFieldRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexPatternFieldRef; /** * A {@code CEPFieldRef} instance represents a node that points to a specified field in a {@code diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java index 911a3fa3e23f..7f33afdcf511 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java @@ -20,7 +20,7 @@ import java.math.BigDecimal; import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; import org.joda.time.ReadableDateTime; /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperation.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperation.java index 3ac8d0c4fe22..ba07cac2359e 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperation.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperation.java @@ -19,10 +19,10 @@ import java.io.Serializable; import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexPatternFieldRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexPatternFieldRef; /** * {@code CEPOperation} is the base class for the evaluation operations defined in the {@code diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperator.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperator.java index 14cff75875d9..c13a7261ae4d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperator.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperator.java @@ -19,8 +19,8 @@ import java.io.Serializable; import java.util.Map; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java index 4f802c460346..2e2150f25904 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java @@ -20,7 +20,7 @@ import java.io.Serializable; import javax.annotation.Nullable; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; /** Core pattern class that stores the definition of a single pattern. */ @SuppressWarnings({ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java index 3948f2191787..c921974fb8d2 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java @@ -21,14 +21,14 @@ import java.util.List; import java.util.Map; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelFieldCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.ImmutableBitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelCollation; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelFieldCollation; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.ImmutableBitSet; /** * Some utility methods for transforming Calcite's constructs into our own Beam constructs (for diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/OrderKey.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/OrderKey.java index d60afd2462f3..2e34074e7b9f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/OrderKey.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/OrderKey.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.extensions.sql.impl.cep; import java.io.Serializable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelFieldCollation; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelFieldCollation; /** * The {@code OrderKey} class stores the information to sort a column. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamSqlParser.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamSqlParser.java index 5ec9bb2ae4b4..25c0639a4a44 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamSqlParser.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamSqlParser.java @@ -19,10 +19,10 @@ import java.io.Reader; import org.apache.beam.sdk.extensions.sql.impl.parser.impl.BeamSqlParserImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.server.DdlExecutor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlAbstractParserImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserImplFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.server.DdlExecutor; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlAbstractParserImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserImplFactory; public class BeamSqlParser { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCheckConstraint.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCheckConstraint.java index 176085ed29f0..05ef4c8e1bf6 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCheckConstraint.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCheckConstraint.java @@ -18,15 +18,15 @@ package org.apache.beam.sdk.extensions.sql.impl.parser; import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.ImmutableNullableList; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.ImmutableNullableList; /** * Parse tree for {@code UNIQUE}, {@code PRIMARY KEY} constraints. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlColumnDeclaration.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlColumnDeclaration.java index bf8907b9d099..d6561e3b2841 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlColumnDeclaration.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlColumnDeclaration.java @@ -18,15 +18,15 @@ package org.apache.beam.sdk.extensions.sql.impl.parser; import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlDataTypeSpec; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlDataTypeSpec; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** Parse tree for column. */ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java index c1d96eea7bae..5626520f21dd 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java @@ -19,7 +19,7 @@ import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import java.util.Collections; @@ -28,20 +28,20 @@ import java.util.Map; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCreate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNodeList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlCreate; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNodeList; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java index 9938ad0e699c..c2524e3c9867 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java @@ -18,25 +18,25 @@ package org.apache.beam.sdk.extensions.sql.impl.parser; import static java.lang.String.format; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCreate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlCreate; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java index 2d98c03574ff..96b534e36d25 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.extensions.sql.impl.parser; import static org.apache.beam.sdk.schemas.Schema.toSchema; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; @@ -29,19 +29,19 @@ import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCreate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNodeList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlCreate; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNodeList; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.checkerframework.checker.nullness.qual.Nullable; /** Parse tree for {@code CREATE EXTERNAL TABLE} statement. */ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateFunction.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateFunction.java index 0378be8b9a74..978cb88a709d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateFunction.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateFunction.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import java.lang.reflect.Method; import java.util.Arrays; @@ -28,21 +28,21 @@ import org.apache.beam.sdk.extensions.sql.impl.ScalarFnReflector; import org.apache.beam.sdk.extensions.sql.impl.ScalarFunctionImpl; import org.apache.beam.sdk.extensions.sql.udf.ScalarFn; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCharStringLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCreate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlCharStringLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlCreate; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** Parse tree for {@code CREATE FUNCTION} statement. */ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java index e0378d859e2a..4c99b3aa3518 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java @@ -20,16 +20,16 @@ import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlDataTypeSpec; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.NlsString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Util; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlDataTypeSpec; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.NlsString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Util; import org.checkerframework.checker.nullness.qual.Nullable; /** Utilities concerning {@link SqlNode} for DDL. */ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java index ac1dfe5c2a83..8985484128cf 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java @@ -17,24 +17,24 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlDrop; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlDrop; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java index 639edc9ca15d..f4938b5fff45 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java @@ -18,25 +18,25 @@ package org.apache.beam.sdk.extensions.sql.impl.parser; import static java.lang.String.format; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlDrop; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlDrop; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropObject.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropObject.java index 16e2e536eaa5..1efcb373f1f8 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropObject.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropObject.java @@ -17,19 +17,19 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlDrop; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlDrop; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java index 5a62b0ee931e..18d06ef8aebc 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java @@ -17,11 +17,11 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; /** Parse tree for {@code DROP TABLE} statement. */ public class SqlDropTable extends SqlDropObject { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java index 239d117b5f72..f949a1fc9ae7 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java @@ -17,17 +17,17 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSetOption; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSetOption; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; /** SQL parse tree node to represent {@code SET} and {@code RESET} statements. */ @SuppressWarnings({ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java index 7088c7183027..1e96e3799ad1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java @@ -17,24 +17,24 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import java.util.Collections; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSetOption; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSetOption; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java index 40523e50a63f..b3bf122cadbf 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java @@ -18,25 +18,25 @@ package org.apache.beam.sdk.extensions.sql.impl.parser; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import java.util.Collections; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSetOption; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSetOption; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamCostModel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamCostModel.java index bceaf22805fe..4558acd40648 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamCostModel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamCostModel.java @@ -18,9 +18,9 @@ package org.apache.beam.sdk.extensions.sql.impl.planner; import java.util.Objects; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCost; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCostFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCost; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCostFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptUtil; import org.checkerframework.checker.nullness.qual.Nullable; /** @@ -218,7 +218,7 @@ public static BeamCostModel convertRelOptCost(RelOptCost ic) { /** * Implementation of {@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCostFactory} that creates + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCostFactory} that creates * {@link BeamCostModel}s. */ public static class Factory implements RelOptCostFactory { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamJavaTypeFactory.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamJavaTypeFactory.java index 0892bf6fe09c..b331f4d8e8b9 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamJavaTypeFactory.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamJavaTypeFactory.java @@ -18,12 +18,12 @@ package org.apache.beam.sdk.extensions.sql.impl.planner; import java.lang.reflect.Type; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.java.JavaTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.BasicSqlType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.IntervalSqlType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.BasicSqlType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.IntervalSqlType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; /** customized data type in Beam. */ public class BeamJavaTypeFactory extends JavaTypeFactoryImpl { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java index 37fc4b7078c4..fb7f738fd2cc 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java @@ -17,9 +17,9 @@ */ package org.apache.beam.sdk.extensions.sql.impl.planner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeSystemImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeSystemImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; /** customized data type in Beam. */ public class BeamRelDataTypeSystem extends RelDataTypeSystemImpl { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelMetadataQuery.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelMetadataQuery.java index 91257f4abd4c..3ee9242fb219 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelMetadataQuery.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelMetadataQuery.java @@ -17,9 +17,9 @@ */ package org.apache.beam.sdk.extensions.sql.impl.planner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.JaninoRelMetadataProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.JaninoRelMetadataProvider; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataQuery; public class BeamRelMetadataQuery extends RelMetadataQuery { private NodeStatsMetadata.Handler nodeStatsMetadataHandler; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java index 8d5b4d4fa08d..8382fcb6e382 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java @@ -42,12 +42,12 @@ import org.apache.beam.sdk.extensions.sql.impl.rule.BeamUnnestRule; import org.apache.beam.sdk.extensions.sql.impl.rule.BeamValuesRule; import org.apache.beam.sdk.extensions.sql.impl.rule.BeamWindowRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CoreRules; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.PruneEmptyRules; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSets; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.CoreRules; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.PruneEmptyRules; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** @@ -72,6 +72,7 @@ public class BeamRuleSets { // push a filter into a join CoreRules.FILTER_INTO_JOIN, + CoreRules.FILTER_SUB_QUERY_TO_CORRELATE, // push filter into the children of a join CoreRules.JOIN_CONDITION_PUSH, // push filter through an aggregation diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsMetadata.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsMetadata.java index c8d4d0e0a4c3..f42cb01174de 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsMetadata.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsMetadata.java @@ -18,12 +18,12 @@ package org.apache.beam.sdk.extensions.sql.impl.planner; import java.lang.reflect.Method; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Types; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.Metadata; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.MetadataDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.MetadataHandler; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Types; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.Metadata; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.MetadataDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.MetadataHandler; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataQuery; /** * This is a metadata used for row count and rate estimation. It extends Calcite's Metadata diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/RelMdNodeStats.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/RelMdNodeStats.java index 63787b514259..40db8b074efa 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/RelMdNodeStats.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/RelMdNodeStats.java @@ -22,13 +22,13 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; -import org.apache.beam.vendor.calcite.v1_28_0.com.google.common.collect.Table; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.MetadataDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.MetadataHandler; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.beam.vendor.calcite.v1_40_0.com.google.common.collect.Table; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.MetadataDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.MetadataHandler; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataProvider; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataQuery; /** * This is the implementation of NodeStatsMetadata. Methods to estimate rate and row count for @@ -75,7 +75,7 @@ private NodeStats getBeamNodeStats(BeamRelNode rel, BeamRelMetadataQuery mq) { // wraps the metadata provider with CachingRelMetadataProvider. However, // CachingRelMetadataProvider checks timestamp before returning previous results. Therefore, // there wouldn't be a problem in that case. - List> keys = + List> keys = mq.map.cellSet().stream() .filter(entry -> entry != null) .filter(entry -> entry.getValue() != null) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java index 626e680ba66e..cb2c9598f34f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java @@ -21,15 +21,15 @@ import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel; import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRelMetadataQuery; import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLocalRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexProgram; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Calc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLocalRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexProgram; /** BeamRelNode to replace {@code Project} and {@code Filter} node. */ @Internal diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java index a1880f6cb8c8..3f635cd081a7 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java @@ -52,14 +52,14 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Aggregate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.AggregateCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.ImmutableBitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Aggregate; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.AggregateCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.ImmutableBitSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java index 4895c1478766..ea8658e4d5f0 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java @@ -70,39 +70,39 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.DataContext; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.JavaRowFormat; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.PhysType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.PhysTypeImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.RexToLixTranslator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.java.JavaTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.ByteString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.QueryProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.BlockBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expression; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expressions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.MemberDeclaration; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.ParameterExpression; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Types; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPredicateList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexProgram; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexSimplify; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.SqlFunctions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlConformance; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlConformanceEnum; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.DataContext; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.JavaRowFormat; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.PhysType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.PhysTypeImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.RexToLixTranslator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.util.ByteString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.QueryProvider; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.BlockBuilder; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Expression; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Expressions; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.MemberDeclaration; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Types; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPredicateList; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Calc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexBuilder; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexProgram; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexSimplify; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.SqlFunctions; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.validate.SqlConformance; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.validate.SqlConformanceEnum; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRel.java index d8ef988fbf81..b16755d22180 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRel.java @@ -40,14 +40,14 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.CorrelationId; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Join; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.JoinRelType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.CorrelationId; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Join; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.JoinRelType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; /** * A {@code BeamJoinRel} which does CoGBK Join diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java index 7866d86971f9..c54ab14ba8d8 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java @@ -58,24 +58,24 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.EnumerableRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.EnumerableRelImplementor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.PhysType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.PhysTypeImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.Enumerable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.Linq4j; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.BlockBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expression; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expressions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.ConventionTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCost; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.EnumerableRel; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.EnumerableRelImplementor; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.PhysType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.PhysTypeImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.Enumerable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.Linq4j; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.BlockBuilder; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Expression; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Expressions; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.ConventionTraitDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCost; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.joda.time.ReadableInstant; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java index 7c292e7ec4ea..cc81d0c1d6d2 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java @@ -33,15 +33,15 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.prepare.Prepare; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.TableModify; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql2rel.RelStructuredTypeFlattener; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.prepare.Prepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.TableModify; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql2rel.RelStructuredTypeFlattener; import org.checkerframework.checker.nullness.qual.Nullable; /** BeamRelNode to replace a {@code TableModify} node. */ @@ -63,7 +63,7 @@ public BeamIOSinkRel( boolean flattened, BeamSqlTable sqlTable, Map pipelineOptions) { - super( + this( cluster, cluster.traitSetOf(BeamLogicalConvention.INSTANCE), table, @@ -72,6 +72,33 @@ public BeamIOSinkRel( operation, updateColumnList, sourceExpressionList, + flattened, + sqlTable, + pipelineOptions); + } + + /** For copy. */ + private BeamIOSinkRel( + RelOptCluster cluster, + RelTraitSet traitSet, + RelOptTable table, + Prepare.CatalogReader catalogReader, + RelNode child, + Operation operation, + @Nullable List updateColumnList, + @Nullable List sourceExpressionList, + boolean flattened, + BeamSqlTable sqlTable, + Map pipelineOptions) { + super( + cluster, + traitSet, + table, + catalogReader, + child, + operation, + updateColumnList, + sourceExpressionList, flattened); this.sqlTable = sqlTable; this.pipelineOptions = pipelineOptions; @@ -91,20 +118,18 @@ public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, BeamRelMetadataQ @Override public RelNode copy(RelTraitSet traitSet, List inputs) { boolean flattened = isFlattened() || isFlattening; - BeamIOSinkRel newRel = - new BeamIOSinkRel( - getCluster(), - getTable(), - getCatalogReader(), - sole(inputs), - getOperation(), - getUpdateColumnList(), - getSourceExpressionList(), - flattened, - sqlTable, - pipelineOptions); - newRel.traitSet = traitSet; - return newRel; + return new BeamIOSinkRel( + getCluster(), + traitSet, + getTable(), + getCatalogReader(), + sole(inputs), + getOperation(), + getUpdateColumnList(), + getSourceExpressionList(), + flattened, + sqlTable, + pipelineOptions); } @Override diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java index 989172769992..5a46414dbb7f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java @@ -32,15 +32,15 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCost; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.prepare.RelOptTableImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.TableScan; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCost; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.prepare.RelOptTableImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.TableScan; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; /** BeamRelNode to replace a {@code TableScan} node. */ public class BeamIOSourceRel extends TableScan implements BeamRelNode { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java index 38001368e0e6..b02ea788a620 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java @@ -25,12 +25,12 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Intersect; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.SetOp; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Intersect; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.SetOp; /** * {@code BeamRelNode} to replace a {@code Intersect} node. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java index d6e2e4fe27b1..7b5a8941f25d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java @@ -27,20 +27,20 @@ import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.volcano.RelSubset; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.CorrelationId; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Join; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.JoinRelType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexFieldAccess; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.volcano.RelSubset; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.CorrelationId; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Join; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.JoinRelType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexFieldAccess; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java index 8801d3dcd180..d51115d1f2a0 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java @@ -17,12 +17,12 @@ */ package org.apache.beam.sdk.extensions.sql.impl.rel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.ConventionTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTrait; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.ConventionTraitDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTrait; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; /** Convention for Beam SQL. */ @SuppressWarnings({ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java index d1b118b3f295..d2a738beacf3 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java @@ -49,17 +49,17 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Match; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.ImmutableBitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelCollation; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Match; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.ImmutableBitSet; import org.checkerframework.checker.nullness.qual.Nullable; /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java index 3966d5caddb6..1e8fd577a138 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java @@ -25,12 +25,12 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Minus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.SetOp; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Minus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.SetOp; /** * {@code BeamRelNode} to replace a {@code Minus} node. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamPushDownIOSourceRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamPushDownIOSourceRel.java index 06493333b12b..cc580ebd354d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamPushDownIOSourceRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamPushDownIOSourceRel.java @@ -34,11 +34,11 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelWriter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; public class BeamPushDownIOSourceRel extends BeamIOSourceRel { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java index 0a9f0f62781d..a1a9ae861b60 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java @@ -27,8 +27,8 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.checkerframework.checker.nullness.qual.Nullable; /** A {@link RelNode} that can also give a {@link PTransform} that implements the expression. */ @@ -85,8 +85,8 @@ default Map getPipelineOptions() { * estimate its NodeStats, it may need NodeStat of its inputs. However, it should not call this * directly (because maybe its inputs are not physical yet). It should call {@link * org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils#getNodeStats( - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode, - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery)} + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode, + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataQuery)} * instead. */ NodeStats estimateNodeStats(BeamRelMetadataQuery mq); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRel.java index ca9eceb78fcf..1c14ee2310b5 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRel.java @@ -32,14 +32,14 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.CorrelationId; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Join; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.JoinRelType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.CorrelationId; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Join; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.JoinRelType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRel.java index 95863d66a2c4..5b25139525ab 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRel.java @@ -26,13 +26,13 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.CorrelationId; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Join; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.JoinRelType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.CorrelationId; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Join; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.JoinRelType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; /** * A {@code BeamJoinRel} which does Lookup Join diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java index d94f228c1f99..aaa4d66011a6 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java @@ -51,18 +51,18 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollationImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelFieldCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Sort; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelCollation; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelCollationImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelFieldCollation; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Sort; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; import org.checkerframework.checker.nullness.qual.Nullable; /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java index 8bcb8389e563..43e9b7ff333b 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java @@ -31,9 +31,9 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.volcano.RelSubset; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.volcano.RelSubset; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.checkerframework.checker.nullness.qual.Nullable; /** Utilities for {@code BeamRelNode}. */ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java index a975f7fdabdc..20fcf25cb07b 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java @@ -52,17 +52,17 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.TableFunctionScan; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelColumnMapping; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.TableFunctionScan; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelColumnMapping; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.Duration; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java index 55dc9afe4e17..ce6a11544da6 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java @@ -31,11 +31,11 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Uncollect; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Uncollect; /** {@link BeamRelNode} to implement an uncorrelated {@link Uncollect}, aka UNNEST. */ @SuppressWarnings({ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java index 4cb8e81c40b0..a0b710213a51 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java @@ -26,12 +26,12 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.SetOp; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Union; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.SetOp; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Union; /** * {@link BeamRelNode} to replace a {@link Union}. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java index 157e5b9d1017..a37ade47f925 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java @@ -35,16 +35,16 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Correlate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.JoinRelType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Uncollect; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlValidatorUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Correlate; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.JoinRelType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Uncollect; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.validate.SqlValidatorUtil; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java index 6cd1716d5c5d..6163dbe770ae 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java @@ -35,13 +35,13 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Values; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Values; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java index de7ef2a29f69..b47ff66329df 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java @@ -42,17 +42,17 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelFieldCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.AggregateCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Window; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelFieldCollation; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.AggregateCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Window; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; /** @@ -431,6 +431,11 @@ public RelNode copy(RelTraitSet traitSet, List inputs) { return this.copy(traitSet, sole(inputs), this.constants, this.rowType, this.groups); } + @Override + public Window copy(List constants) { + return this.copy(traitSet, input, constants, this.rowType, this.groups); + } + public BeamWindowRel copy( RelTraitSet traitSet, RelNode input, diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/CalcRelSplitter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/CalcRelSplitter.java index 4cad8f671690..a6bddd142cfc 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/CalcRelSplitter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/CalcRelSplitter.java @@ -24,32 +24,32 @@ import java.util.Collections; import java.util.List; import java.util.Set; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalCalc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexDynamicParam; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexFieldAccess; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLocalRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexProgram; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexShuttle; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexVisitorImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Litmus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Util; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.graph.DefaultDirectedGraph; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.graph.DefaultEdge; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.graph.DirectedGraph; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.graph.TopologicalOrderIterator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Calc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalCalc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexDynamicParam; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexFieldAccess; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLocalRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexProgram; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexShuttle; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexVisitorImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RelBuilder; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Litmus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Util; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.graph.DefaultDirectedGraph; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.graph.DefaultEdge; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.graph.DirectedGraph; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.graph.TopologicalOrderIterator; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Ints; import org.checkerframework.checker.nullness.qual.Nullable; @@ -60,7 +60,7 @@ * cannot all be implemented by a single concrete {@link RelNode}. * *

This is a copy of {@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CalcRelSplitter} modified to + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.CalcRelSplitter} modified to * work with Beam. TODO(CALCITE-4538) consider contributing these changes back upstream. * *

For example, the Java and Fennel calculator do not implement an identical set of operators. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java index 6c86d780d021..7a973d9c8b33 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java @@ -18,7 +18,7 @@ /** * BeamSQL specified nodes, to replace {@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode}. + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode}. */ @DefaultAnnotation(NonNull.class) package org.apache.beam.sdk.extensions.sql.impl.rel; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java index 4ec26845549c..7b5d4c1c05d1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java @@ -21,16 +21,16 @@ import java.util.List; import java.util.Set; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIOSourceRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.volcano.RelSubset; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.SingleRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Aggregate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Filter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Project; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.RelFactories; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.AggregateProjectMergeRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelBuilderFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.volcano.RelSubset; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.SingleRel; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Aggregate; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Filter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Project; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.RelFactories; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.AggregateProjectMergeRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RelBuilderFactory; /** * This rule is essentially a wrapper around Calcite's {@code AggregateProjectMergeRule}. In the diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java index 2e434f212b7c..b737ecdbd800 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java @@ -25,18 +25,18 @@ import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Aggregate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Project; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.RelFactories; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelBuilderFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.ImmutableBitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Aggregate; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Project; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.RelFactories; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RelBuilderFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.ImmutableBitSet; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java index 57fa2ca94c1c..e2df3c3a685f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java @@ -22,19 +22,19 @@ import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamAggregationRel; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.volcano.RelSubset; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Aggregate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Filter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Project; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.RelFactories; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelBuilderFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.volcano.RelSubset; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Aggregate; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Calc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Filter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Project; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.RelFactories; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RelBuilderFactory; /** * Aggregation rule that doesn't include projection. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcMergeRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcMergeRule.java index 6f24cf658e1a..dffdc7bd993f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcMergeRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcMergeRule.java @@ -18,11 +18,11 @@ package org.apache.beam.sdk.extensions.sql.impl.rule; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleOperand; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CalcMergeRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CoreRules; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleOperand; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.CalcMergeRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.CoreRules; /** * Planner rule to merge a {@link BeamCalcRel} with a {@link BeamCalcRel}. Subset of {@link diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcRule.java index 00871b74530b..1319e20fdb4c 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcRule.java @@ -20,15 +20,15 @@ import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalCalc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexOver; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Calc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalCalc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexOver; /** A {@code ConverterRule} to replace {@link Calc} with {@link BeamCalcRel}. */ public class BeamCalcRule extends ConverterRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcSplittingRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcSplittingRule.java index cc4ba1f002b0..3f0f02c76657 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcSplittingRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcSplittingRule.java @@ -18,13 +18,13 @@ package org.apache.beam.sdk.extensions.sql.impl.rule; import org.apache.beam.sdk.extensions.sql.impl.rel.CalcRelSplitter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.RelFactories; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalCalc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Calc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.RelFactories; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalCalc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCoGBKJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCoGBKJoinRule.java index 195a2675f957..f44839de062d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCoGBKJoinRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCoGBKJoinRule.java @@ -21,12 +21,12 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRel; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Join; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.RelFactories; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Join; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.RelFactories; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalJoin; /** * Rule to convert {@code LogicalJoin} node to {@code BeamCoGBKJoinRel} node. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamEnumerableConverterRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamEnumerableConverterRule.java index d6b7f4f65eaa..cc508c50d9c9 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamEnumerableConverterRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamEnumerableConverterRule.java @@ -20,10 +20,10 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamEnumerableConverter; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.EnumerableConvention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.EnumerableConvention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; /** A {@code ConverterRule} to Convert {@link BeamRelNode} to {@link EnumerableConvention}. */ public class BeamEnumerableConverterRule extends ConverterRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java index 5abce12fa200..4c37d1be334b 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java @@ -38,25 +38,25 @@ import org.apache.beam.sdk.schemas.FieldAccessDescriptor.FieldDescriptor; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.utils.SelectHelpers; -import org.apache.beam.vendor.calcite.v1_28_0.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.RelFactories; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelRecordType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLocalRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexProgram; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelBuilderFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Calc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.RelFactories; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelRecordType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLocalRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexProgram; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexUtil; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RelBuilder; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RelBuilderFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; @SuppressWarnings({ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java index 0f7698687a08..9c2f9ed13382 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java @@ -20,9 +20,9 @@ import java.util.Arrays; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIOSinkRel; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.TableModify; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.TableModify; /** A {@code ConverterRule} to replace {@link TableModify} with {@link BeamIOSinkRel}. */ public class BeamIOSinkRule extends ConverterRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.java index cb0bcb36e75d..dd0c570dd7f0 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.java @@ -20,11 +20,11 @@ import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIntersectRel; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Intersect; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalIntersect; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Intersect; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalIntersect; /** {@code ConverterRule} to replace {@code Intersect} with {@code BeamIntersectRel}. */ public class BeamIntersectRule extends ConverterRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinAssociateRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinAssociateRule.java index ccf7a1e94bd9..868dfff756b6 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinAssociateRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinAssociateRule.java @@ -18,15 +18,15 @@ package org.apache.beam.sdk.extensions.sql.impl.rule; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Join; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.RelFactories; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.JoinAssociateRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelBuilderFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Join; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.RelFactories; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.JoinAssociateRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RelBuilderFactory; /** * This is very similar to {@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.JoinAssociateRule}. It only + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.JoinAssociateRule}. It only * checks if the resulting condition is supported before transforming. */ public class BeamJoinAssociateRule extends JoinAssociateRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinPushThroughJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinPushThroughJoinRule.java index 7ecdc8e2e2ab..649cce2f39f8 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinPushThroughJoinRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinPushThroughJoinRule.java @@ -18,17 +18,17 @@ package org.apache.beam.sdk.extensions.sql.impl.rule; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Join; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.RelFactories; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalJoin; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.JoinPushThroughJoinRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Join; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.RelFactories; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.JoinPushThroughJoinRule; /** * This is exactly similar to {@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.JoinPushThroughJoinRule}. It + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.JoinPushThroughJoinRule}. It * only checks if the condition of the new bottom join is supported. */ public class BeamJoinPushThroughJoinRule extends RelOptRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMatchRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMatchRule.java index c75d06e30d1a..1efd08769b00 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMatchRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMatchRule.java @@ -19,11 +19,11 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamMatchRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Match; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalMatch; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Match; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalMatch; /** {@code ConverterRule} to replace {@code Match} with {@code BeamMatchRel}. */ public class BeamMatchRule extends ConverterRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.java index 18d37584b8a8..2066546ba4a8 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.java @@ -20,11 +20,11 @@ import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamMinusRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Minus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalMinus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Minus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalMinus; /** {@code ConverterRule} to replace {@code Minus} with {@code BeamMinusRel}. */ public class BeamMinusRule extends ConverterRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputJoinRule.java index f8ff4adc762b..7cac7c0867e3 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputJoinRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputJoinRule.java @@ -21,12 +21,12 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSideInputJoinRel; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Join; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.RelFactories; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Join; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.RelFactories; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalJoin; /** * Rule to convert {@code LogicalJoin} node to {@code BeamSideInputJoinRel} node. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputLookupJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputLookupJoinRule.java index b164ed40c59e..b95ec6df0d7f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputLookupJoinRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputLookupJoinRule.java @@ -20,12 +20,12 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRel; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSideInputLookupJoinRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Join; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Join; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalJoin; /** * Rule to convert {@code LogicalJoin} node to {@code BeamSideInputLookupJoinRel} node. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java index 446b75e4fd23..8350498185c3 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java @@ -19,11 +19,11 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSortRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Sort; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalSort; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Sort; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalSort; /** {@code ConverterRule} to replace {@code Sort} with {@code BeamSortRel}. */ public class BeamSortRule extends ConverterRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java index 9d73f8abc4ff..e23bfebf3313 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java @@ -24,11 +24,11 @@ import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamTableFunctionScanRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.TableFunctionScan; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalTableFunctionScan; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.TableFunctionScan; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalTableFunctionScan; /** * This is the conveter rule that converts a Calcite {@code TableFunctionScan} to Beam {@code diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUncollectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUncollectRule.java index a951cf291183..daf6ebcc6f87 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUncollectRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUncollectRule.java @@ -19,10 +19,10 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamUncollectRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Uncollect; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Uncollect; /** A {@code ConverterRule} to replace {@link Uncollect} with {@link BeamUncollectRule}. */ public class BeamUncollectRule extends ConverterRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.java index 09ab2f0dd301..e41e2b5a6235 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.java @@ -19,15 +19,15 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamUnionRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Union; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalUnion; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Union; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalUnion; /** * A {@code ConverterRule} to replace {@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Union} with {@link + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Union} with {@link * BeamUnionRule}. */ public class BeamUnionRule extends ConverterRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnnestRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnnestRule.java index e2fa2bc6f586..56feed20a634 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnnestRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnnestRule.java @@ -19,18 +19,18 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamUnnestRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.volcano.RelSubset; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.SingleRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Correlate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.JoinRelType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Uncollect; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalCorrelate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalProject; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexFieldAccess; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.volcano.RelSubset; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.SingleRel; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Correlate; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.JoinRelType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Uncollect; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalCorrelate; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalProject; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexFieldAccess; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java index 3b01e80f902d..bfa4c477dc85 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java @@ -19,11 +19,11 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamValuesRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Values; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalValues; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Values; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalValues; /** {@code ConverterRule} to replace {@code Values} with {@code BeamValuesRel}. */ public class BeamValuesRule extends ConverterRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamWindowRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamWindowRule.java index 3f554827ce53..c442ae3d1947 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamWindowRule.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamWindowRule.java @@ -19,11 +19,11 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamWindowRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Window; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalWindow; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.Convention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.convert.ConverterRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Window; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.logical.LogicalWindow; /** A {@code ConverterRule} to replace {@link Window} with {@link BeamWindowRel}. */ public class BeamWindowRule extends ConverterRule { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java index f64c32cef253..39e1cf996843 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java @@ -19,14 +19,14 @@ import java.util.List; import java.util.Map; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelHintsPropagator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleOperand; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelBuilder; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelHintsPropagator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRuleOperand; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RelBuilder; /** * This is a class to catch the built join and check if it is a legal join before passing it to the diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java index b10dce86b42f..8a7179731138 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java @@ -17,7 +17,7 @@ */ /** - * {@link org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule} to generate + * {@link org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule} to generate * {@link org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode}. */ @DefaultAnnotation(NonNull.class) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java index 3dbd14aedf06..d9dbf881850e 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java @@ -35,8 +35,8 @@ import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.Schema.TypeName; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.ByteString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.NlsString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.util.ByteString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.NlsString; import org.apache.commons.csv.CSVFormat; import org.apache.commons.csv.CSVParser; import org.apache.commons.csv.CSVPrinter; @@ -108,6 +108,7 @@ public static String beamRow2CsvLine(Row row, CSVFormat csvFormat) { * @return The casted object in Schema.Field.Type. */ public static Object autoCastField(Schema.Field field, @Nullable Object rawObj) { + // handle null if (rawObj == null) { if (!field.getType().getNullable()) { throw new IllegalArgumentException(String.format("Field %s not nullable", field.getName())); @@ -116,12 +117,14 @@ public static Object autoCastField(Schema.Field field, @Nullable Object rawObj) } FieldType type = field.getType(); + // handle NlsString if (CalciteUtils.isStringType(type)) { if (rawObj instanceof NlsString) { return ((NlsString) rawObj).getValue(); } else { return rawObj; } + // handle date/time } else if (CalciteUtils.DATE.typesEqual(type) || CalciteUtils.NULLABLE_DATE.typesEqual(type)) { if (rawObj instanceof GregorianCalendar) { // used by the SQL CLI GregorianCalendar calendar = (GregorianCalendar) rawObj; @@ -143,6 +146,7 @@ public static Object autoCastField(Schema.Field field, @Nullable Object rawObj) } else if (CalciteUtils.isDateTimeType(type)) { // Internal representation of Date in Calcite is convertible to Joda's Datetime. return new DateTime(rawObj); + // handle decimal } else if (type.getTypeName().isNumericType() && ((rawObj instanceof String) || (rawObj instanceof BigDecimal && type.getTypeName() != TypeName.DECIMAL))) { @@ -168,8 +172,14 @@ public static Object autoCastField(Schema.Field field, @Nullable Object rawObj) String.format("Column type %s is not supported yet!", type)); } } else if (type.getTypeName().isPrimitiveType()) { + // handle bytes represented by ByteString if (TypeName.BYTES.equals(type.getTypeName()) && rawObj instanceof ByteString) { return ((ByteString) rawObj).getBytes(); + // handle Float <-> Double mixed use + } else if (TypeName.FLOAT.equals(type.getTypeName()) && rawObj instanceof Double) { + return ((Double) rawObj).floatValue(); + } else if (TypeName.DOUBLE.equals(type.getTypeName()) && rawObj instanceof Float) { + return ((Float) rawObj).doubleValue(); } } return rawObj; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java index d25f98729bd4..3f2d98f94b2d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java @@ -34,10 +34,10 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; /** Collections of {@code PTransform} and {@code DoFn} used to perform JOIN operation. */ @SuppressWarnings({ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java index 294364cc8a78..ebd42a9ee6b2 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java @@ -27,8 +27,8 @@ import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.AggregateCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlUserDefinedAggFunction; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.AggregateCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.validate.SqlUserDefinedAggFunction; import org.checkerframework.checker.nullness.qual.Nullable; /** Wrapper {@link CombineFn}s for aggregation function calls. */ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java index 4727023c8320..aeee7542fcb3 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.SqlFunctions; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.SqlFunctions; /** * {@link Combine.CombineFn} for Covariance on {@link Number} types. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java index 65e2fa92ee95..f96e7ce750a1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.SqlFunctions; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.SqlFunctions; /** * {@link Combine.CombineFn} for Variance on {@link Number} types. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java index 6b4e3475f73c..9dd36c133bde 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java @@ -19,7 +19,7 @@ import com.google.auto.service.AutoService; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.function.Strict; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.function.Strict; /** Hash Functions. */ @AutoService(BeamBuiltinFunctionProvider.class) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinStringFunctions.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinStringFunctions.java index 1ed7cf186cc8..48463e755489 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinStringFunctions.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinStringFunctions.java @@ -24,7 +24,7 @@ import org.apache.beam.repackaged.core.org.apache.commons.lang3.ArrayUtils; import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils; import org.apache.beam.sdk.schemas.Schema.TypeName; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.function.Strict; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.function.Strict; import org.apache.commons.codec.DecoderException; import org.apache.commons.codec.binary.Hex; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java index 610dd4f6888a..5bcac6ad256f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java @@ -29,12 +29,12 @@ import org.apache.beam.sdk.schemas.logicaltypes.PassThroughLogicalType; import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.ByteString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlTypeNameSpec; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.util.ByteString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlTypeNameSpec; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableBiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexFieldAccess.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexFieldAccess.java index 50a8062e1e25..d2f450ac9fc3 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexFieldAccess.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexFieldAccess.java @@ -20,8 +20,8 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexFieldAccess; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexFieldAccess; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; /** SerializableRexFieldAccess. */ public class SerializableRexFieldAccess extends SerializableRexNode { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexInputRef.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexInputRef.java index 2630dd7e1d3b..9520282c1aa7 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexInputRef.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexInputRef.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.impl.utils; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; /** SerializableRexInputRef. */ public class SerializableRexInputRef extends SerializableRexNode { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexNode.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexNode.java index 0bbcd7f17956..cb09c43bac47 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexNode.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexNode.java @@ -18,9 +18,9 @@ package org.apache.beam.sdk.extensions.sql.impl.utils; import java.io.Serializable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexFieldAccess; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexFieldAccess; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; /** SerializableRexNode. */ public abstract class SerializableRexNode implements Serializable { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BaseBeamTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BaseBeamTable.java index 31944ebee134..019ada39a425 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BaseBeamTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BaseBeamTable.java @@ -23,7 +23,7 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; /** Basic implementation of {@link BeamSqlTable}. */ public abstract class BaseBeamTable implements BeamSqlTable { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTable.java index f3bd34d8b1b9..e658db8c3d82 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTable.java @@ -25,7 +25,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; /** This interface defines a Beam Sql Table. */ public interface BeamSqlTable { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTableFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTableFilter.java index 8cdd430ad01b..7dd0088ea381 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTableFilter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTableFilter.java @@ -18,8 +18,8 @@ package org.apache.beam.sdk.extensions.sql.meta; import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; /** This interface defines Beam SQL Table Filter. */ public interface BeamSqlTableFilter { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/DefaultTableFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/DefaultTableFilter.java index 5a63f51ba399..e23e202511e6 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/DefaultTableFilter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/DefaultTableFilter.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.extensions.sql.meta; import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; /** * This default implementation of {@link BeamSqlTableFilter} interface. Assumes that predicate diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java index 91ba2c708d28..0b03cd73c51c 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java @@ -19,17 +19,17 @@ import java.util.List; import java.util.Map; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.Casing; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.NullCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlAbstractDateTimeLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlDialect; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIntervalLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlTimestampLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.dialect.BigQuerySqlDialect; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.util.Casing; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.config.NullCollation; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlAbstractDateTimeLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlDialect; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIntervalLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlTimestampLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.dialect.BigQuerySqlDialect; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java index 5ab6bbcacec3..143bab558866 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rel2sql.SqlImplementor.POS; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rel2sql.SqlImplementor.POS; import java.util.HashMap; import java.util.Map; @@ -27,27 +27,27 @@ import org.apache.beam.repackaged.core.org.apache.commons.lang3.text.translate.EntityArrays; import org.apache.beam.repackaged.core.org.apache.commons.lang3.text.translate.JavaUnicodeEscaper; import org.apache.beam.repackaged.core.org.apache.commons.lang3.text.translate.LookupTranslator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.ByteString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.TimeUnitRange; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rel2sql.RelToSqlConverter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rel2sql.SqlImplementor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexDynamicParam; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLocalRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexProgram; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlDynamicParam; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeFamily; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.BitString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.TimestampString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.util.ByteString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.util.TimeUnitRange; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rel2sql.RelToSqlConverter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rel2sql.SqlImplementor; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexDynamicParam; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLocalRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexProgram; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlDynamicParam; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.BitString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.TimestampString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; @@ -128,9 +128,11 @@ public SqlNode toSql(RexProgram program, RexNode rex) { } else if (SqlKind.SEARCH.equals(rex.getKind())) { // Workaround CALCITE-4716 RexCall search = (RexCall) rex; - RexLocalRef ref = (RexLocalRef) search.operands.get(1); - RexLiteral literal = (RexLiteral) program.getExprList().get(ref.getIndex()); - rex = search.clone(search.getType(), ImmutableList.of(search.operands.get(0), literal)); + if (search.operands.get(1) instanceof RexLocalRef) { + RexLocalRef ref = (RexLocalRef) search.operands.get(1); + RexLiteral literal = (RexLiteral) program.getExprList().get(ref.getIndex()); + rex = search.clone(search.getType(), ImmutableList.of(search.operands.get(0), literal)); + } } return super.toSql(program, rex); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java index 36450e3914cc..7ae198fd1cd7 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java @@ -17,29 +17,29 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.AND; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.BETWEEN; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.CAST; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.COMPARISON; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.DIVIDE; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.LIKE; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.MINUS; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.MOD; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.OR; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.PLUS; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.TIMES; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.AND; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.BETWEEN; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.CAST; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.COMPARISON; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.DIVIDE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.LIKE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.MINUS; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.MOD; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.OR; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.PLUS; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.TIMES; import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; @SuppressWarnings({ diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java index 1898c28f670c..bc10d6b99a95 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java @@ -47,12 +47,12 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rel2sql.SqlImplementor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rel2sql.SqlImplementor; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.slf4j.Logger; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableFilter.java index f419896f2ed5..d42fff695b2f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableFilter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableFilter.java @@ -20,7 +20,7 @@ import static java.util.stream.Collectors.toList; import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.KEY; import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.byteStringUtf8; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.LIKE; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.LIKE; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import com.google.bigtable.v2.RowFilter; @@ -28,10 +28,10 @@ import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; /** * BigtableFilter for queries with WHERE clause. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTable.java index 60c722d32d2d..e24b9a437e36 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTable.java @@ -46,7 +46,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; public class BigtableTable extends SchemaBaseBeamTable implements Serializable { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datagen/DataGeneratorRowFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datagen/DataGeneratorRowFn.java index e5f720b4ced5..944d8ec9b002 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datagen/DataGeneratorRowFn.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datagen/DataGeneratorRowFn.java @@ -29,8 +29,8 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.commons.lang3.RandomStringUtils; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.commons.lang3.RandomStringUtils; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java index b3854ced46c6..834aed0e9048 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java @@ -19,18 +19,18 @@ import static org.apache.beam.sdk.io.iceberg.FilterUtils.SUPPORTED_OPS; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.AND; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.OR; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.AND; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.OR; import java.util.List; import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.commons.lang3.tuple.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.commons.lang3.tuple.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java index 596a1d6d0457..000ca50e4309 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java @@ -38,13 +38,13 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rel2sql.SqlImplementor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.dialect.BigQuerySqlDialect; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rel2sql.SqlImplementor; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.dialect.BigQuerySqlDialect; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java index 576b623b28ab..34f56082324a 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java @@ -17,9 +17,9 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.mongodb; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.AND; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.COMPARISON; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.OR; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.AND; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.COMPARISON; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.OR; import com.mongodb.client.model.Filters; import java.io.Serializable; @@ -56,12 +56,12 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.bson.Document; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java index 9646b91a0386..771ebad1a996 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java @@ -17,18 +17,18 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.test; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.COMPARISON; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.IN; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.COMPARISON; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.IN; import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java index 5e0851a3685e..4ca1ceeb9853 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java @@ -55,11 +55,11 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; /** * Test in-memory table provider for use in tests. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java index 5433f9be380b..c18df6b0d3f8 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_28_0.org.codehaus.commons.compiler.properties b/sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_40_0.org.codehaus.commons.compiler.properties similarity index 93% rename from sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_28_0.org.codehaus.commons.compiler.properties rename to sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_40_0.org.codehaus.commons.compiler.properties index bc2e006e9bb6..980241fe8488 100644 --- a/sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_28_0.org.codehaus.commons.compiler.properties +++ b/sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_40_0.org.codehaus.commons.compiler.properties @@ -15,4 +15,4 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -compilerFactory=org.apache.beam.vendor.calcite.v1_28_0.org.codehaus.janino.CompilerFactory +compilerFactory=org.apache.beam.vendor.calcite.v1_40_0.org.codehaus.janino.CompilerFactory diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java index 1530141c6e22..0d93792bcad2 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java @@ -20,7 +20,7 @@ import static org.junit.Assert.assertEquals; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.CalciteContextException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.CalciteContextException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.junit.Before; import org.junit.Rule; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java index b8aa030649ab..b8e6e90d680c 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java @@ -43,7 +43,7 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.CalciteContextException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.CalciteContextException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java index cfa2df719679..90c8008c6d0a 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java @@ -44,10 +44,10 @@ import org.apache.beam.sdk.extensions.sql.integrationtest.BeamSqlBuiltinFunctionsIntegrationTestBase; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.SqlFunctions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.SqlFunctions; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -59,13 +59,14 @@ /** * DSL compliance tests for the row-level operators of {@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable}. + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.fun.SqlStdOperatorTable}. */ public class BeamSqlDslSqlStdOperatorsTest extends BeamSqlBuiltinFunctionsIntegrationTestBase { private static final BigDecimal ZERO_0 = BigDecimal.valueOf(0).setScale(0, UNNECESSARY); private static final BigDecimal ZERO_1 = BigDecimal.valueOf(0).setScale(1, UNNECESSARY); private static final BigDecimal ONE_0 = BigDecimal.valueOf(1).setScale(0, UNNECESSARY); private static final BigDecimal ONE_1 = BigDecimal.valueOf(1).setScale(1, UNNECESSARY); + private static final BigDecimal ONE_2 = BigDecimal.valueOf(1).setScale(2, UNNECESSARY); private static final BigDecimal TWO_0 = BigDecimal.valueOf(2).setScale(0, UNNECESSARY); private static final BigDecimal TWO_1 = BigDecimal.valueOf(2).setScale(1, UNNECESSARY); @@ -311,7 +312,7 @@ public void testArithmeticOperator() { .addExpr("c_double + c_bigint", 2.0) .addExpr("1 - 1", 0) .addExpr("1.0 - 1", ZERO_1) - .addExpr("1 - 0.0", ONE_0) + .addExpr("1 - 0.0", ONE_1) .addExpr("1.0 - 1.0", ZERO_1) .addExpr("c_tinyint - c_tinyint", (byte) 0) .addExpr("c_smallint - c_smallint", (short) 0) @@ -326,9 +327,9 @@ public void testArithmeticOperator() { .addExpr("c_float - c_bigint", 0.0f) .addExpr("c_double - c_bigint", 0.0) .addExpr("1 * 1", 1) - .addExpr("1.0 * 1", ONE_0) + .addExpr("1.0 * 1", ONE_1) .addExpr("1 * 1.0", ONE_1) - .addExpr("1.0 * 1.0", ONE_1) + .addExpr("1.0 * 1.0", ONE_2) .addExpr("c_tinyint * c_tinyint", (byte) 1) .addExpr("c_smallint * c_smallint", (short) 1) .addExpr("c_bigint * c_bigint", 1L) @@ -366,11 +367,11 @@ public void testArithmeticOperator() { .addExpr("mod(c_bigint, c_bigint)", 0L) .addExpr("mod(c_decimal, c_decimal)", ZERO_0) .addExpr("mod(c_tinyint, c_decimal)", ZERO_0) - // Test overflow - .addExpr("c_tinyint_max + c_tinyint_max", (byte) -2) - .addExpr("c_smallint_max + c_smallint_max", (short) -2) - .addExpr("c_integer_max + c_integer_max", -2) - .addExpr("c_bigint_max + c_bigint_max", -2L); + // conversions + .addExpr("c_tinyint_max + c_smallint", (short) 128) + .addExpr("c_integer + c_smallint_max", 32768) + .addExpr("c_integer_max + c_bigint", 2147483648L) + .addExpr("c_smallint - c_integer_max", -2147483646); checker.buildRunAndCheck(); } @@ -708,8 +709,8 @@ public void testAggrationFunctions() { @Test @SqlOperatorTests({ - @SqlOperatorTest(name = "CHARACTER_LENGTH", kind = "OTHER_FUNCTION"), - @SqlOperatorTest(name = "CHAR_LENGTH", kind = "OTHER_FUNCTION"), + @SqlOperatorTest(name = "CHARACTER_LENGTH", kind = "CHAR_LENGTH"), + @SqlOperatorTest(name = "CHAR_LENGTH", kind = "CHAR_LENGTH"), @SqlOperatorTest(name = "INITCAP", kind = "OTHER_FUNCTION"), @SqlOperatorTest(name = "LOWER", kind = "OTHER_FUNCTION"), @SqlOperatorTest(name = "POSITION", kind = "POSITION"), @@ -1129,8 +1130,7 @@ public void testBasicDateTimeFunctions() { } @Test - // https://github.com/apache/beam/issues/19001 - // @SqlOperatorTest(name = "FLOOR", kind = "FLOOR") + @SqlOperatorTest(name = "FLOOR", kind = "FLOOR") public void testFloor() { ExpressionChecker checker = new ExpressionChecker() @@ -1140,14 +1140,14 @@ public void testFloor() { .addExpr("FLOOR(ts TO DAY)", parseTimestampWithUTCTimeZone("1986-02-15 00:00:00")) .addExpr("FLOOR(ts TO MONTH)", parseTimestampWithUTCTimeZone("1986-02-01 00:00:00")) .addExpr("FLOOR(ts TO YEAR)", parseTimestampWithUTCTimeZone("1986-01-01 00:00:00")) - .addExpr("FLOOR(c_double)", 1.0); + .addExpr("FLOOR(c_double)", 1.0) + .addExpr("FLOOR(-c_double)", -2.0); checker.buildRunAndCheck(getFloorCeilingTestPCollection()); } @Test - // https://github.com/apache/beam/issues/19001 - // @SqlOperatorTest(name = "CEIL", kind = "CEIL") + @SqlOperatorTest(name = "CEIL", kind = "CEIL") public void testCeil() { ExpressionChecker checker = new ExpressionChecker() @@ -1157,7 +1157,8 @@ public void testCeil() { .addExpr("CEIL(ts TO DAY)", parseTimestampWithUTCTimeZone("1986-02-16 00:00:00")) .addExpr("CEIL(ts TO MONTH)", parseTimestampWithUTCTimeZone("1986-03-01 00:00:00")) .addExpr("CEIL(ts TO YEAR)", parseTimestampWithUTCTimeZone("1987-01-01 00:00:00")) - .addExpr("CEIL(c_double)", 2.0); + .addExpr("CEIL(c_double)", 2.0) + .addExpr("CEIL(-c_double)", -1.0); checker.buildRunAndCheck(getFloorCeilingTestPCollection()); } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java index 991d4d260fd9..41288dd21e36 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java @@ -48,8 +48,8 @@ import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.function.Parameter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.TranslatableTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.function.Parameter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.TranslatableTable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.Instant; import org.junit.Test; @@ -302,7 +302,7 @@ public void testBeamSqlUdfWithDefaultParameters() throws Exception { } /** - * test {@link org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.TableMacro} UDF. + * test {@link org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.TableMacro} UDF. */ @Test public void testTableMacroUdf() throws Exception { @@ -503,7 +503,7 @@ public static Integer eval(java.util.List i) { /** * UDF to test support for {@link - * org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.TableMacro}. + * org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.TableMacro}. */ public static final class RangeUdf implements BeamSqlUdf { public static TranslatableTable eval(int startInclusive, int endExclusive) { diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java index 06e30936b4c3..fa891914db1b 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java @@ -21,9 +21,9 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParseException; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelConversionException; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.ValidationException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParseException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RelConversionException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.ValidationException; import org.junit.Before; import org.junit.Ignore; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TypedCombineFnDelegateTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TypedCombineFnDelegateTest.java index 70938ff9d692..5dc194d48167 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TypedCombineFnDelegateTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TypedCombineFnDelegateTest.java @@ -25,11 +25,11 @@ import org.apache.beam.sdk.extensions.sql.impl.UdafImpl; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Max; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.FunctionParameter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.FunctionParameter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java index e83ee61af4ab..b9aa4ae2ecc7 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java @@ -50,9 +50,9 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteConnection; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteConnection; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaPlus; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.DateTime; import org.joda.time.Duration; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/LazyAggregateCombineFnTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/LazyAggregateCombineFnTest.java index fda230756c24..17636c628eb8 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/LazyAggregateCombineFnTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/LazyAggregateCombineFnTest.java @@ -28,12 +28,12 @@ import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.extensions.sql.udf.AggregateFn; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.AggregateFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.FunctionParameter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.AggregateFunction; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.FunctionParameter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Rule; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java index 6876caff3274..e9daf57816bf 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.extensions.sql.utils.QuickCheckGenerators.PrimitiveTypes; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParseException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParseException; import org.junit.runner.RunWith; /** diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java index 704a9d4586e1..518a830041e2 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java @@ -35,12 +35,12 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.pretty.SqlPrettyWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.pretty.SqlPrettyWriter; import org.junit.Test; /** UnitTest for {@link BeamSqlParserImpl}. */ diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java index 17611fa12f19..95bf8e1ea9d6 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java @@ -21,11 +21,11 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.volcano.RelSubset; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.SingleRel; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.volcano.RelSubset; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.SingleRel; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -62,7 +62,7 @@ public static void prepare() { public void testUnknownRel() { String sql = " select * from ORDER_DETAILS1 "; RelNode root = env.parseQuery(sql); - RelNode unknown = new UnknownRel(root.getCluster(), null, null); + RelNode unknown = new UnknownRel(root.getCluster(), RelTraitSet.createEmpty(), null); NodeStats nodeStats = unknown .metadata(NodeStatsMetadata.class, unknown.getCluster().getMetadataQuery()) diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRelTest.java index d227113cd08b..3692c2514c62 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRelTest.java @@ -24,7 +24,7 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable; import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestUnboundedTable; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.joda.time.DateTime; import org.joda.time.Duration; import org.junit.Assert; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRelTest.java index 5d53ca364075..8aeb77fc0490 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRelTest.java @@ -33,7 +33,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.DateTime; import org.joda.time.Duration; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java index cbaf56b09d41..9a0977e9018c 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.hamcrest.core.StringContains; import org.junit.Assert; import org.junit.BeforeClass; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java index efe8516240ad..0caa08d0a634 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.joda.time.DateTime; import org.joda.time.Duration; import org.junit.Assert; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java index ffb9b6064c5d..99915b33bf8b 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java @@ -38,19 +38,20 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.java.JavaTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.Enumerable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.Enumerator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.volcano.VolcanoPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.prepare.RelOptTableImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.TableModify.Operation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.Enumerable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.Enumerator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptCluster; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.volcano.VolcanoPlanner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.prepare.RelOptTableImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.TableModify.Operation; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexBuilder; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexLiteral; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -74,7 +75,7 @@ public void testToEnumerable_collectSingle() { RelDataType type = CalciteUtils.toCalciteRowType(schema, TYPE_FACTORY); ImmutableList> tuples = ImmutableList.of(ImmutableList.of(rexBuilder.makeBigintLiteral(BigDecimal.ZERO))); - BeamRelNode node = new BeamValuesRel(cluster, type, tuples, null); + BeamRelNode node = new BeamValuesRel(cluster, type, tuples, RelTraitSet.createEmpty()); Enumerable enumerable = BeamEnumerableConverter.toEnumerable(options, node); Enumerator enumerator = enumerable.enumerator(); @@ -94,7 +95,7 @@ public void testToEnumerable_collectMultiple() { ImmutableList.of( rexBuilder.makeBigintLiteral(BigDecimal.ZERO), rexBuilder.makeBigintLiteral(BigDecimal.ONE))); - BeamRelNode node = new BeamValuesRel(cluster, type, tuples, null); + BeamRelNode node = new BeamValuesRel(cluster, type, tuples, RelTraitSet.createEmpty()); Enumerable enumerable = BeamEnumerableConverter.toEnumerable(options, node); Enumerator enumerator = enumerable.enumerator(); @@ -117,7 +118,7 @@ public void testToListRow_collectMultiple() { ImmutableList.of( rexBuilder.makeBigintLiteral(BigDecimal.ZERO), rexBuilder.makeBigintLiteral(BigDecimal.ONE))); - BeamRelNode node = new BeamValuesRel(cluster, type, tuples, null); + BeamRelNode node = new BeamValuesRel(cluster, type, tuples, RelTraitSet.createEmpty()); List rowList = BeamEnumerableConverter.toRowList(options, node); assertTrue(rowList.size() == 1); @@ -164,7 +165,7 @@ public void testToEnumerable_count() { cluster, RelOptTableImpl.create(null, type, ImmutableList.of(), null), null, - new BeamValuesRel(cluster, type, tuples, null), + new BeamValuesRel(cluster, type, tuples, RelTraitSet.createEmpty()), Operation.INSERT, null, null, @@ -222,7 +223,7 @@ public void testToEnumerable_collectNullValue() { ImmutableList.of( ImmutableList.of( rexBuilder.makeNullLiteral(CalciteUtils.toRelDataType(TYPE_FACTORY, fieldType)))); - BeamRelNode node = new BeamValuesRel(cluster, type, tuples, null); + BeamRelNode node = new BeamValuesRel(cluster, type, tuples, RelTraitSet.createEmpty()); Enumerable enumerable = BeamEnumerableConverter.toEnumerable(options, node); Enumerator enumerator = enumerable.enumerator(); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRelTest.java index 174c5fbe1d41..beb6db955229 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRelTest.java @@ -25,8 +25,8 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestUnboundedTable; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.metadata.RelMetadataQuery; import org.joda.time.DateTime; import org.joda.time.Duration; import org.junit.Assert; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java index 3baf1e471480..0dc39ccfa173 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Rule; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java index 21d01df18827..3c1c40ad4532 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.joda.time.DateTime; import org.joda.time.Duration; import org.junit.Assert; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java index c5f8f28bfa6d..581cf6c005c2 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.joda.time.DateTime; import org.joda.time.Duration; import org.junit.Assert; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java index e2fa12350452..dbe8be441ac6 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java @@ -26,7 +26,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Before; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRelTest.java index 98791ead1228..2feaad7bf06a 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRelTest.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java index af81a5b97de5..ed2b5df68358 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Rule; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java index 1ae169f5d062..bc21cd443cde 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java @@ -26,7 +26,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Rule; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java index 32f59ddad79b..e9b0a995297e 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java @@ -36,13 +36,13 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CoreRules; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSets; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Calc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.CoreRules; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSets; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java index 77de4cdec0f9..92b77ec9efbd 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java @@ -30,41 +30,41 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.DataContext; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.EnumerableConvention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.EnumerableRules; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.Enumerable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.Linq4j; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.ConventionTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollationTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollations; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelFieldCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelRoot; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Join; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.TableScan; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CoreRules; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.JoinCommuteRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.ScannableTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Statistic; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Statistics; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Table; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.impl.AbstractSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.impl.AbstractTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParser; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.FrameworkConfig; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.Frameworks; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.Planner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.Programs; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSets; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.ImmutableBitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.DataContext; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.EnumerableConvention; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.adapter.enumerable.EnumerableRules; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.Enumerable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.Linq4j; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.ConventionTraitDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelTraitSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelCollationTraitDef; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelCollations; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelFieldCollation; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.RelRoot; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Join; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.TableScan; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.CoreRules; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.JoinCommuteRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.ScannableTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Statistic; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Statistics; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Table; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.impl.AbstractSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.impl.AbstractTable; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParser; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.FrameworkConfig; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.Frameworks; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.Planner; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.Programs; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSet; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSets; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.ImmutableBitSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Assert; @@ -415,7 +415,7 @@ public ThreeTablesSchema() { } @Override - protected Map + protected Map getTableMap() { return tables; } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamSqlRowCoderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamSqlRowCoderTest.java index cc4e8c6b0b5e..41cd9ed2d7ef 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamSqlRowCoderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamSqlRowCoderTest.java @@ -27,10 +27,10 @@ import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; import org.joda.time.DateTime; import org.junit.Test; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java index e51a1b2dc6c8..6c85c3582e95 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java @@ -24,11 +24,11 @@ import java.util.Map; import java.util.stream.Collectors; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeFactoryImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeFactoryImpl; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java index d612c9d19145..559aeb04cfb6 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.extensions.sql.integrationtest; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_SECOND; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_SECOND; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java index 7f9d52f02906..b19026af72c6 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rex.RexNode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.junit.Rule; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datastore/DataStoreReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datastore/DataStoreReadWriteIT.java index 1937c6ea9efd..ea9cf4a02e54 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datastore/DataStoreReadWriteIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datastore/DataStoreReadWriteIT.java @@ -50,7 +50,7 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.ByteString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.avatica.util.ByteString; import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java index cc6e3b426ec3..0c51b31f1927 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java @@ -28,7 +28,7 @@ import java.util.UUID; import org.apache.beam.sdk.extensions.sql.BeamSqlCli; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.CalciteContextException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.CalciteContextException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.junit.Assert; import org.junit.Before; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java index f14344b4f1fe..c78965d214b2 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.commons.lang3.tuple.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.commons.lang3.tuple.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Before; import org.junit.Rule; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java index 15fe4769c61b..a223194b8e91 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -218,9 +218,9 @@ public void runSqlWriteAndRead(boolean withPartitionFields) + "'char', " + "ARRAY['123', '456'], " + "ARRAY[" - + "ROW(ARRAY['abc', 'xyz'], 123), " - + "ROW(ARRAY['foo', 'bar'], 456), " - + "ROW(ARRAY['cat', 'dog'], 789)]" + + "CAST(ROW(ARRAY['abc', 'xyz'], 123) AS ROW(c_arr_struct_arr VARCHAR ARRAY, c_arr_struct_integer INTEGER)), " + + "CAST(ROW(ARRAY['foo', 'bar'], 456) AS ROW(c_arr_struct_arr VARCHAR ARRAY, c_arr_struct_integer INTEGER)), " + + "CAST(ROW(ARRAY['cat', 'dog'], 789) AS ROW(c_arr_struct_arr VARCHAR ARRAY, c_arr_struct_integer INTEGER))]" + ")"; BeamSqlRelUtils.toPCollection(writePipeline, sqlEnv.parseQuery(insertStatement)); writePipeline.run().waitUntilFinish(); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java index 3e63eb8457e0..cf066b1abed8 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.beam.vendor.calcite.v1_40_0.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Test; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java index 6153e6e33c2c..b3684d88d220 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java @@ -71,7 +71,7 @@ import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteConnection; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteConnection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java index ce9f52ffea94..c3a07fafa584 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java @@ -41,9 +41,9 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CoreRules; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSets; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.CoreRules; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.junit.Before; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java index eae1dfb49ee8..823aa8998e98 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java @@ -42,10 +42,10 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CoreRules; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSets; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.core.Calc; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.CoreRules; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.collection.IsIterableContainingInAnyOrder; import org.joda.time.Duration; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java index 4fa45cbab9ec..bd65e8353b06 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java @@ -41,9 +41,9 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CoreRules; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSets; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.rules.CoreRules; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.junit.Before; diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index e4e7e2f1095b..ba1d27b0e3e6 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -64,7 +64,7 @@ dependencies { // TODO(https://github.com/apache/beam/issues/21156): Determine how to build without this dependency provided "org.immutables:value:2.8.8" permitUnusedDeclared "org.immutables:value:2.8.8" - implementation library.java.vendored_calcite_1_28_0 + implementation library.java.vendored_calcite_1_40_0 runtimeOnly "org.apache.iceberg:iceberg-gcp:$iceberg_version" runtimeOnly library.java.bigdataoss_gcs_connector runtimeOnly library.java.hadoop_client diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java index 36cf584dfd41..fd008701c548 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java @@ -38,19 +38,19 @@ import java.util.function.BiFunction; import java.util.stream.Collectors; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlBasicCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNodeList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParseException; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParser; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.DateString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.TimeString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.TimestampString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlBasicCall; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlLiteral; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNodeList; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlOperator; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParseException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParser; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.DateString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.TimeString; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.TimestampString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.iceberg.Schema; @@ -153,7 +153,17 @@ static Expression convert(@Nullable String filter, Schema schema) { } private static Expression convert(SqlNode expression, Schema schema) throws SqlParseException { - checkArgument(expression instanceof SqlBasicCall); + if (expression instanceof SqlIdentifier) { + String fieldName = ((SqlIdentifier) expression).getSimple(); + Types.NestedField field = schema.caseInsensitiveFindField(fieldName); + if (field.type().equals(Types.BooleanType.get())) { + return Expressions.equal(field.name(), true); + } + } + checkArgument( + expression instanceof SqlBasicCall, + String.format( + "Expected SqlBasicCall, got %s: %s", expression.getClass().getName(), expression)); SqlBasicCall call = (SqlBasicCall) expression; SqlOperator op = call.getOperator(); @@ -340,7 +350,7 @@ private static Object convertLiteral(SqlLiteral literal, String field, TypeID ty return literal.getValueAs(String.class); case DATE: LocalDate date; - if (SqlTypeName.STRING_TYPES.contains(typeName)) { + if (SqlTypeName.STRING_TYPES.contains(typeName) || SqlTypeName.UNKNOWN.equals(typeName)) { date = LocalDate.parse(literal.getValueAs(String.class)); } else if (SqlTypeName.DATE.equals(typeName)) { DateString dateValue = literal.getValueAs(DateString.class); @@ -351,7 +361,7 @@ private static Object convertLiteral(SqlLiteral literal, String field, TypeID ty return DateTimeUtil.daysFromDate(date); case TIME: LocalTime time; - if (SqlTypeName.STRING_TYPES.contains(typeName)) { + if (SqlTypeName.STRING_TYPES.contains(typeName) || SqlTypeName.UNKNOWN.equals(typeName)) { time = LocalTime.parse(literal.getValueAs(String.class)); } else if (SqlTypeName.TIME.equals(typeName)) { TimeString timeString = literal.getValueAs(TimeString.class); @@ -362,7 +372,7 @@ private static Object convertLiteral(SqlLiteral literal, String field, TypeID ty return DateTimeUtil.microsFromTime(time); case TIMESTAMP: LocalDateTime datetime; - if (SqlTypeName.STRING_TYPES.contains(typeName)) { + if (SqlTypeName.STRING_TYPES.contains(typeName) || SqlTypeName.UNKNOWN.equals(typeName)) { String value = literal.getValueAs(String.class); datetime = getLocalDateTime(value); } else if (SqlTypeName.DATE.equals(typeName)) { diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FilterUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FilterUtilsTest.java index f2914c708a9c..893e24b61559 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FilterUtilsTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FilterUtilsTest.java @@ -47,7 +47,7 @@ import java.util.Set; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.commons.lang3.tuple.Pair; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.commons.lang3.tuple.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoin.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoin.java index 9806a265df4e..171bf9c69fed 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoin.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoin.java @@ -68,13 +68,13 @@ public static SqlBoundedSideInputJoin calciteSqlBoundedSideInputJoin( return new SqlBoundedSideInputJoin( configuration, CalciteQueryPlanner.class, - "WITH bid_with_side (auction, bidder, price, dateTime, extra, side_id) AS (%n" + "WITH bid_with_side (auction, bidder, price, `dateTime`, extra, side_id) AS (%n" + " SELECT *, CAST(MOD(bidder, %d) AS BIGINT) side_id FROM bid%n" + ")%n" + " SELECT bid_with_side.auction%n" + ", bid_with_side.bidder%n" + ", bid_with_side.price%n" - + ", bid_with_side.dateTime%n" + + ", bid_with_side.`dateTime`%n" + ", side.extra%n" + " FROM bid_with_side, side%n" + " WHERE bid_with_side.side_id = side.id"); diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1.java index 80501c1bb9ee..6d0ca9f726ea 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1.java @@ -35,7 +35,7 @@ * Query 1, 'Currency Conversion'. Convert each bid value from dollars to euros. In CQL syntax: * *
- * SELECT Istream(auction, DOLTOEUR(price), bidder, datetime)
+ * SELECT Istream(auction, DOLTOEUR(price), bidder, `datetime`)
  * FROM bid [ROWS UNBOUNDED];
  * 
* @@ -46,7 +46,7 @@ public class SqlQuery1 extends NexmarkQueryTransform { private static final PTransform> QUERY = SqlTransform.query( - "SELECT auction, bidder, DolToEur(price) as price, dateTime, extra FROM PCOLLECTION") + "SELECT auction, bidder, DolToEur(price) as price, `dateTime`, extra FROM PCOLLECTION") .registerUdf("DolToEur", new DolToEur()); /** Dollar to Euro conversion. */ diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery7.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery7.java index 00f256fa1d70..4d3f2120f6d3 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery7.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery7.java @@ -55,17 +55,16 @@ public SqlQuery7(NexmarkConfiguration configuration) { String queryString = String.format( - "" - + " SELECT B.auction, B.price, B.bidder, B.dateTime, B.extra " - + " FROM (SELECT B.auction, B.price, B.bidder, B.dateTime, B.extra, " - + " TUMBLE_START(B.dateTime, INTERVAL '%1$d' SECOND) AS starttime " + " SELECT B.auction, B.price, B.bidder, B.`dateTime`, B.extra " + + " FROM (SELECT B.auction, B.price, B.bidder, B.`dateTime`, B.extra, " + + " TUMBLE_START(B.`dateTime`, INTERVAL '%1$d' SECOND) AS starttime " + " FROM Bid B " - + " GROUP BY B.auction, B.price, B.bidder, B.dateTime, B.extra, " - + " TUMBLE(B.dateTime, INTERVAL '%1$d' SECOND)) B " + + " GROUP BY B.auction, B.price, B.bidder, B.`dateTime`, B.extra, " + + " TUMBLE(B.`dateTime`, INTERVAL '%1$d' SECOND)) B " + " JOIN (SELECT MAX(B1.price) AS maxprice, " - + " TUMBLE_START(B1.dateTime, INTERVAL '%1$d' SECOND) AS starttime " + + " TUMBLE_START(B1.`dateTime`, INTERVAL '%1$d' SECOND) AS starttime " + " FROM Bid B1 " - + " GROUP BY TUMBLE(B1.dateTime, INTERVAL '%1$d' SECOND)) B1 " + + " GROUP BY TUMBLE(B1.`dateTime`, INTERVAL '%1$d' SECOND)) B1 " + " ON B.starttime = B1.starttime AND B.price = B1.maxprice ", configuration.windowSizeSec); query = SqlTransform.query(queryString); diff --git a/sdks/java/testing/tpcds/build.gradle b/sdks/java/testing/tpcds/build.gradle index d59895421777..1714e6e61a80 100644 --- a/sdks/java/testing/tpcds/build.gradle +++ b/sdks/java/testing/tpcds/build.gradle @@ -62,10 +62,9 @@ dependencies { implementation library.java.avro implementation library.java.joda_time implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.vendored_calcite_1_28_0 + implementation library.java.vendored_calcite_1_40_0 implementation library.java.commons_csv implementation library.java.slf4j_api - implementation "com.googlecode.json-simple:json-simple:1.1.1" implementation library.java.jackson_databind implementation project(":sdks:java:extensions:sql") implementation project(":sdks:java:io:parquet") @@ -79,6 +78,7 @@ dependencies { testRuntimeOnly library.java.slf4j_jdk14 testImplementation project(path: ":sdks:java:io:google-cloud-platform") testImplementation project(path: ":sdks:java:testing:test-utils") + testImplementation library.java.junit gradleRun project(project.path) gradleRun project(path: tpcdsRunnerDependency, configuration: runnerConfiguration) } diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java index 8071bad84d73..f6bb0d500920 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java @@ -19,9 +19,9 @@ import java.nio.charset.StandardCharsets; import java.util.Set; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParseException; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParser; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParseException; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParser; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Resources; /** diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java index 6efb7e7e0659..a8f479557005 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java @@ -55,8 +55,8 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.util.SqlBasicVisitor; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.util.SqlBasicVisitor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Resources; import org.apache.commons.csv.CSVFormat; diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java index 97116e14cdcd..0d683c6b66da 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java @@ -19,6 +19,10 @@ import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.node.TextNode; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -27,9 +31,6 @@ import java.util.Map; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Resources; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.reflect.ClassPath; -import org.json.simple.JSONArray; -import org.json.simple.JSONObject; -import org.json.simple.parser.JSONParser; /** * TableSchemaJSONLoader can get all table's names from resource/schemas directory and parse a @@ -51,8 +52,9 @@ public static String parseTableSchema(String tableName) throws Exception { String path = "schemas/" + tableName + ".json"; String schema = Resources.toString(Resources.getResource(path), StandardCharsets.UTF_8); - JSONObject jsonObject = (JSONObject) new JSONParser().parse(schema); - JSONArray jsonArray = (JSONArray) jsonObject.get("schema"); + ObjectMapper mapper = new ObjectMapper(); + + ArrayNode jsonArray = (ArrayNode) mapper.readTree(schema).get("schema"); if (jsonArray == null) { throw new RuntimeException("Can't get Json array for \"schema\" key."); } @@ -62,19 +64,20 @@ public static String parseTableSchema(String tableName) throws Exception { Iterator jsonArrIterator = jsonArray.iterator(); while (jsonArrIterator.hasNext()) { - Map jsonMap = checkArgumentNotNull((Map) jsonArrIterator.next()); - Iterator recordIterator = jsonMap.entrySet().iterator(); + ObjectNode jsonMap = checkArgumentNotNull((ObjectNode) jsonArrIterator.next()); + Iterator recordIterator = jsonMap.properties().iterator(); while (recordIterator.hasNext()) { Map.Entry pair = checkArgumentNotNull((Map.Entry) recordIterator.next()); Object key = checkArgumentNotNull(pair.getKey()); + TextNode valueNode = (TextNode) checkArgumentNotNull(pair.getValue()); + String value = valueNode.textValue(); if (key.equals("type")) { // If the key of the pair is "type", make some modification before appending it to the // schemaStringBuilder, then append a comma. - String typeName = checkArgumentNotNull((String) pair.getValue()); - if (typeName.equalsIgnoreCase("identifier") || typeName.equalsIgnoreCase("integer")) { + if (value.equalsIgnoreCase("identifier") || value.equalsIgnoreCase("integer")) { // Use long type to represent int, prevent overflow schemaStringBuilder.append("bigint"); - } else if (typeName.contains("decimal")) { + } else if (value.contains("decimal")) { // Currently Beam SQL doesn't handle "decimal" type properly, use "double" to replace it // for now. schemaStringBuilder.append("double"); @@ -87,7 +90,7 @@ public static String parseTableSchema(String tableName) throws Exception { } else { // If the key of the pair is "name", directly append it to the StringBuilder, then append // a space. - schemaStringBuilder.append(pair.getValue()); + schemaStringBuilder.append(value); schemaStringBuilder.append(' '); } } diff --git a/settings.gradle.kts b/settings.gradle.kts index b2ff57701ba4..1879e0b151eb 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -324,7 +324,6 @@ include(":sdks:python:test-suites:xlang") include(":sdks:typescript") include(":sdks:typescript:container") include(":vendor:grpc-1_69_0") -include(":vendor:calcite-1_28_0") include(":vendor:calcite-1_40_0") include(":vendor:guava-32_1_2-jre") include(":website") diff --git a/vendor/calcite-1_28_0/build.gradle b/vendor/calcite-1_28_0/build.gradle deleted file mode 100644 index 576ae44751da..000000000000 --- a/vendor/calcite-1_28_0/build.gradle +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * License); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an AS IS BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - * Vendored version of calcite. - * - * To upgrade: - * 1. Use mvn dependency:tree and/or https://search.maven.org/search?q=g:org.apache.calcite%20AND%20a:calcite-core - * to determine dependency tree. You may need to search for optional transitive dependencies - * and determine if they need to be added or upgraded (e.g. protobuf) - * 2. Validate built artifacts by running linkage tool - * (https://github.com/apache/beam/tree/master/vendor#how-to-validate-the-vendored-dependencies) - * and unit and integration tests in a PR. - */ - -plugins { id 'org.apache.beam.vendor-java' } - -description = "Apache Beam :: Vendored Dependencies :: Calcite 1.28.0" - -group = "org.apache.beam" -version = "0.2" - -def calcite_version = "1.28.0" -def avatica_version = "1.19.0" -def protobuf_version = "3.19.2" -def prefix = "org.apache.beam.vendor.calcite.v1_28_0" - -List packagesToRelocate = [ - "com.esri", - "com.fasterxml", - "com.google.common", - "com.google.gson", - "com.google.protobuf", - "com.google.thirdparty", - "com.google.uzaygezen", - "com.jayway", - "com.yahoo", - "net.minidev", - "org.apache.calcite", - "org.apache.commons", - "org.apache.http", - "org.apiguardian.api", - "org.codehaus", - "org.objectweb", - "org.pentaho", - "org.yaml", -] - -vendorJava( - dependencies: [ - "org.apache.calcite:calcite-core:$calcite_version", - "org.apache.calcite:calcite-linq4j:$calcite_version", - "org.apache.calcite.avatica:avatica-core:$avatica_version", - - // BEAM-13616: Override the version of protobuf to patch a security vulnerability. - // This override can be removed once we upgrade to a newer version of calcite that - // depends on protobuf >= 3.19.2. - "com.google.protobuf:protobuf-java:$protobuf_version", - "com.google.protobuf:protobuf-java-util:$protobuf_version", - ], - runtimeDependencies: [ - library.java.slf4j_api, - ], - relocations: packagesToRelocate.collectEntries { - [ (it): "${prefix}.${it}" ] + [ "jdbc:calcite:": "jdbc:beam-vendor-calcite:"] - }, - exclusions: [ - // Code quality / Building annotations - "com/google/errorprone/**", - "com/google/j2objc/annotations/**", - "javax/annotation/**", - "org/checkerframework/**", - "org/jmlspecs/**", - "org/intellij/lang/annotations/**", - "org/jetbrains/annotations/**", - - // Runtime logging interface - "org/slf4j/**", - "org/apache/log4j/**", - "org/apache/logging/log4j/**", - "META-INF/versions/9/org/apache/logging/log4j/**", - - // Optional loggers - "org/apache/commons/logging/impl/AvalonLogger*", - "org/apache/commons/logging/impl/LogKitLogger*", - - // Optional JSON providers - "com/jayway/jsonpath/spi/json/GsonJsonProvider*", - "com/jayway/jsonpath/spi/json/JettisonProvider*", - "com/jayway/jsonpath/spi/json/JsonOrgJsonProvider*", - "com/jayway/jsonpath/spi/json/TapestryJsonProvider*", - "com/jayway/jsonpath/spi/mapper/GsonMappingProvider*", - "com/jayway/jsonpath/spi/mapper/JsonOrgMappingProvider*", - "com/jayway/jsonpath/spi/mapper/TapestryMappingProvider*", - - // Unused broken code - "org/apache/commons/dbcp2/managed/**", - "org/apache/commons/pool2/proxy/**", - "org/codehaus/janino/AntCompilerAdapter*", - - // Optional kotlin code - "kotlin/**", - - "**/module-info.class", - ], - groupId: group, - artifactId: "beam-vendor-calcite-1_28_0", - version: version, -)