From 3d43232743450e95f7c3641455ba805049068b84 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 14 Oct 2025 10:09:47 -0400 Subject: [PATCH 01/14] add SHOW command --- .../trigger_files/beam_PostCommit_SQL.json | 2 +- .github/trigger_files/beam_PreCommit_SQL.json | 2 +- .../meta/provider/iceberg/IcebergCatalog.java | 7 + .../sql/src/main/codegen/config.fmpp | 10 ++ .../src/main/codegen/includes/parserImpls.ftl | 38 ++++++ .../sql/impl/BeamCalciteSchema.java | 4 + .../sql/impl/CatalogManagerSchema.java | 6 +- .../extensions/sql/impl/CatalogSchema.java | 4 + .../sql/impl/parser/SqlDdlNodes.java | 7 + .../sql/impl/parser/SqlShowCatalogs.java | 113 ++++++++++++++++ .../sql/impl/parser/SqlShowDatabases.java | 105 +++++++++++++++ .../sql/impl/parser/SqlShowTables.java | 123 ++++++++++++++++++ .../extensions/sql/meta/catalog/Catalog.java | 4 + .../sql/meta/catalog/InMemoryCatalog.java | 6 + .../extensions/sql/BeamSqlCliCatalogTest.java | 23 ++++ .../sql/BeamSqlCliDatabaseTest.java | 28 ++++ .../sdk/extensions/sql/BeamSqlCliTest.java | 40 ++++++ 17 files changed, 519 insertions(+), 3 deletions(-) create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java diff --git a/.github/trigger_files/beam_PostCommit_SQL.json b/.github/trigger_files/beam_PostCommit_SQL.json index 6cc79a7a0325..833fd9b0d174 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": 1 + "modification": 2 } diff --git a/.github/trigger_files/beam_PreCommit_SQL.json b/.github/trigger_files/beam_PreCommit_SQL.json index 5abe02fc09c7..ab4daeae2349 100644 --- a/.github/trigger_files/beam_PreCommit_SQL.json +++ b/.github/trigger_files/beam_PreCommit_SQL.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 1 + "modification": 3 } diff --git a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java index 0ca38824204b..bb307207a327 100644 --- a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java +++ b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java @@ -19,12 +19,14 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import java.util.Collection; import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalog; import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; 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.ImmutableMap; +import org.checkerframework.checker.nullness.qual.Nullable; public class IcebergCatalog extends InMemoryCatalog { // TODO(ahmedabu98): extend this to the IO implementation so @@ -71,6 +73,11 @@ public boolean createDatabase(String database) { return catalogConfig.createNamespace(database); } + @Override + public @Nullable Collection databases() { + return catalogConfig.listNamespaces(); + } + @Override public void useDatabase(String database) { checkArgument(databaseExists(database), "Database '%s' does not exist."); diff --git a/sdks/java/extensions/sql/src/main/codegen/config.fmpp b/sdks/java/extensions/sql/src/main/codegen/config.fmpp index 77772c5858e3..cc469f97266c 100644 --- a/sdks/java/extensions/sql/src/main/codegen/config.fmpp +++ b/sdks/java/extensions/sql/src/main/codegen/config.fmpp @@ -35,6 +35,9 @@ data: { "org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes" "org.apache.beam.sdk.extensions.sql.impl.parser.SqlUseCatalog" "org.apache.beam.sdk.extensions.sql.impl.parser.SqlUseDatabase" + "org.apache.beam.sdk.extensions.sql.impl.parser.SqlShowCatalogs" + "org.apache.beam.sdk.extensions.sql.impl.parser.SqlShowDatabases" + "org.apache.beam.sdk.extensions.sql.impl.parser.SqlShowTables" "org.apache.beam.sdk.extensions.sql.impl.parser.SqlSetOptionBeam" "org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils" "org.apache.beam.sdk.schemas.Schema" @@ -50,6 +53,9 @@ data: { "TBLPROPERTIES" "PROPERTIES" "PARTITIONED" + "CATALOGS" + "DATABASES" + "TABLES" "USE" ] @@ -397,6 +403,7 @@ data: { "IF" "JAR" "LOCATION" + "SHOW" "TBLPROPERTIES" "PROPERTIES" "PARTITIONED" @@ -422,6 +429,9 @@ data: { # Return type of method implementation should be 'SqlNode'. # Example: SqlShowDatabases(), SqlShowTables(). statementParserMethods: [ + "SqlShowCatalogs(Span.of(), null)" + "SqlShowDatabases(Span.of(), null)" + "SqlShowTables(Span.of(), null)" "SqlUseCatalog(Span.of(), null)" "SqlUseDatabase(Span.of(), null)" "SqlSetOptionBeam(Span.of(), null)" diff --git a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl index 46102c7b92fe..e7ef7963aec6 100644 --- a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl +++ b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl @@ -264,6 +264,19 @@ SqlDrop SqlDropCatalog(Span s, boolean replace) : } } + +SqlCall SqlShowCatalogs(Span s, String scope) : +{ +} +{ + + + { + return new SqlShowCatalogs(s.end(this), scope); + } +} + + /** * CREATE DATABASE ( IF NOT EXISTS )? ( catalog_name '.' )? database_name */ @@ -332,6 +345,18 @@ SqlDrop SqlDropDatabase(Span s, boolean replace) : } +SqlCall SqlShowDatabases(Span s, String scope) : +{ +} +{ + + + { + return new SqlShowDatabases(s.end(this), scope); + } +} + + SqlNodeList PartitionFieldList() : { final List list = new ArrayList(); @@ -456,6 +481,19 @@ SqlDrop SqlDropTable(Span s, boolean replace) : } } + +SqlCall SqlShowTables(Span s, String scope) : +{ +} +{ + + + { + return new SqlShowTables(s.end(this), scope); + } +} + + Schema.FieldType FieldType() : { final SqlTypeName collectionTypeName; 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 c76ae79dd45d..384423159a52 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 @@ -126,6 +126,10 @@ public org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Table ge connection.getPipelineOptions()); } + public Collection getTables() { + return tableProvider.getTables().values(); + } + @Override public Set getFunctionNames() { return Collections.emptySet(); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java index ec225efc1c39..c77179e3e245 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java @@ -222,7 +222,11 @@ public CatalogSchema getCurrentCatalogSchema() { @Override public Set getSubSchemaNames() { - return catalogManager.catalogs().stream().map(Catalog::name).collect(Collectors.toSet()); + return catalogs().stream().map(Catalog::name).collect(Collectors.toSet()); + } + + public Collection catalogs() { + return catalogManager.catalogs(); } public void setPipelineOption(String key, String value) { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java index 792e5b98bcd3..2e07f0bda533 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java @@ -72,6 +72,10 @@ public Catalog getCatalog() { return getSubSchema(catalog.currentDatabase()); } + public @Nullable Collection databases() { + return catalog.databases(); + } + public BeamCalciteSchema getDatabaseSchema(TableName tablePath) { @Nullable BeamCalciteSchema beamCalciteSchema = getSubSchema(tablePath.database()); if (beamCalciteSchema == null) { 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 c5d162ebbb68..04eb09bebd35 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 @@ -66,6 +66,13 @@ static Pair schema( return Pair.of(checkStateNotNull(schema, "Got null sub-schema for path '%s'", path), name(id)); } + static CalciteSchema schema(CalcitePrepare.Context context, boolean mutable) { + CalciteSchema rootSchema = mutable ? context.getMutableRootSchema() : context.getRootSchema(); + List path = context.getDefaultSchemaPath(); + @Nullable CalciteSchema schema = childSchema(rootSchema, path); + return checkStateNotNull(schema, "Got null sub-schema for path '%s'", path); + } + private static @Nullable CalciteSchema childSchema(CalciteSchema rootSchema, List path) { @Nullable CalciteSchema schema = rootSchema; for (String p : path) { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java new file mode 100644 index 000000000000..713f071d81f3 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.sql.impl.parser; + +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; + +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; +import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; +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.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; + +public class SqlShowCatalogs extends SqlSetOption implements BeamSqlParser.ExecutableStatement { + private static final SqlOperator OPERATOR = + new SqlSpecialOperator("SHOW CATALOGS", SqlKind.OTHER); + + public SqlShowCatalogs(SqlParserPos pos, String scope) { + super(pos, scope, new SqlIdentifier("", pos), null); + } + + @Override + public List getOperandList() { + return Collections.emptyList(); + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public void execute(CalcitePrepare.Context context) { + Schema schema = SqlDdlNodes.schema(context, true).schema; + + if (!(schema instanceof CatalogManagerSchema)) { + throw SqlUtil.newContextException( + pos, + RESOURCE.internal( + "Attempting execute 'SHOW CATALOGS' with unexpected Calcite Schema of type " + + schema.getClass())); + } + + Collection catalogs = ((CatalogManagerSchema) schema).catalogs(); + print(catalogs); + } + + private static void print(Collection catalogs) { + final String HEADER_NAME = "Catalog Name"; + final String HEADER_TYPE = "Type"; + final String SEPARATOR_CHAR = "-"; + + int nameWidth = HEADER_NAME.length(); + int typeWidth = HEADER_TYPE.length(); + + // find the longest string in each column + for (Catalog catalog : catalogs) { + nameWidth = Math.max(nameWidth, catalog.name().length()); + typeWidth = Math.max(typeWidth, catalog.type().length()); + } + + // add a small padding + nameWidth += 2; + typeWidth += 2; + + // format string with calculated widths for left-justification (%-Ns) + String format = "| %-" + nameWidth + "s | %-" + typeWidth + "s |%n"; + + // separator width = column widths + padding + separators - corners ('+') + int separatorWidth = nameWidth + typeWidth + 5; + String separator = + String.format( + "+" + new String(new char[separatorWidth]).replace("\0", SEPARATOR_CHAR) + "+%n"); + + // printing the table + System.out.printf(separator); + System.out.printf(format, HEADER_NAME, HEADER_TYPE); + System.out.printf(separator); + for (Catalog catalog : + catalogs.stream() + .sorted(Comparator.comparing(Catalog::name)) + .collect(Collectors.toList())) { + System.out.printf(format, catalog.name(), catalog.type()); + } + System.out.printf(separator); + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java new file mode 100644 index 000000000000..a4ed339b9176 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.sql.impl.parser; + +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; +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.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.checkerframework.checker.nullness.qual.Nullable; + +public class SqlShowDatabases extends SqlSetOption implements BeamSqlParser.ExecutableStatement { + private static final SqlOperator OPERATOR = + new SqlSpecialOperator("SHOW DATABASES", SqlKind.OTHER); + + public SqlShowDatabases(SqlParserPos pos, String scope) { + super(pos, scope, new SqlIdentifier("", pos), null); + } + + @Override + public List getOperandList() { + return Collections.emptyList(); + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public void execute(CalcitePrepare.Context context) { + Schema schema = SqlDdlNodes.schema(context, true).schema; + + if (!(schema instanceof CatalogManagerSchema)) { + throw SqlUtil.newContextException( + pos, + RESOURCE.internal( + "Attempting execute 'SHOW DATABASES' with unexpected Calcite Schema of type " + + schema.getClass())); + } + + CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); + Collection databases = catalogSchema.databases(); + print(databases, catalogSchema.getCatalog().name()); + } + + private static void print(@Nullable Collection databases, String path) { + final String HEADER_NAME = "Databases in " + path; + final String SEPARATOR_CHAR = "-"; + + int nameWidth = HEADER_NAME.length(); + + if (databases != null) { + for (String dbName : databases) { + nameWidth = Math.max(nameWidth, dbName.length()); + } + } + + nameWidth += 2; + String format = "| %-" + nameWidth + "s |%n"; + + int separatorWidth = nameWidth + 2; + String separator = + String.format( + "+" + new String(new char[separatorWidth]).replace("\0", SEPARATOR_CHAR) + "+%n"); + + System.out.printf(separator); + System.out.printf(format, HEADER_NAME); + System.out.printf(separator); + if (databases != null) { + for (String dbName : databases.stream().sorted().collect(Collectors.toList())) { + System.out.printf(format, dbName); + } + System.out.printf(separator); + } + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java new file mode 100644 index 000000000000..5d2dd1fe66e3 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.sql.impl.parser; + +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; + +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; +import org.apache.beam.sdk.extensions.sql.meta.Table; +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.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.checkerframework.checker.nullness.qual.Nullable; + +public class SqlShowTables extends SqlSetOption implements BeamSqlParser.ExecutableStatement { + private static final SqlOperator OPERATOR = new SqlSpecialOperator("SHOW TABLES", SqlKind.OTHER); + + public SqlShowTables(SqlParserPos pos, String scope) { + super(pos, scope, new SqlIdentifier("", pos), null); + } + + @Override + public List getOperandList() { + return Collections.emptyList(); + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public void execute(CalcitePrepare.Context context) { + Schema schema = SqlDdlNodes.schema(context, true).schema; + + if (!(schema instanceof CatalogManagerSchema)) { + throw SqlUtil.newContextException( + pos, + RESOURCE.internal( + "Attempting to execute 'SHOW TABLES' with unexpected Calcite Schema of type " + + schema.getClass())); + } + + CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); + @Nullable BeamCalciteSchema databaseSchema = catalogSchema.getCurrentDatabaseSchema(); + if (databaseSchema == null) { + throw SqlUtil.newContextException( + pos, + RESOURCE.internal( + "Attempting to execute 'SHOW TABLES' with no Database used. Please set a Database first then re-run.")); + } + + String path = catalogSchema.getCatalog().name() + "." + databaseSchema.name(); + Collection
tables = databaseSchema.getTables(); + print(tables, path); + } + + private static void print(@Nullable Collection
tables, String path) { + final String HEADER_NAME = "Tables in " + path; + final String HEADER_TYPE = "Type"; + final String SEPARATOR_CHAR = "-"; + + int nameWidth = HEADER_NAME.length(); + int typeWidth = HEADER_TYPE.length(); + + if (tables != null) { + for (Table table : tables) { + nameWidth = Math.max(nameWidth, table.getName().length()); + typeWidth = Math.max(typeWidth, table.getType().length()); + } + } + + nameWidth += 2; + typeWidth += 2; + String format = "| %-" + nameWidth + "s | %-" + typeWidth + "s |%n"; + + int separatorWidth = nameWidth + typeWidth + 5; + String separator = + String.format( + "+" + new String(new char[separatorWidth]).replace("\0", SEPARATOR_CHAR) + "+%n"); + + System.out.printf(separator); + System.out.printf(format, HEADER_NAME, HEADER_TYPE); + System.out.printf(separator); + if (tables != null) { + for (Table table : + tables.stream() + .sorted(Comparator.comparing(Table::getName)) + .collect(Collectors.toList())) { + System.out.printf(format, table.getName(), table.getType()); + } + System.out.printf(separator); + } + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java index db7724a4809d..b606e39de0d4 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.meta.catalog; +import java.util.Collection; import java.util.Map; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; @@ -50,6 +51,9 @@ public interface Catalog { @Nullable String currentDatabase(); + @Nullable + Collection databases(); + /** * Creates a database with this name. * diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java index 3c7ef5623b1b..7070cf8dfe61 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java @@ -20,6 +20,7 @@ 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.checkState; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -97,6 +98,11 @@ public void useDatabase(String database) { return currentDatabase; } + @Override + public @Nullable Collection databases() { + return databases; + } + @Override public boolean dropDatabase(String database, boolean cascade) { checkState(!cascade, "%s does not support CASCADE.", getClass().getSimpleName()); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java index 0164c634814b..9badf97c780e 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java @@ -26,6 +26,8 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; @@ -330,4 +332,25 @@ public void testCreateWriteDropTableWithOtherCatalogScope() { cli.execute("DROP TABLE catalog_1.db_1.person"); assertNull(metastoreDb1.getTable("person")); } + + @Test + public void testShowCatalogs() { + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + cli.execute("CREATE CATALOG my_very_long_catalog_name TYPE 'local'"); + ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outputStreamCaptor)); + cli.execute("SHOW CATALOGS"); + @SuppressWarnings("DefaultCharset") + String printOutput = outputStreamCaptor.toString().trim(); + + assertEquals( + "+---------------------------------------+\n" + + "| Catalog Name | Type |\n" + + "+---------------------------------------+\n" + + "| default | local |\n" + + "| my_catalog | local |\n" + + "| my_very_long_catalog_name | local |\n" + + "+---------------------------------------+", + printOutput); + } } 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 cca1bfd93f27..faaa4f8832b7 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 @@ -24,6 +24,8 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; @@ -209,4 +211,30 @@ public void testCreateInsertDropTableUsingOtherDatabase() { cli.execute("DROP TABLE db_1.person"); assertNull(catalogManager.currentCatalog().metaStore("db_1").getTable("person")); } + + @Test + public void testShowDatabases() { + cli.execute("CREATE DATABASE should_not_show_up"); + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + cli.execute("USE CATALOG my_catalog"); + cli.execute("CREATE DATABASE my_db"); + cli.execute("CREATE DATABASE my_other_db"); + cli.execute("CREATE DATABASE my_database_that_has_a_very_long_name"); + ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outputStreamCaptor)); + cli.execute("SHOW DATABASES"); + @SuppressWarnings("DefaultCharset") + String printOutput = outputStreamCaptor.toString().trim(); + + assertEquals( + "+-----------------------------------------+\n" + + "| Databases in my_catalog |\n" + + "+-----------------------------------------+\n" + + "| default |\n" + + "| my_database_that_has_a_very_long_name |\n" + + "| my_db |\n" + + "| my_other_db |\n" + + "+-----------------------------------------+", + printOutput); + } } 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 ffbdeb84f136..289256e073b8 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 @@ -28,11 +28,14 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; import java.time.LocalDate; import java.time.LocalTime; import java.util.stream.Stream; import org.apache.beam.sdk.extensions.sql.impl.ParseException; import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; @@ -300,4 +303,41 @@ public void test_time_types() throws Exception { // test TIMESTAMP field assertEquals(parseTimestampWithUTCTimeZone("2018-07-01 21:26:07.123"), row.getDateTime("f_ts")); } + + @Test + public void testShowTables() { + InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); + catalogManager.registerTableProvider(new TextTableProvider()); + catalogManager.registerTableProvider(new TestTableProvider()); + BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); + + cli.execute("CREATE DATABASE random_db"); + cli.execute("USE DATABASE random_db"); + cli.execute("CREATE EXTERNAL TABLE should_not_show_up (id int, name varchar) TYPE 'text'"); + + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + cli.execute("USE CATALOG my_catalog"); + cli.execute("CREATE DATABASE my_db"); + cli.execute("USE DATABASE my_db"); + cli.execute("CREATE EXTERNAL TABLE my_table (id int, name varchar) TYPE 'text'"); + cli.execute("CREATE EXTERNAL TABLE my_other_table (col1 int, col2 timestamp) TYPE 'text'"); + cli.execute( + "CREATE EXTERNAL TABLE my_other_table_with_a_long_name " + + "(foo varchar, bar boolean) TYPE 'test'"); + ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outputStreamCaptor)); + cli.execute("SHOW TABLES"); + @SuppressWarnings("DefaultCharset") + String printOutput = outputStreamCaptor.toString().trim(); + + assertEquals( + "+--------------------------------------------+\n" + + "| Tables in my_catalog.my_db | Type |\n" + + "+--------------------------------------------+\n" + + "| my_other_table | text |\n" + + "| my_other_table_with_a_long_name | test |\n" + + "| my_table | text |\n" + + "+--------------------------------------------+", + printOutput); + } } From ff335dcc4fb09630becb06f45789187a30549077 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 14 Oct 2025 11:48:51 -0400 Subject: [PATCH 02/14] add SHOW CURRENT --- .../sql/src/main/codegen/config.fmpp | 5 ++- .../src/main/codegen/includes/parserImpls.ftl | 29 +++++++++++++--- .../sql/impl/parser/SqlShowCatalogs.java | 14 ++++++-- .../sql/impl/parser/SqlShowDatabases.java | 15 +++++++-- .../extensions/sql/BeamSqlCliCatalogTest.java | 13 ++++++++ .../sql/BeamSqlCliDatabaseTest.java | 33 +++++++++++++++++++ 6 files changed, 97 insertions(+), 12 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/codegen/config.fmpp b/sdks/java/extensions/sql/src/main/codegen/config.fmpp index cc469f97266c..7e1fd233ad90 100644 --- a/sdks/java/extensions/sql/src/main/codegen/config.fmpp +++ b/sdks/java/extensions/sql/src/main/codegen/config.fmpp @@ -403,7 +403,6 @@ data: { "IF" "JAR" "LOCATION" - "SHOW" "TBLPROPERTIES" "PROPERTIES" "PARTITIONED" @@ -429,9 +428,9 @@ data: { # Return type of method implementation should be 'SqlNode'. # Example: SqlShowDatabases(), SqlShowTables(). statementParserMethods: [ - "SqlShowCatalogs(Span.of(), null)" - "SqlShowDatabases(Span.of(), null)" "SqlShowTables(Span.of(), null)" + "SqlShowDatabases(Span.of(), null)" + "SqlShowCatalogs(Span.of(), null)" "SqlUseCatalog(Span.of(), null)" "SqlUseDatabase(Span.of(), null)" "SqlSetOptionBeam(Span.of(), null)" diff --git a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl index e7ef7963aec6..1e22b1324a71 100644 --- a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl +++ b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl @@ -265,14 +265,25 @@ SqlDrop SqlDropCatalog(Span s, boolean replace) : } +boolean OnlyCurrentCatalog() : +{ +} +{ + { return true; } +| + { return false; } +} + SqlCall SqlShowCatalogs(Span s, String scope) : { + final boolean showCurrentOnly; } { - + showCurrentOnly = OnlyCurrentCatalog() + { - return new SqlShowCatalogs(s.end(this), scope); + return new SqlShowCatalogs(s.end(this), scope, showCurrentOnly); } } @@ -345,14 +356,24 @@ SqlDrop SqlDropDatabase(Span s, boolean replace) : } +boolean OnlyCurrentDatabase() : +{ +} +{ + { return true; } +| + { return false; } +} + SqlCall SqlShowDatabases(Span s, String scope) : { + final boolean showCurrentDatabase; } { - + showCurrentDatabase = OnlyCurrentDatabase() { - return new SqlShowDatabases(s.end(this), scope); + return new SqlShowDatabases(s.end(this), scope, showCurrentDatabase); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java index 713f071d81f3..288682919e29 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java @@ -41,8 +41,11 @@ public class SqlShowCatalogs extends SqlSetOption implements BeamSqlParser.Execu private static final SqlOperator OPERATOR = new SqlSpecialOperator("SHOW CATALOGS", SqlKind.OTHER); - public SqlShowCatalogs(SqlParserPos pos, String scope) { + private final boolean showCurrentOnly; + + public SqlShowCatalogs(SqlParserPos pos, String scope, boolean showCurrentOnly) { super(pos, scope, new SqlIdentifier("", pos), null); + this.showCurrentOnly = showCurrentOnly; } @Override @@ -66,8 +69,13 @@ public void execute(CalcitePrepare.Context context) { "Attempting execute 'SHOW CATALOGS' with unexpected Calcite Schema of type " + schema.getClass())); } - - Collection catalogs = ((CatalogManagerSchema) schema).catalogs(); + CatalogManagerSchema managerSchema = ((CatalogManagerSchema) schema); + if (showCurrentOnly) { + Catalog currentCatalog = managerSchema.getCurrentCatalogSchema().getCatalog(); + System.out.printf("%s (type: %s)", currentCatalog.name(), currentCatalog.type()); + return; + } + Collection catalogs = managerSchema.catalogs(); print(catalogs); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java index a4ed339b9176..9f2daa30920e 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java @@ -23,6 +23,7 @@ import java.util.Collections; import java.util.List; import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; @@ -41,8 +42,11 @@ public class SqlShowDatabases extends SqlSetOption implements BeamSqlParser.Exec private static final SqlOperator OPERATOR = new SqlSpecialOperator("SHOW DATABASES", SqlKind.OTHER); - public SqlShowDatabases(SqlParserPos pos, String scope) { + private final boolean showCurrentOnly; + + public SqlShowDatabases(SqlParserPos pos, String scope, boolean showCurrentOnly) { super(pos, scope, new SqlIdentifier("", pos), null); + this.showCurrentOnly = showCurrentOnly; } @Override @@ -63,11 +67,18 @@ public void execute(CalcitePrepare.Context context) { throw SqlUtil.newContextException( pos, RESOURCE.internal( - "Attempting execute 'SHOW DATABASES' with unexpected Calcite Schema of type " + "Attempting to execute 'SHOW DATABASES' with unexpected Calcite Schema of type " + schema.getClass())); } CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); + if (showCurrentOnly) { + @Nullable BeamCalciteSchema currentDatabase = catalogSchema.getCurrentDatabaseSchema(); + String output = + currentDatabase == null ? "No database is currently set" : currentDatabase.name(); + System.out.println(output); + return; + } Collection databases = catalogSchema.databases(); print(databases, catalogSchema.getCatalog().name()); } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java index 9badf97c780e..5f318eadf8ae 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java @@ -333,6 +333,19 @@ public void testCreateWriteDropTableWithOtherCatalogScope() { assertNull(metastoreDb1.getTable("person")); } + @Test + public void testShowCurrentCatalog() { + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + cli.execute("CREATE CATALOG my_very_long_catalog_name TYPE 'local'"); + ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outputStreamCaptor)); + cli.execute("SHOW CURRENT CATALOG"); + @SuppressWarnings("DefaultCharset") + String printOutput = outputStreamCaptor.toString().trim(); + + assertEquals("default (type: local)", printOutput); + } + @Test public void testShowCatalogs() { cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); 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 faaa4f8832b7..372de04cbacd 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 @@ -212,6 +212,39 @@ public void testCreateInsertDropTableUsingOtherDatabase() { assertNull(catalogManager.currentCatalog().metaStore("db_1").getTable("person")); } + @Test + public void testShowCurrentDatabase() { + cli.execute("CREATE DATABASE should_not_show_up"); + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + cli.execute("USE CATALOG my_catalog"); + cli.execute("CREATE DATABASE my_db"); + cli.execute("CREATE DATABASE my_other_db"); + cli.execute("CREATE DATABASE my_database_that_has_a_very_long_name"); + cli.execute("USE DATABASE my_other_db"); + ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outputStreamCaptor)); + cli.execute("SHOW CURRENT database"); + @SuppressWarnings("DefaultCharset") + String printOutput = outputStreamCaptor.toString().trim(); + + assertEquals("my_other_db", printOutput); + } + + @Test + public void testShowCurrentDatabaseWithNoneSet() { + cli.execute("CREATE DATABASE should_not_show_up"); + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + cli.execute("USE CATALOG my_catalog"); + cli.execute("DROP DATABASE `default`"); + ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outputStreamCaptor)); + cli.execute("SHOW CURRENT DATABASE"); + @SuppressWarnings("DefaultCharset") + String printOutput = outputStreamCaptor.toString().trim(); + + assertEquals("No database is currently set", printOutput); + } + @Test public void testShowDatabases() { cli.execute("CREATE DATABASE should_not_show_up"); From 50928b8b23d6e24f08ed4d81d1c0625a6a4df826 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 14 Oct 2025 12:02:30 -0400 Subject: [PATCH 03/14] add SHOW CURRENT --- .../sql/src/main/codegen/config.fmpp | 1 + .../src/main/codegen/includes/parserImpls.ftl | 47 ++++++++----------- 2 files changed, 20 insertions(+), 28 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/codegen/config.fmpp b/sdks/java/extensions/sql/src/main/codegen/config.fmpp index 7e1fd233ad90..fa98fc27b363 100644 --- a/sdks/java/extensions/sql/src/main/codegen/config.fmpp +++ b/sdks/java/extensions/sql/src/main/codegen/config.fmpp @@ -431,6 +431,7 @@ data: { "SqlShowTables(Span.of(), null)" "SqlShowDatabases(Span.of(), null)" "SqlShowCatalogs(Span.of(), null)" + "SqlShowCurrent(Span.of(), null)" "SqlUseCatalog(Span.of(), null)" "SqlUseDatabase(Span.of(), null)" "SqlSetOptionBeam(Span.of(), null)" diff --git a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl index 1e22b1324a71..75558494281c 100644 --- a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl +++ b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl @@ -264,26 +264,13 @@ SqlDrop SqlDropCatalog(Span s, boolean replace) : } } - -boolean OnlyCurrentCatalog() : -{ -} -{ - { return true; } -| - { return false; } -} - SqlCall SqlShowCatalogs(Span s, String scope) : { - final boolean showCurrentOnly; } { - - showCurrentOnly = OnlyCurrentCatalog() - + { s.add(this); } { - return new SqlShowCatalogs(s.end(this), scope, showCurrentOnly); + return new SqlShowCatalogs(s.end(this), scope, false); } } @@ -356,25 +343,30 @@ SqlDrop SqlDropDatabase(Span s, boolean replace) : } -boolean OnlyCurrentDatabase() : +SqlCall SqlShowDatabases(Span s, String scope) : { } { - { return true; } -| - { return false; } + { s.add(this); } + { + return new SqlShowDatabases(s.end(this), scope, false); + } } -SqlCall SqlShowDatabases(Span s, String scope) : +SqlCall SqlShowCurrent(Span s, String scope) : { - final boolean showCurrentDatabase; } { - - showCurrentDatabase = OnlyCurrentDatabase() - { - return new SqlShowDatabases(s.end(this), scope, showCurrentDatabase); - } + { s.add(this); } + ( + { + return new SqlShowCatalogs(s.end(this), scope, true); + } + | + { + return new SqlShowDatabases(s.end(this), scope, true); + } + ) } @@ -507,8 +499,7 @@ SqlCall SqlShowTables(Span s, String scope) : { } { - - + { s.add(this); } { return new SqlShowTables(s.end(this), scope); } From feb2d8196e945e6cdb0c34f4e12e5b5199b3cb76 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 14 Oct 2025 15:02:20 -0400 Subject: [PATCH 04/14] add LIKE pattern --- .../src/main/codegen/includes/parserImpls.ftl | 31 +++++++--- .../sql/impl/CatalogManagerSchema.java | 16 +++-- .../impl/parser/SqlCreateExternalTable.java | 5 +- .../sql/impl/parser/SqlDdlNodes.java | 2 +- .../sql/impl/parser/SqlDropDatabase.java | 5 +- .../sql/impl/parser/SqlDropTable.java | 5 +- .../sql/impl/parser/SqlShowCatalogs.java | 17 ++++-- .../sql/impl/parser/SqlShowDatabases.java | 36 ++++++++--- .../sql/impl/parser/SqlShowTables.java | 45 +++++++++++--- .../sql/impl/parser/SqlUseDatabase.java | 5 +- .../extensions/sql/BeamSqlCliCatalogTest.java | 21 +++++++ .../sql/BeamSqlCliDatabaseTest.java | 51 +++++++++++++++- .../sdk/extensions/sql/BeamSqlCliTest.java | 59 +++++++++++++++++++ 13 files changed, 260 insertions(+), 38 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl index 75558494281c..74dca414b02b 100644 --- a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl +++ b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl @@ -264,13 +264,18 @@ SqlDrop SqlDropCatalog(Span s, boolean replace) : } } +/** + * SHOW CATALOGS [ LIKE regex_pattern ] + */ SqlCall SqlShowCatalogs(Span s, String scope) : { + SqlNode regex = null; } { { s.add(this); } + [ regex = StringLiteral() ] { - return new SqlShowCatalogs(s.end(this), scope, false); + return new SqlShowCatalogs(s.end(this), scope, false, regex); } } @@ -342,14 +347,20 @@ SqlDrop SqlDropDatabase(Span s, boolean replace) : } } - +/** + * SHOW DATABASES [ ( FROM | IN )? catalog_name ] [LIKE regex_pattern ] + */ SqlCall SqlShowDatabases(Span s, String scope) : { + SqlIdentifier catalogName = null; + SqlNode regex = null; } { { s.add(this); } + [ ( | ) catalogName = SimpleIdentifier() ] + [ regex = StringLiteral() ] { - return new SqlShowDatabases(s.end(this), scope, false); + return new SqlShowDatabases(s.end(this), scope, false, catalogName, regex); } } @@ -360,11 +371,11 @@ SqlCall SqlShowCurrent(Span s, String scope) : { s.add(this); } ( { - return new SqlShowCatalogs(s.end(this), scope, true); + return new SqlShowCatalogs(s.end(this), scope, true, null); } | { - return new SqlShowDatabases(s.end(this), scope, true); + return new SqlShowDatabases(s.end(this), scope, true, null, null); } ) } @@ -494,14 +505,20 @@ SqlDrop SqlDropTable(Span s, boolean replace) : } } - +/** + * SHOW TABLES [ ( FROM | IN )? [ catalog_name '.' ] database_name ] [ LIKE regex_pattern ] + */ SqlCall SqlShowTables(Span s, String scope) : { + SqlIdentifier database = null; + SqlNode regex = null; } { { s.add(this); } + [ ( | ) database = CompoundIdentifier() ] + [ regex = StringLiteral() ] { - return new SqlShowTables(s.end(this), scope); + return new SqlShowTables(s.end(this), scope, database, regex); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java index c77179e3e245..480a5a99a70b 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.impl; +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_40_0.org.apache.calcite.util.Static.RESOURCE; @@ -142,7 +143,8 @@ public void dropCatalog(SqlIdentifier identifier, boolean ifExists) { // will attempt to do so. public void maybeRegisterProvider(TableName path, String type) { type = type.toLowerCase(); - CatalogSchema catalogSchema = getCatalogSchema(path); + CatalogSchema catalogSchema = + path.catalog() != null ? getCatalogSchema(path) : getCurrentCatalogSchema(); BeamCalciteSchema beamCalciteSchema = catalogSchema.getDatabaseSchema(path); if (beamCalciteSchema.getTableProvider() instanceof MetaStore) { @@ -177,14 +179,16 @@ public Set getTableNames() { } public CatalogSchema getCatalogSchema(TableName tablePath) { - @Nullable Schema catalogSchema = getSubSchema(tablePath.catalog()); - if (catalogSchema == null) { - catalogSchema = getCurrentCatalogSchema(); - } + return getCatalogSchema(tablePath.catalog()); + } + + public CatalogSchema getCatalogSchema(@Nullable String catalog) { + Schema catalogSchema = + checkArgumentNotNull(getSubSchema(catalog), "Catalog '%s' not found.", catalog); Preconditions.checkState( catalogSchema instanceof CatalogSchema, "Unexpected Schema type for Catalog '%s': %s", - tablePath.catalog(), + catalog, catalogSchema.getClass()); return (CatalogSchema) catalogSchema; } 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 ab644145b4f7..5adc88ccc407 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 @@ -159,7 +159,10 @@ public void execute(CalcitePrepare.Context context) { CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; catalogManagerSchema.maybeRegisterProvider(pathOverride, SqlDdlNodes.getString(type)); - CatalogSchema catalogSchema = catalogManagerSchema.getCatalogSchema(pathOverride); + CatalogSchema catalogSchema = + pathOverride.catalog() != null + ? catalogManagerSchema.getCatalogSchema(pathOverride) + : catalogManagerSchema.getCurrentCatalogSchema(); beamCalciteSchema = catalogSchema.getDatabaseSchema(pathOverride); } else if (schema instanceof BeamCalciteSchema) { beamCalciteSchema = (BeamCalciteSchema) schema; 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 04eb09bebd35..44b6d092378b 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 @@ -92,7 +92,7 @@ public static String name(SqlIdentifier id) { } } - static @Nullable String getString(SqlNode n) { + static @Nullable String getString(@Nullable SqlNode n) { if (n == null) { return null; } 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 4b838c9f4182..e3db29c4c14b 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 @@ -84,7 +84,10 @@ public void execute(CalcitePrepare.Context context) { List components = Lists.newArrayList(Splitter.on(".").split(databaseName.toString())); TableName pathOverride = TableName.create(components, ""); - CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); + CatalogSchema catalogSchema = + pathOverride.catalog() != null + ? ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride) + : ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); catalogSchema.dropDatabase(databaseName, cascade, ifExists); } 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 0bc5cd911614..b8cc5f4c72dc 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 @@ -53,7 +53,10 @@ public void execute(CalcitePrepare.Context context) { BeamCalciteSchema beamCalciteSchema; if (schema instanceof CatalogManagerSchema) { - CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); + CatalogSchema catalogSchema = + pathOverride.catalog() != null + ? ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride) + : ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); beamCalciteSchema = catalogSchema.getDatabaseSchema(pathOverride); } else if (schema instanceof BeamCalciteSchema) { beamCalciteSchema = (BeamCalciteSchema) schema; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java index 288682919e29..88c9a6addbd0 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; 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.runtime.SqlFunctions; 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; @@ -36,16 +37,20 @@ 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.checkerframework.checker.nullness.qual.Nullable; public class SqlShowCatalogs extends SqlSetOption implements BeamSqlParser.ExecutableStatement { private static final SqlOperator OPERATOR = new SqlSpecialOperator("SHOW CATALOGS", SqlKind.OTHER); private final boolean showCurrentOnly; + private final @Nullable SqlNode regex; - public SqlShowCatalogs(SqlParserPos pos, String scope, boolean showCurrentOnly) { + public SqlShowCatalogs( + SqlParserPos pos, String scope, boolean showCurrentOnly, @Nullable SqlNode regex) { super(pos, scope, new SqlIdentifier("", pos), null); this.showCurrentOnly = showCurrentOnly; + this.regex = regex; } @Override @@ -76,10 +81,12 @@ public void execute(CalcitePrepare.Context context) { return; } Collection catalogs = managerSchema.catalogs(); - print(catalogs); + print(catalogs, SqlDdlNodes.getString(regex)); } - private static void print(Collection catalogs) { + private static void print(Collection catalogs, @Nullable String pattern) { + SqlFunctions.LikeFunction calciteLike = new SqlFunctions.LikeFunction(); + final String HEADER_NAME = "Catalog Name"; final String HEADER_TYPE = "Type"; final String SEPARATOR_CHAR = "-"; @@ -114,7 +121,9 @@ private static void print(Collection catalogs) { catalogs.stream() .sorted(Comparator.comparing(Catalog::name)) .collect(Collectors.toList())) { - System.out.printf(format, catalog.name(), catalog.type()); + if (pattern == null || calciteLike.like(catalog.name(), pattern)) { + System.out.printf(format, catalog.name(), catalog.type()); + } } System.out.printf(separator); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java index 9f2daa30920e..5d0bee2d3467 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; 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.runtime.SqlFunctions; 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; @@ -43,10 +44,19 @@ public class SqlShowDatabases extends SqlSetOption implements BeamSqlParser.Exec new SqlSpecialOperator("SHOW DATABASES", SqlKind.OTHER); private final boolean showCurrentOnly; - - public SqlShowDatabases(SqlParserPos pos, String scope, boolean showCurrentOnly) { + private final @Nullable SqlIdentifier catalogName; + private final @Nullable SqlNode regex; + + public SqlShowDatabases( + SqlParserPos pos, + String scope, + boolean showCurrentOnly, + @Nullable SqlIdentifier catalogName, + @Nullable SqlNode regex) { super(pos, scope, new SqlIdentifier("", pos), null); this.showCurrentOnly = showCurrentOnly; + this.catalogName = catalogName; + this.regex = regex; } @Override @@ -71,7 +81,14 @@ public void execute(CalcitePrepare.Context context) { + schema.getClass())); } - CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); + CatalogSchema catalogSchema; + if (catalogName != null) { + String catalog = catalogName.toString(); + catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(catalog); + } else { + catalogSchema = ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); + } + if (showCurrentOnly) { @Nullable BeamCalciteSchema currentDatabase = catalogSchema.getCurrentDatabaseSchema(); String output = @@ -80,10 +97,13 @@ public void execute(CalcitePrepare.Context context) { return; } Collection databases = catalogSchema.databases(); - print(databases, catalogSchema.getCatalog().name()); + print(databases, catalogSchema.getCatalog().name(), SqlDdlNodes.getString(regex)); } - private static void print(@Nullable Collection databases, String path) { + private static void print( + @Nullable Collection databases, String path, @Nullable String pattern) { + SqlFunctions.LikeFunction calciteLike = new SqlFunctions.LikeFunction(); + final String HEADER_NAME = "Databases in " + path; final String SEPARATOR_CHAR = "-"; @@ -108,9 +128,11 @@ private static void print(@Nullable Collection databases, String path) { System.out.printf(separator); if (databases != null) { for (String dbName : databases.stream().sorted().collect(Collectors.toList())) { - System.out.printf(format, dbName); + if (pattern == null || calciteLike.like(dbName, pattern)) { + System.out.printf(format, dbName); + } } - System.out.printf(separator); } + System.out.printf(separator); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java index 5d2dd1fe66e3..52bb5b0e8219 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java @@ -27,8 +27,10 @@ import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; +import org.apache.beam.sdk.extensions.sql.impl.TableName; import org.apache.beam.sdk.extensions.sql.meta.Table; 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.runtime.SqlFunctions; 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; @@ -38,13 +40,23 @@ 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.guava.v32_1_2_jre.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; public class SqlShowTables extends SqlSetOption implements BeamSqlParser.ExecutableStatement { private static final SqlOperator OPERATOR = new SqlSpecialOperator("SHOW TABLES", SqlKind.OTHER); - - public SqlShowTables(SqlParserPos pos, String scope) { + private final @Nullable SqlIdentifier databaseName; + private final @Nullable SqlNode regex; + + public SqlShowTables( + SqlParserPos pos, + String scope, + @Nullable SqlIdentifier databaseName, + @Nullable SqlNode regex) { super(pos, scope, new SqlIdentifier("", pos), null); + this.databaseName = databaseName; + this.regex = regex; } @Override @@ -69,8 +81,22 @@ public void execute(CalcitePrepare.Context context) { + schema.getClass())); } - CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); - @Nullable BeamCalciteSchema databaseSchema = catalogSchema.getCurrentDatabaseSchema(); + CatalogSchema catalogSchema; + @Nullable BeamCalciteSchema databaseSchema; + if (databaseName != null) { + List components = Lists.newArrayList(Splitter.on(".").split(databaseName.toString())); + TableName pathOverride = TableName.create(components, ""); + catalogSchema = + pathOverride.catalog() != null + ? ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride) + : ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); + + databaseSchema = catalogSchema.getDatabaseSchema(pathOverride); + } else { + catalogSchema = ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); + databaseSchema = catalogSchema.getCurrentDatabaseSchema(); + } + if (databaseSchema == null) { throw SqlUtil.newContextException( pos, @@ -80,10 +106,13 @@ public void execute(CalcitePrepare.Context context) { String path = catalogSchema.getCatalog().name() + "." + databaseSchema.name(); Collection
tables = databaseSchema.getTables(); - print(tables, path); + print(tables, path, SqlDdlNodes.getString(regex)); } - private static void print(@Nullable Collection
tables, String path) { + private static void print( + @Nullable Collection
tables, String path, @Nullable String pattern) { + SqlFunctions.LikeFunction calciteLike = new SqlFunctions.LikeFunction(); + final String HEADER_NAME = "Tables in " + path; final String HEADER_TYPE = "Type"; final String SEPARATOR_CHAR = "-"; @@ -115,7 +144,9 @@ private static void print(@Nullable Collection
tables, String path) { tables.stream() .sorted(Comparator.comparing(Table::getName)) .collect(Collectors.toList())) { - System.out.printf(format, table.getName(), table.getType()); + if (pattern == null || calciteLike.like(table.getName(), pattern)) { + System.out.printf(format, table.getName(), table.getType()); + } } System.out.printf(separator); } 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 9d06e471dbbe..da749c2ceb1e 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 @@ -78,7 +78,10 @@ public void execute(CalcitePrepare.Context context) { } CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; - CatalogSchema catalogSchema = catalogManagerSchema.getCatalogSchema(pathOverride); + CatalogSchema catalogSchema = + pathOverride.catalog() != null + ? catalogManagerSchema.getCatalogSchema(pathOverride) + : catalogManagerSchema.getCurrentCatalogSchema(); // if database exists in a different catalog, we need to also switch to that catalog if (pathOverride.catalog() != null && !pathOverride diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java index 5f318eadf8ae..a0677ac4d40a 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java @@ -366,4 +366,25 @@ public void testShowCatalogs() { + "+---------------------------------------+", printOutput); } + + @Test + public void testShowCatalogsWithPattern() { + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + cli.execute("CREATE CATALOG my_very_long_catalog_name TYPE 'local'"); + ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outputStreamCaptor)); + cli.execute("SHOW CATALOGS"); + @SuppressWarnings("DefaultCharset") + String printOutput = outputStreamCaptor.toString().trim(); + + assertEquals( + "+---------------------------------------+\n" + + "| Catalog Name | Type |\n" + + "+---------------------------------------+\n" + + "| default | local |\n" + + "| my_catalog | local |\n" + + "| my_very_long_catalog_name | local |\n" + + "+---------------------------------------+", + printOutput); + } } 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 372de04cbacd..a497e294971a 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 @@ -107,8 +107,8 @@ public void testUseDatabaseWithDeletedCatalog_notFound() { assertEquals( ImmutableSet.of("default"), catalogManager.catalogs().stream().map(Catalog::name).collect(Collectors.toSet())); - thrown.expect(CalciteContextException.class); - thrown.expectMessage("Cannot use catalog: 'my_catalog' not found."); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Catalog 'my_catalog' not found"); cli.execute("USE DATABASE my_catalog.my_database"); } @@ -270,4 +270,51 @@ public void testShowDatabases() { + "+-----------------------------------------+", printOutput); } + + @Test + public void testShowDatabasesInOtherCatalog() { + cli.execute("CREATE DATABASE should_not_show_up"); + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + cli.execute("USE CATALOG my_catalog"); + cli.execute("CREATE DATABASE my_db"); + cli.execute("CREATE CATALOG my_other_catalog TYPE 'local'"); + cli.execute("CREATE DATABASE my_other_catalog.other_db"); + ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outputStreamCaptor)); + cli.execute("SHOW DATABASES FROM my_other_catalog"); + @SuppressWarnings("DefaultCharset") + String printOutput = outputStreamCaptor.toString().trim(); + + assertEquals( + "+---------------------------------+\n" + + "| Databases in my_other_catalog |\n" + + "+---------------------------------+\n" + + "| default |\n" + + "| other_db |\n" + + "+---------------------------------+", + printOutput); + } + + @Test + public void testShowDatabasesWithPattern() { + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + cli.execute("CREATE DATABASE my_catalog.my_db"); + cli.execute("CREATE DATABASE my_catalog.other_db"); + cli.execute("CREATE DATABASE my_catalog.some_foo_db"); + cli.execute("CREATE DATABASE my_catalog.some_other_foo_db"); + ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outputStreamCaptor)); + cli.execute("SHOW DATABASES FROM my_catalog LIKE '%foo%'"); + @SuppressWarnings("DefaultCharset") + String printOutput = outputStreamCaptor.toString().trim(); + + assertEquals( + "+---------------------------+\n" + + "| Databases in my_catalog |\n" + + "+---------------------------+\n" + + "| some_foo_db |\n" + + "| some_other_foo_db |\n" + + "+---------------------------+", + printOutput); + } } 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 289256e073b8..06fec2e7a111 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 @@ -340,4 +340,63 @@ public void testShowTables() { + "+--------------------------------------------+", printOutput); } + + @Test + public void testShowTablesInOtherDatabase() { + InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); + catalogManager.registerTableProvider(new TextTableProvider()); + catalogManager.registerTableProvider(new TestTableProvider()); + BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); + + cli.execute("CREATE DATABASE my_db"); + cli.execute( + "CREATE EXTERNAL TABLE my_db.should_not_show_up (id int, name varchar) TYPE 'text'"); + + cli.execute("CREATE CATALOG other_catalog TYPE 'local'"); + cli.execute("CREATE DATABASE other_catalog.other_db"); + cli.execute( + "CREATE EXTERNAL TABLE other_catalog.other_db.other_table (id int, name varchar) TYPE 'text'"); + ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outputStreamCaptor)); + cli.execute("SHOW TABLES IN other_catalog.other_db"); + @SuppressWarnings("DefaultCharset") + String printOutput = outputStreamCaptor.toString().trim(); + + assertEquals( + "+---------------------------------------------+\n" + + "| Tables in other_catalog.other_db | Type |\n" + + "+---------------------------------------------+\n" + + "| other_table | text |\n" + + "+---------------------------------------------+", + printOutput); + } + + @Test + public void testShowTablesWithPattern() { + InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); + catalogManager.registerTableProvider(new TextTableProvider()); + catalogManager.registerTableProvider(new TestTableProvider()); + BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); + + cli.execute("CREATE DATABASE my_db"); + cli.execute("CREATE EXTERNAL TABLE my_db.my_table (id int, name varchar) TYPE 'text'"); + cli.execute("CREATE EXTERNAL TABLE my_db.my_table_2 (id int, name varchar) TYPE 'text'"); + cli.execute("CREATE EXTERNAL TABLE my_db.my_foo_table_1 (id int, name varchar) TYPE 'text'"); + cli.execute("CREATE EXTERNAL TABLE my_db.my_foo_table_2 (id int, name varchar) TYPE 'text'"); + + ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outputStreamCaptor)); + cli.execute("SHOW TABLES IN my_db LIKE '%foo%'"); + @SuppressWarnings("DefaultCharset") + String printOutput = outputStreamCaptor.toString().trim(); + + assertEquals( + "+------------------------------------+\n" + + "| Tables in default.my_db | Type |\n" + + "+------------------------------------+\n" + + "| my_foo_table_1 | text |\n" + + "| my_foo_table_2 | text |\n" + + "+------------------------------------+", + printOutput); + } } From 0e816b0d270fe52f490561903d9ba735c069c99a Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 14 Oct 2025 16:01:22 -0400 Subject: [PATCH 05/14] spotless --- .../sql/impl/parser/SqlShowCatalogs.java | 14 +++++++------- .../sql/impl/parser/SqlShowDatabases.java | 10 +++++----- .../extensions/sql/impl/parser/SqlShowTables.java | 14 +++++++------- 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java index 88c9a6addbd0..83e8ca7a0520 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java @@ -87,12 +87,12 @@ public void execute(CalcitePrepare.Context context) { private static void print(Collection catalogs, @Nullable String pattern) { SqlFunctions.LikeFunction calciteLike = new SqlFunctions.LikeFunction(); - final String HEADER_NAME = "Catalog Name"; - final String HEADER_TYPE = "Type"; - final String SEPARATOR_CHAR = "-"; + final String headerName = "Catalog Name"; + final String headerType = "Type"; + final String separatorChar = "-"; - int nameWidth = HEADER_NAME.length(); - int typeWidth = HEADER_TYPE.length(); + int nameWidth = headerName.length(); + int typeWidth = headerType.length(); // find the longest string in each column for (Catalog catalog : catalogs) { @@ -111,11 +111,11 @@ private static void print(Collection catalogs, @Nullable String pattern int separatorWidth = nameWidth + typeWidth + 5; String separator = String.format( - "+" + new String(new char[separatorWidth]).replace("\0", SEPARATOR_CHAR) + "+%n"); + "+" + new String(new char[separatorWidth]).replace("\0", separatorChar) + "+%n"); // printing the table System.out.printf(separator); - System.out.printf(format, HEADER_NAME, HEADER_TYPE); + System.out.printf(format, headerName, headerType); System.out.printf(separator); for (Catalog catalog : catalogs.stream() diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java index 5d0bee2d3467..24bc47200fed 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java @@ -104,10 +104,10 @@ private static void print( @Nullable Collection databases, String path, @Nullable String pattern) { SqlFunctions.LikeFunction calciteLike = new SqlFunctions.LikeFunction(); - final String HEADER_NAME = "Databases in " + path; - final String SEPARATOR_CHAR = "-"; + final String headerName = "Databases in " + path; + final String separatorChar = "-"; - int nameWidth = HEADER_NAME.length(); + int nameWidth = headerName.length(); if (databases != null) { for (String dbName : databases) { @@ -121,10 +121,10 @@ private static void print( int separatorWidth = nameWidth + 2; String separator = String.format( - "+" + new String(new char[separatorWidth]).replace("\0", SEPARATOR_CHAR) + "+%n"); + "+" + new String(new char[separatorWidth]).replace("\0", separatorChar) + "+%n"); System.out.printf(separator); - System.out.printf(format, HEADER_NAME); + System.out.printf(format, headerName); System.out.printf(separator); if (databases != null) { for (String dbName : databases.stream().sorted().collect(Collectors.toList())) { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java index 52bb5b0e8219..b5e6d7a66e08 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java @@ -113,12 +113,12 @@ private static void print( @Nullable Collection
tables, String path, @Nullable String pattern) { SqlFunctions.LikeFunction calciteLike = new SqlFunctions.LikeFunction(); - final String HEADER_NAME = "Tables in " + path; - final String HEADER_TYPE = "Type"; - final String SEPARATOR_CHAR = "-"; + final String headerName = "Tables in " + path; + final String headerType = "Type"; + final String separatorChar = "-"; - int nameWidth = HEADER_NAME.length(); - int typeWidth = HEADER_TYPE.length(); + int nameWidth = headerName.length(); + int typeWidth = headerType.length(); if (tables != null) { for (Table table : tables) { @@ -134,10 +134,10 @@ private static void print( int separatorWidth = nameWidth + typeWidth + 5; String separator = String.format( - "+" + new String(new char[separatorWidth]).replace("\0", SEPARATOR_CHAR) + "+%n"); + "+" + new String(new char[separatorWidth]).replace("\0", separatorChar) + "+%n"); System.out.printf(separator); - System.out.printf(format, HEADER_NAME, HEADER_TYPE); + System.out.printf(format, headerName, headerType); System.out.printf(separator); if (tables != null) { for (Table table : From f71cf7e32f78d2bdc6f75b9d68413cbb139c06fc Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 11 Nov 2025 23:02:48 +0300 Subject: [PATCH 06/14] use SqlCall; remove @Nullable --- .../meta/provider/iceberg/IcebergCatalog.java | 2 +- .../sql/src/main/codegen/config.fmpp | 8 ++++---- .../src/main/codegen/includes/parserImpls.ftl | 18 +++++++++--------- .../sdk/extensions/sql/impl/CatalogSchema.java | 2 +- .../sql/impl/parser/SqlShowCatalogs.java | 12 +++++------- .../sql/impl/parser/SqlShowDatabases.java | 18 +++++++----------- .../sql/impl/parser/SqlShowTables.java | 14 ++++++-------- .../extensions/sql/meta/catalog/Catalog.java | 1 - .../sql/meta/catalog/InMemoryCatalog.java | 2 +- 9 files changed, 34 insertions(+), 43 deletions(-) diff --git a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java index bb307207a327..4fdbad94a95f 100644 --- a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java +++ b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java @@ -74,7 +74,7 @@ public boolean createDatabase(String database) { } @Override - public @Nullable Collection databases() { + public Collection databases() { return catalogConfig.listNamespaces(); } diff --git a/sdks/java/extensions/sql/src/main/codegen/config.fmpp b/sdks/java/extensions/sql/src/main/codegen/config.fmpp index fa98fc27b363..ef067356f8c5 100644 --- a/sdks/java/extensions/sql/src/main/codegen/config.fmpp +++ b/sdks/java/extensions/sql/src/main/codegen/config.fmpp @@ -428,10 +428,10 @@ data: { # Return type of method implementation should be 'SqlNode'. # Example: SqlShowDatabases(), SqlShowTables(). statementParserMethods: [ - "SqlShowTables(Span.of(), null)" - "SqlShowDatabases(Span.of(), null)" - "SqlShowCatalogs(Span.of(), null)" - "SqlShowCurrent(Span.of(), null)" + "SqlShowTables(Span.of())" + "SqlShowDatabases(Span.of())" + "SqlShowCatalogs(Span.of())" + "SqlShowCurrent(Span.of())" "SqlUseCatalog(Span.of(), null)" "SqlUseDatabase(Span.of(), null)" "SqlSetOptionBeam(Span.of(), null)" diff --git a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl index 74dca414b02b..e88fb67bbce9 100644 --- a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl +++ b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl @@ -267,7 +267,7 @@ SqlDrop SqlDropCatalog(Span s, boolean replace) : /** * SHOW CATALOGS [ LIKE regex_pattern ] */ -SqlCall SqlShowCatalogs(Span s, String scope) : +SqlCall SqlShowCatalogs(Span s) : { SqlNode regex = null; } @@ -275,7 +275,7 @@ SqlCall SqlShowCatalogs(Span s, String scope) : { s.add(this); } [ regex = StringLiteral() ] { - return new SqlShowCatalogs(s.end(this), scope, false, regex); + return new SqlShowCatalogs(s.end(this), false, regex); } } @@ -350,7 +350,7 @@ SqlDrop SqlDropDatabase(Span s, boolean replace) : /** * SHOW DATABASES [ ( FROM | IN )? catalog_name ] [LIKE regex_pattern ] */ -SqlCall SqlShowDatabases(Span s, String scope) : +SqlCall SqlShowDatabases(Span s) : { SqlIdentifier catalogName = null; SqlNode regex = null; @@ -360,22 +360,22 @@ SqlCall SqlShowDatabases(Span s, String scope) : [ ( | ) catalogName = SimpleIdentifier() ] [ regex = StringLiteral() ] { - return new SqlShowDatabases(s.end(this), scope, false, catalogName, regex); + return new SqlShowDatabases(s.end(this), false, catalogName, regex); } } -SqlCall SqlShowCurrent(Span s, String scope) : +SqlCall SqlShowCurrent(Span s) : { } { { s.add(this); } ( { - return new SqlShowCatalogs(s.end(this), scope, true, null); + return new SqlShowCatalogs(s.end(this), true, null); } | { - return new SqlShowDatabases(s.end(this), scope, true, null, null); + return new SqlShowDatabases(s.end(this), true, null, null); } ) } @@ -508,7 +508,7 @@ SqlDrop SqlDropTable(Span s, boolean replace) : /** * SHOW TABLES [ ( FROM | IN )? [ catalog_name '.' ] database_name ] [ LIKE regex_pattern ] */ -SqlCall SqlShowTables(Span s, String scope) : +SqlCall SqlShowTables(Span s) : { SqlIdentifier database = null; SqlNode regex = null; @@ -518,7 +518,7 @@ SqlCall SqlShowTables(Span s, String scope) : [ ( | ) database = CompoundIdentifier() ] [ regex = StringLiteral() ] { - return new SqlShowTables(s.end(this), scope, database, regex); + return new SqlShowTables(s.end(this), database, regex); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java index 2e07f0bda533..08b5ac440332 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java @@ -72,7 +72,7 @@ public Catalog getCatalog() { return getSubSchema(catalog.currentDatabase()); } - public @Nullable Collection databases() { + public Collection databases() { return catalog.databases(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java index 83e8ca7a0520..2045f6973d88 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java @@ -29,26 +29,24 @@ 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.runtime.SqlFunctions; 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.SqlCall; 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.checkerframework.checker.nullness.qual.Nullable; -public class SqlShowCatalogs extends SqlSetOption implements BeamSqlParser.ExecutableStatement { +public class SqlShowCatalogs extends SqlCall implements BeamSqlParser.ExecutableStatement { private static final SqlOperator OPERATOR = - new SqlSpecialOperator("SHOW CATALOGS", SqlKind.OTHER); + new SqlSpecialOperator("SHOW CATALOGS", SqlKind.OTHER_DDL); private final boolean showCurrentOnly; private final @Nullable SqlNode regex; - public SqlShowCatalogs( - SqlParserPos pos, String scope, boolean showCurrentOnly, @Nullable SqlNode regex) { - super(pos, scope, new SqlIdentifier("", pos), null); + public SqlShowCatalogs(SqlParserPos pos, boolean showCurrentOnly, @Nullable SqlNode regex) { + super(pos); this.showCurrentOnly = showCurrentOnly; this.regex = regex; } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java index 24bc47200fed..51e0e6e4c87f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java @@ -29,19 +29,19 @@ 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.runtime.SqlFunctions; 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.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.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.checkerframework.checker.nullness.qual.Nullable; -public class SqlShowDatabases extends SqlSetOption implements BeamSqlParser.ExecutableStatement { +public class SqlShowDatabases extends SqlCall implements BeamSqlParser.ExecutableStatement { private static final SqlOperator OPERATOR = - new SqlSpecialOperator("SHOW DATABASES", SqlKind.OTHER); + new SqlSpecialOperator("SHOW DATABASES", SqlKind.OTHER_DDL); private final boolean showCurrentOnly; private final @Nullable SqlIdentifier catalogName; @@ -49,11 +49,10 @@ public class SqlShowDatabases extends SqlSetOption implements BeamSqlParser.Exec public SqlShowDatabases( SqlParserPos pos, - String scope, boolean showCurrentOnly, @Nullable SqlIdentifier catalogName, @Nullable SqlNode regex) { - super(pos, scope, new SqlIdentifier("", pos), null); + super(pos); this.showCurrentOnly = showCurrentOnly; this.catalogName = catalogName; this.regex = regex; @@ -100,8 +99,7 @@ public void execute(CalcitePrepare.Context context) { print(databases, catalogSchema.getCatalog().name(), SqlDdlNodes.getString(regex)); } - private static void print( - @Nullable Collection databases, String path, @Nullable String pattern) { + private static void print(Collection databases, String path, @Nullable String pattern) { SqlFunctions.LikeFunction calciteLike = new SqlFunctions.LikeFunction(); final String headerName = "Databases in " + path; @@ -109,10 +107,8 @@ private static void print( int nameWidth = headerName.length(); - if (databases != null) { - for (String dbName : databases) { - nameWidth = Math.max(nameWidth, dbName.length()); - } + for (String dbName : databases) { + nameWidth = Math.max(nameWidth, dbName.length()); } nameWidth += 2; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java index b5e6d7a66e08..71cc3bfd681a 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java @@ -32,11 +32,11 @@ 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.runtime.SqlFunctions; 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.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.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; @@ -44,17 +44,15 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; -public class SqlShowTables extends SqlSetOption implements BeamSqlParser.ExecutableStatement { - private static final SqlOperator OPERATOR = new SqlSpecialOperator("SHOW TABLES", SqlKind.OTHER); +public class SqlShowTables extends SqlCall implements BeamSqlParser.ExecutableStatement { + private static final SqlOperator OPERATOR = + new SqlSpecialOperator("SHOW TABLES", SqlKind.OTHER_DDL); private final @Nullable SqlIdentifier databaseName; private final @Nullable SqlNode regex; public SqlShowTables( - SqlParserPos pos, - String scope, - @Nullable SqlIdentifier databaseName, - @Nullable SqlNode regex) { - super(pos, scope, new SqlIdentifier("", pos), null); + SqlParserPos pos, @Nullable SqlIdentifier databaseName, @Nullable SqlNode regex) { + super(pos); this.databaseName = databaseName; this.regex = regex; } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java index b606e39de0d4..158e56fa785f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java @@ -51,7 +51,6 @@ public interface Catalog { @Nullable String currentDatabase(); - @Nullable Collection databases(); /** diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java index 7070cf8dfe61..7c0d8b9d32ea 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java @@ -99,7 +99,7 @@ public void useDatabase(String database) { } @Override - public @Nullable Collection databases() { + public Collection databases() { return databases; } From 7e14d95895acd877d15cf7e5dc6fc63a15e54748 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 11 Nov 2025 23:14:15 +0300 Subject: [PATCH 07/14] spotless --- .../sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java index 4fdbad94a95f..7dee72511e85 100644 --- a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java +++ b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java @@ -26,7 +26,6 @@ import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; 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.ImmutableMap; -import org.checkerframework.checker.nullness.qual.Nullable; public class IcebergCatalog extends InMemoryCatalog { // TODO(ahmedabu98): extend this to the IO implementation so From 653cc40605bc360d60fab97da09649955a92165c Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 20 Nov 2025 11:23:32 -0500 Subject: [PATCH 08/14] minor fixes --- .../src/main/codegen/includes/parserImpls.ftl | 3 +++ .../sql/impl/parser/SqlShowCatalogs.java | 12 ++++++---- .../sql/impl/parser/SqlShowDatabases.java | 14 +++++------ .../sql/impl/parser/SqlShowTables.java | 23 ++++++++----------- .../extensions/sql/BeamSqlCliCatalogTest.java | 16 ++++++------- 5 files changed, 34 insertions(+), 34 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl index e88fb67bbce9..0c7fde925f4a 100644 --- a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl +++ b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl @@ -364,6 +364,9 @@ SqlCall SqlShowDatabases(Span s) : } } +/** + * SHOW CURRENT ( CATALOG | DATABASE ) + */ SqlCall SqlShowCurrent(Span s) : { } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java index 2045f6973d88..4d7f49f213a7 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java @@ -75,7 +75,7 @@ public void execute(CalcitePrepare.Context context) { CatalogManagerSchema managerSchema = ((CatalogManagerSchema) schema); if (showCurrentOnly) { Catalog currentCatalog = managerSchema.getCurrentCatalogSchema().getCatalog(); - System.out.printf("%s (type: %s)", currentCatalog.name(), currentCatalog.type()); + System.out.printf("%s (type: %s)%n", currentCatalog.name(), currentCatalog.type()); return; } Collection catalogs = managerSchema.catalogs(); @@ -94,8 +94,10 @@ private static void print(Collection catalogs, @Nullable String pattern // find the longest string in each column for (Catalog catalog : catalogs) { - nameWidth = Math.max(nameWidth, catalog.name().length()); - typeWidth = Math.max(typeWidth, catalog.type().length()); + if (pattern == null || calciteLike.like(catalog.name(), pattern)) { + nameWidth = Math.max(nameWidth, catalog.name().length()); + typeWidth = Math.max(typeWidth, catalog.type().length()); + } } // add a small padding @@ -109,9 +111,9 @@ private static void print(Collection catalogs, @Nullable String pattern int separatorWidth = nameWidth + typeWidth + 5; String separator = String.format( - "+" + new String(new char[separatorWidth]).replace("\0", separatorChar) + "+%n"); + "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+%n"); - // printing the table + // printing the catalogs System.out.printf(separator); System.out.printf(format, headerName, headerType); System.out.printf(separator); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java index 51e0e6e4c87f..755317dcfe26 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java @@ -108,7 +108,9 @@ private static void print(Collection databases, String path, @Nullable S int nameWidth = headerName.length(); for (String dbName : databases) { - nameWidth = Math.max(nameWidth, dbName.length()); + if (pattern == null || calciteLike.like(dbName, pattern)) { + nameWidth = Math.max(nameWidth, dbName.length()); + } } nameWidth += 2; @@ -117,16 +119,14 @@ private static void print(Collection databases, String path, @Nullable S int separatorWidth = nameWidth + 2; String separator = String.format( - "+" + new String(new char[separatorWidth]).replace("\0", separatorChar) + "+%n"); + "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+%n"); System.out.printf(separator); System.out.printf(format, headerName); System.out.printf(separator); - if (databases != null) { - for (String dbName : databases.stream().sorted().collect(Collectors.toList())) { - if (pattern == null || calciteLike.like(dbName, pattern)) { - System.out.printf(format, dbName); - } + for (String dbName : databases.stream().sorted().collect(Collectors.toList())) { + if (pattern == null || calciteLike.like(dbName, pattern)) { + System.out.printf(format, dbName); } } System.out.printf(separator); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java index 71cc3bfd681a..05ff4bdc2a9f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java @@ -107,8 +107,7 @@ public void execute(CalcitePrepare.Context context) { print(tables, path, SqlDdlNodes.getString(regex)); } - private static void print( - @Nullable Collection
tables, String path, @Nullable String pattern) { + private static void print(Collection
tables, String path, @Nullable String pattern) { SqlFunctions.LikeFunction calciteLike = new SqlFunctions.LikeFunction(); final String headerName = "Tables in " + path; @@ -118,8 +117,8 @@ private static void print( int nameWidth = headerName.length(); int typeWidth = headerType.length(); - if (tables != null) { - for (Table table : tables) { + for (Table table : tables) { + if (pattern == null || calciteLike.like(table.getName(), pattern)) { nameWidth = Math.max(nameWidth, table.getName().length()); typeWidth = Math.max(typeWidth, table.getType().length()); } @@ -132,21 +131,17 @@ private static void print( int separatorWidth = nameWidth + typeWidth + 5; String separator = String.format( - "+" + new String(new char[separatorWidth]).replace("\0", separatorChar) + "+%n"); + "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+%n"); System.out.printf(separator); System.out.printf(format, headerName, headerType); System.out.printf(separator); - if (tables != null) { - for (Table table : - tables.stream() - .sorted(Comparator.comparing(Table::getName)) - .collect(Collectors.toList())) { - if (pattern == null || calciteLike.like(table.getName(), pattern)) { - System.out.printf(format, table.getName(), table.getType()); - } + for (Table table : + tables.stream().sorted(Comparator.comparing(Table::getName)).collect(Collectors.toList())) { + if (pattern == null || calciteLike.like(table.getName(), pattern)) { + System.out.printf(format, table.getName(), table.getType()); } - System.out.printf(separator); } + System.out.printf(separator); } } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java index a0677ac4d40a..6ae34647b52b 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java @@ -370,21 +370,21 @@ public void testShowCatalogs() { @Test public void testShowCatalogsWithPattern() { cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + cli.execute("CREATE CATALOG my_catalog_2 TYPE 'local'"); cli.execute("CREATE CATALOG my_very_long_catalog_name TYPE 'local'"); ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW CATALOGS"); + cli.execute("SHOW CATALOGS LIKE 'my_catalog%'"); @SuppressWarnings("DefaultCharset") String printOutput = outputStreamCaptor.toString().trim(); assertEquals( - "+---------------------------------------+\n" - + "| Catalog Name | Type |\n" - + "+---------------------------------------+\n" - + "| default | local |\n" - + "| my_catalog | local |\n" - + "| my_very_long_catalog_name | local |\n" - + "+---------------------------------------+", + "+--------------------------+\n" + + "| Catalog Name | Type |\n" + + "+--------------------------+\n" + + "| my_catalog | local |\n" + + "| my_catalog_2 | local |\n" + + "+--------------------------+", printOutput); } } From c97da8e5bd30fc78e14edc6d222725fceb60c37a Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 2 Dec 2025 10:49:20 -0500 Subject: [PATCH 09/14] use PrintWriter and flush all at once --- .../sql/impl/parser/SqlShowCatalogs.java | 34 +++++++++++-------- .../sql/impl/parser/SqlShowDatabases.java | 26 ++++++++------ .../sql/impl/parser/SqlShowTables.java | 30 +++++++++------- 3 files changed, 52 insertions(+), 38 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java index 4d7f49f213a7..1d1dc825425d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java @@ -19,6 +19,7 @@ import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; +import java.io.PrintWriter; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -105,26 +106,29 @@ private static void print(Collection catalogs, @Nullable String pattern typeWidth += 2; // format string with calculated widths for left-justification (%-Ns) - String format = "| %-" + nameWidth + "s | %-" + typeWidth + "s |%n"; + String rowFormat = "| %-" + nameWidth + "s | %-" + typeWidth + "s |%n"; // separator width = column widths + padding + separators - corners ('+') int separatorWidth = nameWidth + typeWidth + 5; String separator = - String.format( - "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+%n"); - - // printing the catalogs - System.out.printf(separator); - System.out.printf(format, headerName, headerType); - System.out.printf(separator); - for (Catalog catalog : - catalogs.stream() - .sorted(Comparator.comparing(Catalog::name)) - .collect(Collectors.toList())) { - if (pattern == null || calciteLike.like(catalog.name(), pattern)) { - System.out.printf(format, catalog.name(), catalog.type()); + "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+%n"; + + // print the catalogs + try (PrintWriter writer = new PrintWriter(System.out)) { + writer.println(separator); + writer.printf(rowFormat, headerName, headerType); + writer.println(separator); + for (Catalog catalog : + catalogs.stream() + .sorted(Comparator.comparing(Catalog::name)) + .collect(Collectors.toList())) { + if (pattern == null || calciteLike.like(catalog.name(), pattern)) { + writer.printf(rowFormat, catalog.name(), catalog.type()); + } } + writer.println(separator); + + writer.flush(); } - System.out.printf(separator); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java index 755317dcfe26..0f05593f3f09 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java @@ -19,6 +19,7 @@ import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; +import java.io.PrintWriter; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -114,21 +115,24 @@ private static void print(Collection databases, String path, @Nullable S } nameWidth += 2; - String format = "| %-" + nameWidth + "s |%n"; + String rowFormat = "| %-" + nameWidth + "s |%n"; int separatorWidth = nameWidth + 2; String separator = - String.format( - "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+%n"); - - System.out.printf(separator); - System.out.printf(format, headerName); - System.out.printf(separator); - for (String dbName : databases.stream().sorted().collect(Collectors.toList())) { - if (pattern == null || calciteLike.like(dbName, pattern)) { - System.out.printf(format, dbName); + "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+"; + + try (PrintWriter writer = new PrintWriter(System.out)) { + writer.println(separator); + writer.printf(rowFormat, headerName); + writer.println(separator); + for (String dbName : databases.stream().sorted().collect(Collectors.toList())) { + if (pattern == null || calciteLike.like(dbName, pattern)) { + writer.printf(rowFormat, dbName); + } } + writer.println(separator); + + writer.flush(); } - System.out.printf(separator); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java index 05ff4bdc2a9f..a2ed23f24179 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java @@ -19,6 +19,7 @@ import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; +import java.io.PrintWriter; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -126,22 +127,27 @@ private static void print(Collection
tables, String path, @Nullable Strin nameWidth += 2; typeWidth += 2; - String format = "| %-" + nameWidth + "s | %-" + typeWidth + "s |%n"; + String rowFormat = "| %-" + nameWidth + "s | %-" + typeWidth + "s |%n"; int separatorWidth = nameWidth + typeWidth + 5; String separator = - String.format( - "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+%n"); - - System.out.printf(separator); - System.out.printf(format, headerName, headerType); - System.out.printf(separator); - for (Table table : - tables.stream().sorted(Comparator.comparing(Table::getName)).collect(Collectors.toList())) { - if (pattern == null || calciteLike.like(table.getName(), pattern)) { - System.out.printf(format, table.getName(), table.getType()); + "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+"; + + try (PrintWriter writer = new PrintWriter(System.out)) { + writer.println(separator); + writer.printf(rowFormat, headerName, headerType); + writer.println(separator); + for (Table table : + tables.stream() + .sorted(Comparator.comparing(Table::getName)) + .collect(Collectors.toList())) { + if (pattern == null || calciteLike.like(table.getName(), pattern)) { + writer.printf(rowFormat, table.getName(), table.getType()); + } } + writer.println(separator); + + writer.flush(); } - System.out.printf(separator); } } From 6fde9c9db23f0e549f6dbe8eed6f55532871e3bc Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 2 Dec 2025 11:37:14 -0500 Subject: [PATCH 10/14] correct PrintWriter usage --- .../sql/impl/parser/SqlShowCatalogs.java | 31 ++++++++++--------- .../sql/impl/parser/SqlShowDatabases.java | 25 ++++++++------- .../sql/impl/parser/SqlShowTables.java | 29 ++++++++--------- 3 files changed, 46 insertions(+), 39 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java index 1d1dc825425d..ce1e1ecc06e1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java @@ -17,8 +17,11 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; +import java.io.BufferedWriter; +import java.io.OutputStreamWriter; import java.io.PrintWriter; import java.util.Collection; import java.util.Collections; @@ -114,21 +117,21 @@ private static void print(Collection catalogs, @Nullable String pattern "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+%n"; // print the catalogs - try (PrintWriter writer = new PrintWriter(System.out)) { - writer.println(separator); - writer.printf(rowFormat, headerName, headerType); - writer.println(separator); - for (Catalog catalog : - catalogs.stream() - .sorted(Comparator.comparing(Catalog::name)) - .collect(Collectors.toList())) { - if (pattern == null || calciteLike.like(catalog.name(), pattern)) { - writer.printf(rowFormat, catalog.name(), catalog.type()); - } + PrintWriter writer = + new PrintWriter(new BufferedWriter(new OutputStreamWriter(System.out, UTF_8))); + writer.println(separator); + writer.printf(rowFormat, headerName, headerType); + writer.println(separator); + for (Catalog catalog : + catalogs.stream() + .sorted(Comparator.comparing(Catalog::name)) + .collect(Collectors.toList())) { + if (pattern == null || calciteLike.like(catalog.name(), pattern)) { + writer.printf(rowFormat, catalog.name(), catalog.type()); } - writer.println(separator); - - writer.flush(); } + writer.println(separator); + + writer.flush(); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java index 0f05593f3f09..9cd86129204a 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java @@ -17,8 +17,11 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; +import java.io.BufferedWriter; +import java.io.OutputStreamWriter; import java.io.PrintWriter; import java.util.Collection; import java.util.Collections; @@ -121,18 +124,18 @@ private static void print(Collection databases, String path, @Nullable S String separator = "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+"; - try (PrintWriter writer = new PrintWriter(System.out)) { - writer.println(separator); - writer.printf(rowFormat, headerName); - writer.println(separator); - for (String dbName : databases.stream().sorted().collect(Collectors.toList())) { - if (pattern == null || calciteLike.like(dbName, pattern)) { - writer.printf(rowFormat, dbName); - } + PrintWriter writer = + new PrintWriter(new BufferedWriter(new OutputStreamWriter(System.out, UTF_8))); + writer.println(separator); + writer.printf(rowFormat, headerName); + writer.println(separator); + for (String dbName : databases.stream().sorted().collect(Collectors.toList())) { + if (pattern == null || calciteLike.like(dbName, pattern)) { + writer.printf(rowFormat, dbName); } - writer.println(separator); - - writer.flush(); } + writer.println(separator); + + writer.flush(); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java index a2ed23f24179..0f5b6165ec1d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java @@ -17,8 +17,11 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; +import java.io.BufferedWriter; +import java.io.OutputStreamWriter; import java.io.PrintWriter; import java.util.Collection; import java.util.Collections; @@ -133,21 +136,19 @@ private static void print(Collection
tables, String path, @Nullable Strin String separator = "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+"; - try (PrintWriter writer = new PrintWriter(System.out)) { - writer.println(separator); - writer.printf(rowFormat, headerName, headerType); - writer.println(separator); - for (Table table : - tables.stream() - .sorted(Comparator.comparing(Table::getName)) - .collect(Collectors.toList())) { - if (pattern == null || calciteLike.like(table.getName(), pattern)) { - writer.printf(rowFormat, table.getName(), table.getType()); - } + PrintWriter writer = + new PrintWriter(new BufferedWriter(new OutputStreamWriter(System.out, UTF_8))); + writer.println(separator); + writer.printf(rowFormat, headerName, headerType); + writer.println(separator); + for (Table table : + tables.stream().sorted(Comparator.comparing(Table::getName)).collect(Collectors.toList())) { + if (pattern == null || calciteLike.like(table.getName(), pattern)) { + writer.printf(rowFormat, table.getName(), table.getType()); } - writer.println(separator); - - writer.flush(); } + writer.println(separator); + + writer.flush(); } } From bb2baa998b54577e5b5d380db11e8be545415009 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 8 Dec 2025 14:05:43 -0500 Subject: [PATCH 11/14] cleanup --- .../sql/jdbc/BeamSqlLineShowTests.java | 303 ++++++++++++++++++ .../sql/src/main/codegen/config.fmpp | 3 - .../src/main/codegen/includes/parserImpls.ftl | 135 +++++++- .../sql/impl/BeamSystemDbMetadataSchema.java | 111 +++++++ .../extensions/sql/impl/BeamSystemSchema.java | 155 +++++++++ .../impl/BeamSystemTableMetadataSchema.java | 126 ++++++++ .../sql/impl/CatalogManagerSchema.java | 5 + .../extensions/sql/impl/CatalogSchema.java | 4 - .../sql/impl/parser/SqlDdlNodes.java | 7 - .../sql/impl/parser/SqlShowCatalogs.java | 137 -------- .../sql/impl/parser/SqlShowDatabases.java | 141 -------- .../sql/impl/parser/SqlShowTables.java | 154 --------- .../sdk/extensions/sql/meta/SystemTables.java | 182 +++++++++++ .../extensions/sql/meta/catalog/Catalog.java | 1 + .../extensions/sql/BeamSqlCliCatalogTest.java | 57 ---- .../sql/BeamSqlCliDatabaseTest.java | 108 ------- .../sdk/extensions/sql/BeamSqlCliTest.java | 99 ------ 17 files changed, 1011 insertions(+), 717 deletions(-) create mode 100644 sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineShowTests.java create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemDbMetadataSchema.java create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemSchema.java create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemTableMetadataSchema.java delete mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java delete mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java delete mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/SystemTables.java diff --git a/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineShowTests.java b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineShowTests.java new file mode 100644 index 000000000000..9511f52ed534 --- /dev/null +++ b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineShowTests.java @@ -0,0 +1,303 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.sql.jdbc; + +import static org.apache.beam.sdk.extensions.sql.jdbc.BeamSqlLineTestingUtils.buildArgs; +import static org.hamcrest.CoreMatchers.everyItem; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.oneOf; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import org.junit.Test; + +public class BeamSqlLineShowTests { + @Test + public void testShowTables() throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + String[] args = + buildArgs( + "CREATE DATABASE other_db", + "CREATE EXTERNAL TABLE other_db.should_not_show_up (id int, name varchar) TYPE 'text'", + "CREATE CATALOG my_catalog TYPE 'local'", + "CREATE DATABASE my_catalog.my_db", + "USE DATABASE my_catalog.my_db", + "CREATE EXTERNAL TABLE my_table (id int, name varchar) TYPE 'text'", + "CREATE EXTERNAL TABLE my_other_table (col1 int, col2 timestamp) TYPE 'text'", + "CREATE EXTERNAL TABLE my_other_table_with_a_long_name (foo varchar, bar boolean) TYPE 'test'", + "SHOW TABLES"); + + BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); + + List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + assertThat( + Arrays.asList( + "+------+------+", + "| NAME | TYPE |", + "+------+------+", + "| my_other_table | text |", + "| my_other_table_with_a_long_name | test |", + "| my_table | text |", + "+------+------+"), + everyItem(is(oneOf(lines.toArray())))); + } + + @Test + public void testShowTablesInOtherDatabase() throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + String[] args = + buildArgs( + "CREATE DATABASE my_db", + "USE DATABASE my_db", + "CREATE EXTERNAL TABLE should_not_show_up (id int, name varchar) TYPE 'text'", + "CREATE CATALOG other_catalog TYPE 'local'", + "CREATE DATABASE other_catalog.other_db", + "CREATE EXTERNAL TABLE other_catalog.other_db.other_table (id int, name varchar) TYPE 'text'", + "SHOW TABLES IN other_catalog.other_db"); + + BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); + + List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + assertThat( + Arrays.asList( + "+------+------+", + "| NAME | TYPE |", + "+------+------+", + "| other_table | text |", + "+------+------+"), + everyItem(is(oneOf(lines.toArray())))); + } + + @Test + public void testShowTablesWithPattern() throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + String[] args = + buildArgs( + "CREATE DATABASE my_db", + "USE DATABASE my_db", + "CREATE EXTERNAL TABLE my_table (id int, name varchar) TYPE 'text'", + "CREATE EXTERNAL TABLE my_table_2 (id int, name varchar) TYPE 'text'", + "CREATE EXTERNAL TABLE my_foo_table_1 (id int, name varchar) TYPE 'text'", + "CREATE EXTERNAL TABLE my_foo_table_2 (id int, name varchar) TYPE 'text'", + "SHOW TABLES LIKE '%foo%'"); + + BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); + + List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + assertThat( + Arrays.asList( + "+------+------+", + "| NAME | TYPE |", + "+------+------+", + "| my_foo_table_1 | text |", + "| my_foo_table_2 | text |", + "+------+------+"), + everyItem(is(oneOf(lines.toArray())))); + } + + @Test + public void testShowCurrentDatabase() throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + String[] args = + buildArgs( + "CREATE DATABASE should_not_show_up", + "CREATE CATALOG my_catalog TYPE 'local'", + "USE CATALOG my_catalog", + "CREATE DATABASE my_db", + "CREATE DATABASE my_other_db", + "CREATE DATABASE my_database_that_has_a_very_long_name", + "USE DATABASE my_other_db", + "SHOW CURRENT database"); + + BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); + + List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + assertThat( + Arrays.asList("+------+", "| NAME |", "+------+", "| my_other_db |", "+------+"), + everyItem(is(oneOf(lines.toArray())))); + } + + @Test + public void testShowCurrentDatabaseWithNoneSet() throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + String[] args = + buildArgs( + "CREATE DATABASE should_not_show_up", + "CREATE CATALOG my_catalog TYPE 'local'", + "USE CATALOG my_catalog", + "DROP DATABASE `default`", + "SHOW CURRENT DATABASE"); + + BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); + + List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + assertThat( + Arrays.asList("+------+", "| NAME |", "+------+", "+------+"), + everyItem(is(oneOf(lines.toArray())))); + } + + @Test + public void testShowDatabases() throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + String[] args = + buildArgs( + "CREATE DATABASE should_not_show_up", + "CREATE CATALOG my_catalog TYPE 'local'", + "USE CATALOG my_catalog", + "CREATE DATABASE my_db", + "CREATE DATABASE my_other_db", + "CREATE DATABASE my_database_that_has_a_very_long_name", + "SHOW DATABASES"); + + BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); + + List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + assertThat( + Arrays.asList( + "+------+", + "| NAME |", + "+------+", + "| default |", + "| my_database_that_has_a_very_long_name |", + "| my_db |", + "| my_other_db |", + "+------+"), + everyItem(is(oneOf(lines.toArray())))); + } + + @Test + public void testShowDatabasesInOtherCatalog() throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + String[] args = + buildArgs( + "CREATE DATABASE should_not_show_up", + "CREATE CATALOG my_catalog TYPE 'local'", + "USE CATALOG my_catalog", + "CREATE DATABASE my_db", + "CREATE CATALOG my_other_catalog TYPE 'local'", + "CREATE DATABASE my_other_catalog.other_db", + "SHOW DATABASES FROM my_other_catalog"); + + BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); + + List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + assertThat( + Arrays.asList( + "+------+", "| NAME |", "+------+", "| default |", "| other_db |", "+------+"), + everyItem(is(oneOf(lines.toArray())))); + } + + @Test + public void testShowDatabasesWithPattern() throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + String[] args = + buildArgs( + "CREATE CATALOG my_catalog TYPE 'local'", + "CREATE DATABASE my_catalog.my_db", + "CREATE DATABASE my_catalog.other_db", + "CREATE DATABASE my_catalog.some_foo_db", + "CREATE DATABASE my_catalog.some_other_foo_db", + "SHOW DATABASES FROM my_catalog LIKE '%foo%'"); + + BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); + + List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + assertThat( + Arrays.asList( + "+------+", + "| NAME |", + "+------+", + "| some_foo_db |", + "| some_other_foo_db |", + "+------+"), + everyItem(is(oneOf(lines.toArray())))); + } + + @Test + public void testShowCurrentCatalog() throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + String[] args = + buildArgs( + "CREATE CATALOG my_catalog TYPE 'local'", + "CREATE CATALOG my_very_long_catalog_name TYPE 'local'", + "SHOW CURRENT CATALOG"); + + BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); + + List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + assertThat( + Arrays.asList( + "+------+------+", + "| NAME | TYPE |", + "+------+------+", + "| default | local |", + "+------+------+"), + everyItem(is(oneOf(lines.toArray())))); + } + + @Test + public void testShowCatalogs() throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + String[] args = + buildArgs( + "CREATE CATALOG my_catalog TYPE 'local'", + "CREATE CATALOG my_very_long_catalog_name TYPE 'local'", + "SHOW CATALOGS"); + + BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); + + List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + assertThat( + Arrays.asList( + "+------+------+", + "| NAME | TYPE |", + "+------+------+", + "| default | local |", + "| my_catalog | local |", + "| my_very_long_catalog_name | local |", + "+------+------+"), + everyItem(is(oneOf(lines.toArray())))); + } + + @Test + public void testShowCatalogsWithPattern() throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + String[] args = + buildArgs( + "CREATE CATALOG my_catalog TYPE 'local'", + "CREATE CATALOG my_catalog_2 TYPE 'local'", + "CREATE CATALOG my_very_long_catalog_name TYPE 'local'", + "SHOW CATALOGS LIKE 'my_catalog%'"); + + BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); + + List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + assertThat( + Arrays.asList( + "+------+------+", + "| NAME | TYPE |", + "+------+------+", + "| my_catalog | local |", + "| my_catalog_2 | local |", + "+------+------+"), + everyItem(is(oneOf(lines.toArray())))); + } +} diff --git a/sdks/java/extensions/sql/src/main/codegen/config.fmpp b/sdks/java/extensions/sql/src/main/codegen/config.fmpp index ef067356f8c5..73af7e18150b 100644 --- a/sdks/java/extensions/sql/src/main/codegen/config.fmpp +++ b/sdks/java/extensions/sql/src/main/codegen/config.fmpp @@ -35,9 +35,6 @@ data: { "org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes" "org.apache.beam.sdk.extensions.sql.impl.parser.SqlUseCatalog" "org.apache.beam.sdk.extensions.sql.impl.parser.SqlUseDatabase" - "org.apache.beam.sdk.extensions.sql.impl.parser.SqlShowCatalogs" - "org.apache.beam.sdk.extensions.sql.impl.parser.SqlShowDatabases" - "org.apache.beam.sdk.extensions.sql.impl.parser.SqlShowTables" "org.apache.beam.sdk.extensions.sql.impl.parser.SqlSetOptionBeam" "org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils" "org.apache.beam.sdk.schemas.Schema" diff --git a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl index 0c7fde925f4a..d3bb8c2af56c 100644 --- a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl +++ b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl @@ -275,7 +275,32 @@ SqlCall SqlShowCatalogs(Span s) : { s.add(this); } [ regex = StringLiteral() ] { - return new SqlShowCatalogs(s.end(this), false, regex); + List path = new ArrayList(); + path.add("beamsystem"); + path.add("catalogs"); + SqlNodeList selectList = SqlNodeList.of(SqlIdentifier.star(s.end(this))); + SqlIdentifier from = new SqlIdentifier(path, s.end(this)); + SqlNode where = null; + if (regex != null) { + SqlIdentifier nameIdentifier = new SqlIdentifier("NAME", s.end(this)); + where = SqlStdOperatorTable.LIKE.createCall( + s.end(this), + nameIdentifier, regex); + } + + return new SqlSelect( + s.end(this), + null, + selectList, + from, + where, + null, + null, + null, + null, + null, + null, + null); } } @@ -360,7 +385,37 @@ SqlCall SqlShowDatabases(Span s) : [ ( | ) catalogName = SimpleIdentifier() ] [ regex = StringLiteral() ] { - return new SqlShowDatabases(s.end(this), false, catalogName, regex); + List path = new ArrayList(); + path.add("beamsystem"); + path.add("databases"); + SqlNodeList selectList = SqlNodeList.of(SqlIdentifier.star(s.end(this))); + SqlNode where = null; + if (regex != null) { + SqlIdentifier nameIdentifier = new SqlIdentifier("NAME", s.end(this)); + where = SqlStdOperatorTable.LIKE.createCall( + s.end(this), + nameIdentifier, regex); + } + if (catalogName != null) { + path.add(catalogName.getSimple()); + } else { + path.add("__current_catalog__"); + } + SqlIdentifier from = new SqlIdentifier(path, s.end(this)); + + return new SqlSelect( + s.end(this), + null, + selectList, + from, + where, + null, + null, + null, + null, + null, + null, + null); } } @@ -372,15 +427,41 @@ SqlCall SqlShowCurrent(Span s) : } { { s.add(this); } + { + List path = new ArrayList(); + path.add("beamsystem"); + } ( { - return new SqlShowCatalogs(s.end(this), true, null); + path.add("__current_catalog__"); } | { - return new SqlShowDatabases(s.end(this), true, null, null); + path.add("__current_database__"); } ) + { + if (path.size() != 2) { + throw new ParseException( + "Expected SHOW CURRENT CATALOG or SHOW CURRENT DATABASE"); + } + SqlNodeList selectList = SqlNodeList.of(SqlIdentifier.star(s.end(this))); + SqlIdentifier from = new SqlIdentifier(path, s.end(this)); + + return new SqlSelect( + s.end(this), + null, + selectList, + from, + null, + null, + null, + null, + null, + null, + null, + null); + } } @@ -513,15 +594,55 @@ SqlDrop SqlDropTable(Span s, boolean replace) : */ SqlCall SqlShowTables(Span s) : { - SqlIdentifier database = null; + SqlIdentifier databaseCatalog = null; SqlNode regex = null; } { { s.add(this); } - [ ( | ) database = CompoundIdentifier() ] + [ ( | ) databaseCatalog = CompoundIdentifier() ] [ regex = StringLiteral() ] { - return new SqlShowTables(s.end(this), database, regex); + List path = new ArrayList(); + path.add("beamsystem"); + path.add("tables"); + SqlNodeList selectList = SqlNodeList.of(SqlIdentifier.star(s.end(this))); + SqlNode where = null; + if (regex != null) { + SqlIdentifier nameIdentifier = new SqlIdentifier("NAME", s.end(this)); + where = SqlStdOperatorTable.LIKE.createCall( + s.end(this), + nameIdentifier, regex); + } + if (databaseCatalog != null) { + List components = databaseCatalog.names; + if (components.size() == 1) { + path.add("__current_catalog__"); + path.add(components.get(0)); + } else if (components.size() == 2) { + path.addAll(components); + } else { + throw new ParseException( + "SHOW TABLES FROM/IN accepts at most a catalog name and a database name."); + } + } else { + path.add("__current_catalog__"); + path.add("__current_database__"); + } + SqlIdentifier from = new SqlIdentifier(path, s.end(this)); + + return new SqlSelect( + s.end(this), + null, + selectList, + from, + where, + null, + null, + null, + null, + null, + null, + null); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemDbMetadataSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemDbMetadataSchema.java new file mode 100644 index 000000000000..66c05a35313e --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemDbMetadataSchema.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.sql.impl; + +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.util.Collection; +import java.util.Collections; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.sql.meta.SystemTables; +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_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.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Table; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** A Calcite {@link Schema} responsible for {@code SHOW DATABASES} requests. */ +public class BeamSystemDbMetadataSchema implements Schema { + private final CatalogManager catalogManager; + + BeamSystemDbMetadataSchema(CatalogManager catalogManager) { + this.catalogManager = catalogManager; + } + + @Override + public @Nullable Table getTable(String catalogName) { + Catalog catalog; + if (catalogName.equals("__current_catalog__")) { + catalog = catalogManager.currentCatalog(); + } else { + catalog = + checkArgumentNotNull( + catalogManager.getCatalog(catalogName), "Catalog '%s' does not exist.", catalogName); + } + + return BeamCalciteTable.of(SystemTables.databases(catalog, false)); + } + + @Override + public Set getTableNames() { + return catalogManager.catalogs().stream().map(Catalog::name).collect(Collectors.toSet()); + } + + @Override + public @Nullable Schema getSubSchema(@Nullable String name) { + return null; + } + + @Override + public Set getSubSchemaNames() { + return Collections.emptySet(); + } + + @Override + public Set getTypeNames() { + return Collections.emptySet(); + } + + @Override + public @Nullable RelProtoDataType getType(String s) { + return null; + } + + @Override + public Collection getFunctions(String s) { + return Collections.emptySet(); + } + + @Override + public Set getFunctionNames() { + return Collections.emptySet(); + } + + @Override + public Expression getExpression(@Nullable SchemaPlus schemaPlus, String s) { + return Schemas.subSchemaExpression(checkStateNotNull(schemaPlus), s, getClass()); + } + + @Override + public boolean isMutable() { + return true; + } + + @Override + public Schema snapshot(SchemaVersion schemaVersion) { + return this; + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemSchema.java new file mode 100644 index 000000000000..c9f7c417ca94 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemSchema.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.sql.impl; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.util.Collection; +import java.util.Collections; +import java.util.Set; +import org.apache.beam.sdk.extensions.sql.meta.SystemTables; +import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +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.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.ImmutableSet; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A Calcite {@link Schema} specialized for displaying the session's metadata. Top node that manages + * requests to {@code SHOW} {@code CATALOGS}, {@code DATABASES}, and {@code TABLES}. Used by {@link + * CatalogManagerSchema}. + * + *

{@code SHOW} requests are treated as aliases, listed below: + * + *

    + *
  • {@code SHOW CURRENT CATALOG} --> {@code SELECT * FROM `beamsystem`.`__current_catalog__`} + *
  • {@code SHOW CATALOGS} --> {@code SELECT * FROM `beamsystem`.`catalogs`} + *
  • {@code SHOW CATALOGS LIKE '{pattern}'} --> {@code SELECT * FROM `beamsystem`.`catalogs` + * WHERE NAME LIKE '{pattern}'} + *
  • {@code SHOW CURRENT DATABASE} --> {@code SELECT * FROM `beamsystem`.`__current_database__`} + *
  • {@code SHOW DATABASES} --> {@code SELECT * FROM + * `beamsystem`.`databases`.`__current_catalog__`} + *
  • {@code SHOW DATABASES FROM my_catalog} --> {@code SELECT * FROM + * `beamsystem`.`databases`.`my_catalog`} + *
  • {@code SHOW DATABASES FROM my_catalog LIKE '{pattern}'} --> {@code SELECT * FROM + * `beamsystem`.`databases`.`my_catalog` WHERE NAME LIKE '{pattern}'} + *
  • {@code SHOW TABLES} --> {@code SELECT * FROM + * `beamsystem`.`tables`.`__current_catalog__`.`__current_database__`} + *
  • {@code SHOW TABLES FROM my_db} --> {@code SELECT * FROM + * `beamsystem`.`tables`.`__current_catalog__`.`my_db`} + *
  • {@code SHOW TABLES FROM my_catalog.my_db} --> {@code SELECT * FROM + * `beamsystem`.`tables`.`my_catalog`.`my_db`} + *
  • {@code SHOW TABLES FROM my_catalog.my_db LIKE '{pattern}'} --> {@code SELECT * FROM + * `beamsystem`.`tables`.`my_catalog`.`my_db` WHERE NAME LIKE '{pattern}'} + *
+ */ +public class BeamSystemSchema implements Schema { + private final CatalogManager catalogManager; + private final BeamSystemDbMetadataSchema dbSchema; + private final BeamSystemTableMetadataSchema tableSchema; + public static final String BEAMSYSTEM = "beamsystem"; + private static final String CATALOGS = "catalogs"; + private static final String DATABASES = "databases"; + private static final String TABLES = "tables"; + + BeamSystemSchema(CatalogManager catalogManager) { + this.catalogManager = catalogManager; + this.dbSchema = new BeamSystemDbMetadataSchema(catalogManager); + this.tableSchema = new BeamSystemTableMetadataSchema(catalogManager, null); + } + + @Override + public @Nullable Table getTable(String table) { + switch (table) { + case CATALOGS: + return BeamCalciteTable.of(SystemTables.catalogs(catalogManager, false)); + case "__current_catalog__": + return BeamCalciteTable.of(SystemTables.catalogs(catalogManager, true)); + case "__current_database__": + return BeamCalciteTable.of(SystemTables.databases(catalogManager.currentCatalog(), true)); + default: + return null; + } + } + + @Override + public Set getTableNames() { + return ImmutableSet.of(CATALOGS); + } + + @Override + public @Nullable Schema getSubSchema(@Nullable String name) { + if (name == null) { + return null; + } + switch (name) { + case DATABASES: + return dbSchema; + case TABLES: + return tableSchema; + default: + return null; + } + } + + @Override + public Set getSubSchemaNames() { + return ImmutableSet.of(DATABASES, TABLES); + } + + @Override + public Set getTypeNames() { + return Collections.emptySet(); + } + + @Override + public @Nullable RelProtoDataType getType(String s) { + return null; + } + + @Override + public Collection getFunctions(String s) { + return Collections.emptySet(); + } + + @Override + public Set getFunctionNames() { + return Collections.emptySet(); + } + + @Override + public Expression getExpression(@Nullable SchemaPlus schemaPlus, String s) { + return Schemas.subSchemaExpression(checkStateNotNull(schemaPlus), s, getClass()); + } + + @Override + public boolean isMutable() { + return true; + } + + @Override + public Schema snapshot(SchemaVersion schemaVersion) { + return this; + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemTableMetadataSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemTableMetadataSchema.java new file mode 100644 index 000000000000..b081a1b886c3 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSystemTableMetadataSchema.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.sql.impl; + +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.util.Collection; +import java.util.Collections; +import java.util.Set; +import org.apache.beam.sdk.extensions.sql.meta.SystemTables; +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_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.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Table; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** A Calcite {@link Schema} responsible for {@code SHOW TABLES} requests. */ +public class BeamSystemTableMetadataSchema implements Schema { + private final CatalogManager catalogManager; + private final @MonotonicNonNull String catalog; + + BeamSystemTableMetadataSchema(CatalogManager catalogManager, @Nullable String catalog) { + this.catalogManager = catalogManager; + this.catalog = catalog; + } + + @Override + public @Nullable Table getTable(String dbName) { + // returns a table if this instance has a catalog referenced + if (catalog == null) { + return null; + } + + Catalog cat = + checkArgumentNotNull( + catalogManager.getCatalog(catalog), "Catalog '%s' does not exist.", catalog); + if (dbName.equals("__current_database__")) { + dbName = + checkStateNotNull( + cat.currentDatabase(), + "Catalog '%s' has not set a default database. Please specify one."); + } + return BeamCalciteTable.of(SystemTables.tables(cat, dbName)); + } + + @Override + public Set getTableNames() { + return Collections.emptySet(); + } + + @Override + public @Nullable Schema getSubSchema(@Nullable String catalogName) { + // if this is a top instance (i.e. no catalog reference), return child schema with the specified + // catalog referenced + if (catalog == null && catalogName != null) { + if (catalogName.equals("__current_catalog__")) { + catalogName = catalogManager.currentCatalog().name(); + } + return new BeamSystemTableMetadataSchema(catalogManager, catalogName); + } + return null; + } + + @Override + public Set getSubSchemaNames() { + return Collections.emptySet(); + } + + @Override + public Set getTypeNames() { + return Collections.emptySet(); + } + + @Override + public @Nullable RelProtoDataType getType(String s) { + return null; + } + + @Override + public Collection getFunctions(String s) { + return Collections.emptySet(); + } + + @Override + public Set getFunctionNames() { + return Collections.emptySet(); + } + + @Override + public Expression getExpression(@Nullable SchemaPlus schemaPlus, String s) { + return Schemas.subSchemaExpression(checkStateNotNull(schemaPlus), s, getClass()); + } + + @Override + public boolean isMutable() { + return true; + } + + @Override + public Schema snapshot(SchemaVersion schemaVersion) { + return this; + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java index 480a5a99a70b..5593331e21df 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java @@ -56,11 +56,13 @@ public class CatalogManagerSchema implements Schema { private static final Logger LOG = LoggerFactory.getLogger(CatalogManagerSchema.class); private final JdbcConnection connection; private final CatalogManager catalogManager; + private final BeamSystemSchema beamSystemSchema; private final Map catalogSubSchemas = new HashMap<>(); CatalogManagerSchema(JdbcConnection jdbcConnection, CatalogManager catalogManager) { this.connection = jdbcConnection; this.catalogManager = catalogManager; + this.beamSystemSchema = new BeamSystemSchema(catalogManager); } @VisibleForTesting @@ -206,6 +208,9 @@ public CatalogSchema getCurrentCatalogSchema() { if (name == null) { return null; } + if (name.equals(BeamSystemSchema.BEAMSYSTEM)) { + return beamSystemSchema; + } @Nullable CatalogSchema catalogSchema = catalogSubSchemas.get(name); if (catalogSchema == null) { @Nullable Catalog catalog = catalogManager.getCatalog(name); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java index 08b5ac440332..792e5b98bcd3 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java @@ -72,10 +72,6 @@ public Catalog getCatalog() { return getSubSchema(catalog.currentDatabase()); } - public Collection databases() { - return catalog.databases(); - } - public BeamCalciteSchema getDatabaseSchema(TableName tablePath) { @Nullable BeamCalciteSchema beamCalciteSchema = getSubSchema(tablePath.database()); if (beamCalciteSchema == null) { 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 44b6d092378b..6f4d8ee79d9c 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 @@ -66,13 +66,6 @@ static Pair schema( return Pair.of(checkStateNotNull(schema, "Got null sub-schema for path '%s'", path), name(id)); } - static CalciteSchema schema(CalcitePrepare.Context context, boolean mutable) { - CalciteSchema rootSchema = mutable ? context.getMutableRootSchema() : context.getRootSchema(); - List path = context.getDefaultSchemaPath(); - @Nullable CalciteSchema schema = childSchema(rootSchema, path); - return checkStateNotNull(schema, "Got null sub-schema for path '%s'", path); - } - private static @Nullable CalciteSchema childSchema(CalciteSchema rootSchema, List path) { @Nullable CalciteSchema schema = rootSchema; for (String p : path) { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java deleted file mode 100644 index ce1e1ecc06e1..000000000000 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowCatalogs.java +++ /dev/null @@ -1,137 +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. - */ -package org.apache.beam.sdk.extensions.sql.impl.parser; - -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; - -import java.io.BufferedWriter; -import java.io.OutputStreamWriter; -import java.io.PrintWriter; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; -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.runtime.SqlFunctions; -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.SqlCall; -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.parser.SqlParserPos; -import org.checkerframework.checker.nullness.qual.Nullable; - -public class SqlShowCatalogs extends SqlCall implements BeamSqlParser.ExecutableStatement { - private static final SqlOperator OPERATOR = - new SqlSpecialOperator("SHOW CATALOGS", SqlKind.OTHER_DDL); - - private final boolean showCurrentOnly; - private final @Nullable SqlNode regex; - - public SqlShowCatalogs(SqlParserPos pos, boolean showCurrentOnly, @Nullable SqlNode regex) { - super(pos); - this.showCurrentOnly = showCurrentOnly; - this.regex = regex; - } - - @Override - public List getOperandList() { - return Collections.emptyList(); - } - - @Override - public SqlOperator getOperator() { - return OPERATOR; - } - - @Override - public void execute(CalcitePrepare.Context context) { - Schema schema = SqlDdlNodes.schema(context, true).schema; - - if (!(schema instanceof CatalogManagerSchema)) { - throw SqlUtil.newContextException( - pos, - RESOURCE.internal( - "Attempting execute 'SHOW CATALOGS' with unexpected Calcite Schema of type " - + schema.getClass())); - } - CatalogManagerSchema managerSchema = ((CatalogManagerSchema) schema); - if (showCurrentOnly) { - Catalog currentCatalog = managerSchema.getCurrentCatalogSchema().getCatalog(); - System.out.printf("%s (type: %s)%n", currentCatalog.name(), currentCatalog.type()); - return; - } - Collection catalogs = managerSchema.catalogs(); - print(catalogs, SqlDdlNodes.getString(regex)); - } - - private static void print(Collection catalogs, @Nullable String pattern) { - SqlFunctions.LikeFunction calciteLike = new SqlFunctions.LikeFunction(); - - final String headerName = "Catalog Name"; - final String headerType = "Type"; - final String separatorChar = "-"; - - int nameWidth = headerName.length(); - int typeWidth = headerType.length(); - - // find the longest string in each column - for (Catalog catalog : catalogs) { - if (pattern == null || calciteLike.like(catalog.name(), pattern)) { - nameWidth = Math.max(nameWidth, catalog.name().length()); - typeWidth = Math.max(typeWidth, catalog.type().length()); - } - } - - // add a small padding - nameWidth += 2; - typeWidth += 2; - - // format string with calculated widths for left-justification (%-Ns) - String rowFormat = "| %-" + nameWidth + "s | %-" + typeWidth + "s |%n"; - - // separator width = column widths + padding + separators - corners ('+') - int separatorWidth = nameWidth + typeWidth + 5; - String separator = - "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+%n"; - - // print the catalogs - PrintWriter writer = - new PrintWriter(new BufferedWriter(new OutputStreamWriter(System.out, UTF_8))); - writer.println(separator); - writer.printf(rowFormat, headerName, headerType); - writer.println(separator); - for (Catalog catalog : - catalogs.stream() - .sorted(Comparator.comparing(Catalog::name)) - .collect(Collectors.toList())) { - if (pattern == null || calciteLike.like(catalog.name(), pattern)) { - writer.printf(rowFormat, catalog.name(), catalog.type()); - } - } - writer.println(separator); - - writer.flush(); - } -} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java deleted file mode 100644 index 9cd86129204a..000000000000 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowDatabases.java +++ /dev/null @@ -1,141 +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. - */ -package org.apache.beam.sdk.extensions.sql.impl.parser; - -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; - -import java.io.BufferedWriter; -import java.io.OutputStreamWriter; -import java.io.PrintWriter; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; -import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; -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.runtime.SqlFunctions; -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.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.SqlUtil; -import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.checkerframework.checker.nullness.qual.Nullable; - -public class SqlShowDatabases extends SqlCall implements BeamSqlParser.ExecutableStatement { - private static final SqlOperator OPERATOR = - new SqlSpecialOperator("SHOW DATABASES", SqlKind.OTHER_DDL); - - private final boolean showCurrentOnly; - private final @Nullable SqlIdentifier catalogName; - private final @Nullable SqlNode regex; - - public SqlShowDatabases( - SqlParserPos pos, - boolean showCurrentOnly, - @Nullable SqlIdentifier catalogName, - @Nullable SqlNode regex) { - super(pos); - this.showCurrentOnly = showCurrentOnly; - this.catalogName = catalogName; - this.regex = regex; - } - - @Override - public List getOperandList() { - return Collections.emptyList(); - } - - @Override - public SqlOperator getOperator() { - return OPERATOR; - } - - @Override - public void execute(CalcitePrepare.Context context) { - Schema schema = SqlDdlNodes.schema(context, true).schema; - - if (!(schema instanceof CatalogManagerSchema)) { - throw SqlUtil.newContextException( - pos, - RESOURCE.internal( - "Attempting to execute 'SHOW DATABASES' with unexpected Calcite Schema of type " - + schema.getClass())); - } - - CatalogSchema catalogSchema; - if (catalogName != null) { - String catalog = catalogName.toString(); - catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(catalog); - } else { - catalogSchema = ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); - } - - if (showCurrentOnly) { - @Nullable BeamCalciteSchema currentDatabase = catalogSchema.getCurrentDatabaseSchema(); - String output = - currentDatabase == null ? "No database is currently set" : currentDatabase.name(); - System.out.println(output); - return; - } - Collection databases = catalogSchema.databases(); - print(databases, catalogSchema.getCatalog().name(), SqlDdlNodes.getString(regex)); - } - - private static void print(Collection databases, String path, @Nullable String pattern) { - SqlFunctions.LikeFunction calciteLike = new SqlFunctions.LikeFunction(); - - final String headerName = "Databases in " + path; - final String separatorChar = "-"; - - int nameWidth = headerName.length(); - - for (String dbName : databases) { - if (pattern == null || calciteLike.like(dbName, pattern)) { - nameWidth = Math.max(nameWidth, dbName.length()); - } - } - - nameWidth += 2; - String rowFormat = "| %-" + nameWidth + "s |%n"; - - int separatorWidth = nameWidth + 2; - String separator = - "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+"; - - PrintWriter writer = - new PrintWriter(new BufferedWriter(new OutputStreamWriter(System.out, UTF_8))); - writer.println(separator); - writer.printf(rowFormat, headerName); - writer.println(separator); - for (String dbName : databases.stream().sorted().collect(Collectors.toList())) { - if (pattern == null || calciteLike.like(dbName, pattern)) { - writer.printf(rowFormat, dbName); - } - } - writer.println(separator); - - writer.flush(); - } -} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java deleted file mode 100644 index 0f5b6165ec1d..000000000000 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlShowTables.java +++ /dev/null @@ -1,154 +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. - */ -package org.apache.beam.sdk.extensions.sql.impl.parser; - -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; - -import java.io.BufferedWriter; -import java.io.OutputStreamWriter; -import java.io.PrintWriter; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; -import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; -import org.apache.beam.sdk.extensions.sql.impl.TableName; -import org.apache.beam.sdk.extensions.sql.meta.Table; -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.runtime.SqlFunctions; -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.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.SqlUtil; -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.base.Splitter; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.checkerframework.checker.nullness.qual.Nullable; - -public class SqlShowTables extends SqlCall implements BeamSqlParser.ExecutableStatement { - private static final SqlOperator OPERATOR = - new SqlSpecialOperator("SHOW TABLES", SqlKind.OTHER_DDL); - private final @Nullable SqlIdentifier databaseName; - private final @Nullable SqlNode regex; - - public SqlShowTables( - SqlParserPos pos, @Nullable SqlIdentifier databaseName, @Nullable SqlNode regex) { - super(pos); - this.databaseName = databaseName; - this.regex = regex; - } - - @Override - public List getOperandList() { - return Collections.emptyList(); - } - - @Override - public SqlOperator getOperator() { - return OPERATOR; - } - - @Override - public void execute(CalcitePrepare.Context context) { - Schema schema = SqlDdlNodes.schema(context, true).schema; - - if (!(schema instanceof CatalogManagerSchema)) { - throw SqlUtil.newContextException( - pos, - RESOURCE.internal( - "Attempting to execute 'SHOW TABLES' with unexpected Calcite Schema of type " - + schema.getClass())); - } - - CatalogSchema catalogSchema; - @Nullable BeamCalciteSchema databaseSchema; - if (databaseName != null) { - List components = Lists.newArrayList(Splitter.on(".").split(databaseName.toString())); - TableName pathOverride = TableName.create(components, ""); - catalogSchema = - pathOverride.catalog() != null - ? ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride) - : ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); - - databaseSchema = catalogSchema.getDatabaseSchema(pathOverride); - } else { - catalogSchema = ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); - databaseSchema = catalogSchema.getCurrentDatabaseSchema(); - } - - if (databaseSchema == null) { - throw SqlUtil.newContextException( - pos, - RESOURCE.internal( - "Attempting to execute 'SHOW TABLES' with no Database used. Please set a Database first then re-run.")); - } - - String path = catalogSchema.getCatalog().name() + "." + databaseSchema.name(); - Collection
tables = databaseSchema.getTables(); - print(tables, path, SqlDdlNodes.getString(regex)); - } - - private static void print(Collection
tables, String path, @Nullable String pattern) { - SqlFunctions.LikeFunction calciteLike = new SqlFunctions.LikeFunction(); - - final String headerName = "Tables in " + path; - final String headerType = "Type"; - final String separatorChar = "-"; - - int nameWidth = headerName.length(); - int typeWidth = headerType.length(); - - for (Table table : tables) { - if (pattern == null || calciteLike.like(table.getName(), pattern)) { - nameWidth = Math.max(nameWidth, table.getName().length()); - typeWidth = Math.max(typeWidth, table.getType().length()); - } - } - - nameWidth += 2; - typeWidth += 2; - String rowFormat = "| %-" + nameWidth + "s | %-" + typeWidth + "s |%n"; - - int separatorWidth = nameWidth + typeWidth + 5; - String separator = - "+" + String.join("", Collections.nCopies(separatorWidth, separatorChar)) + "+"; - - PrintWriter writer = - new PrintWriter(new BufferedWriter(new OutputStreamWriter(System.out, UTF_8))); - writer.println(separator); - writer.printf(rowFormat, headerName, headerType); - writer.println(separator); - for (Table table : - tables.stream().sorted(Comparator.comparing(Table::getName)).collect(Collectors.toList())) { - if (pattern == null || calciteLike.like(table.getName(), pattern)) { - writer.printf(rowFormat, table.getName(), table.getType()); - } - } - writer.println(separator); - - writer.flush(); - } -} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/SystemTables.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/SystemTables.java new file mode 100644 index 000000000000..8e91e9eb0309 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/SystemTables.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.extensions.sql.meta; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +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.schemas.Schema; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PBegin; +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.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Provides {@link BeamSqlTable}s that track metadata around catalogs, databases, and tables. For + * now, it tracks the following: + * + *
    + *
  • Catalogs: Name and Type + *
  • Databases: Name + *
  • Tables: Name and Type + *
+ */ +public class SystemTables { + public static CatalogsMetaTable catalogs(CatalogManager catalogManager, boolean currentOnly) { + return new CatalogsMetaTable(catalogManager, currentOnly); + } + + public static DatabasesMetaTable databases(Catalog catalog, boolean currentOnly) { + return new DatabasesMetaTable(catalog, currentOnly); + } + + public static TablesMetaTable tables(Catalog catalog, String dbName) { + return new TablesMetaTable(catalog, dbName); + } + + public static class CatalogsMetaTable extends BaseBeamTable { + private final CatalogManager catalogManager; + private final boolean currentOnly; + + private static final Schema SCHEMA = + Schema.builder().addStringField("NAME").addStringField("TYPE").build(); + + public CatalogsMetaTable(CatalogManager catalogManager, boolean currentOnly) { + this.catalogManager = catalogManager; + this.currentOnly = currentOnly; + } + + @Override + public PCollection buildIOReader(PBegin begin) { + Collection catalogs = + currentOnly + ? ImmutableList.of(catalogManager.currentCatalog()) + : catalogManager.catalogs(); + List rows = + catalogs.stream() + .map(cat -> Row.withSchema(SCHEMA).addValues(cat.name(), cat.type()).build()) + .collect(Collectors.toList()); + + return begin.apply(Create.of(rows).withRowSchema(SCHEMA)); + } + + @Override + public POutput buildIOWriter(PCollection input) { + throw new UnsupportedOperationException("Cannot write to SHOW CATALOGS"); + } + + @Override + public PCollection.IsBounded isBounded() { + return PCollection.IsBounded.BOUNDED; + } + + @Override + public Schema getSchema() { + return SCHEMA; + } + } + + public static class DatabasesMetaTable extends BaseBeamTable { + private final Catalog catalog; + private final boolean currentOnly; + private static final Schema SCHEMA = Schema.builder().addStringField("NAME").build(); + + DatabasesMetaTable(Catalog catalog, boolean currentOnly) { + this.catalog = catalog; + this.currentOnly = currentOnly; + } + + @Override + public PCollection buildIOReader(PBegin begin) { + Collection databases; + if (currentOnly) { + @Nullable String currentDb = catalog.currentDatabase(); + databases = currentDb != null ? Collections.singleton(currentDb) : Collections.emptyList(); + } else { + databases = catalog.databases(); + } + List rows = + databases.stream() + .map(db -> Row.withSchema(SCHEMA).addValues(db).build()) + .collect(Collectors.toList()); + + return begin.apply(Create.of(rows).withRowSchema(SCHEMA)); + } + + @Override + public POutput buildIOWriter(PCollection input) { + throw new UnsupportedOperationException("Cannot write to SHOW DATABASES"); + } + + @Override + public PCollection.IsBounded isBounded() { + return PCollection.IsBounded.BOUNDED; + } + + @Override + public Schema getSchema() { + return SCHEMA; + } + } + + public static class TablesMetaTable extends BaseBeamTable { + private final Catalog catalog; + private final String dbName; + private static final Schema SCHEMA = + Schema.builder().addStringField("NAME").addStringField("TYPE").build(); + + public TablesMetaTable(Catalog catalog, String dbName) { + this.catalog = catalog; + this.dbName = dbName; + } + + @Override + public PCollection buildIOReader(PBegin begin) { + // Note: This captures the state *at the moment of planning* + List rows = + catalog.metaStore(dbName).getTables().values().stream() + .map( + table -> + Row.withSchema(SCHEMA).addValues(table.getName(), table.getType()).build()) + .collect(Collectors.toList()); + + return begin.apply(Create.of(rows).withRowSchema(SCHEMA)); + } + + @Override + public POutput buildIOWriter(PCollection input) { + throw new UnsupportedOperationException("Cannot write to SHOW TABLES"); + } + + @Override + public PCollection.IsBounded isBounded() { + return PCollection.IsBounded.BOUNDED; + } + + @Override + public Schema getSchema() { + return SCHEMA; + } + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java index 158e56fa785f..c387a5ace10c 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java @@ -51,6 +51,7 @@ public interface Catalog { @Nullable String currentDatabase(); + /** Returns a collection of existing database names. */ Collection databases(); /** diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java index 6ae34647b52b..0164c634814b 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java @@ -26,8 +26,6 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; @@ -332,59 +330,4 @@ public void testCreateWriteDropTableWithOtherCatalogScope() { cli.execute("DROP TABLE catalog_1.db_1.person"); assertNull(metastoreDb1.getTable("person")); } - - @Test - public void testShowCurrentCatalog() { - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - cli.execute("CREATE CATALOG my_very_long_catalog_name TYPE 'local'"); - ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); - System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW CURRENT CATALOG"); - @SuppressWarnings("DefaultCharset") - String printOutput = outputStreamCaptor.toString().trim(); - - assertEquals("default (type: local)", printOutput); - } - - @Test - public void testShowCatalogs() { - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - cli.execute("CREATE CATALOG my_very_long_catalog_name TYPE 'local'"); - ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); - System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW CATALOGS"); - @SuppressWarnings("DefaultCharset") - String printOutput = outputStreamCaptor.toString().trim(); - - assertEquals( - "+---------------------------------------+\n" - + "| Catalog Name | Type |\n" - + "+---------------------------------------+\n" - + "| default | local |\n" - + "| my_catalog | local |\n" - + "| my_very_long_catalog_name | local |\n" - + "+---------------------------------------+", - printOutput); - } - - @Test - public void testShowCatalogsWithPattern() { - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - cli.execute("CREATE CATALOG my_catalog_2 TYPE 'local'"); - cli.execute("CREATE CATALOG my_very_long_catalog_name TYPE 'local'"); - ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); - System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW CATALOGS LIKE 'my_catalog%'"); - @SuppressWarnings("DefaultCharset") - String printOutput = outputStreamCaptor.toString().trim(); - - assertEquals( - "+--------------------------+\n" - + "| Catalog Name | Type |\n" - + "+--------------------------+\n" - + "| my_catalog | local |\n" - + "| my_catalog_2 | local |\n" - + "+--------------------------+", - printOutput); - } } 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 a497e294971a..588caa78a2b7 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 @@ -24,8 +24,6 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; @@ -211,110 +209,4 @@ public void testCreateInsertDropTableUsingOtherDatabase() { cli.execute("DROP TABLE db_1.person"); assertNull(catalogManager.currentCatalog().metaStore("db_1").getTable("person")); } - - @Test - public void testShowCurrentDatabase() { - cli.execute("CREATE DATABASE should_not_show_up"); - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - cli.execute("USE CATALOG my_catalog"); - cli.execute("CREATE DATABASE my_db"); - cli.execute("CREATE DATABASE my_other_db"); - cli.execute("CREATE DATABASE my_database_that_has_a_very_long_name"); - cli.execute("USE DATABASE my_other_db"); - ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); - System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW CURRENT database"); - @SuppressWarnings("DefaultCharset") - String printOutput = outputStreamCaptor.toString().trim(); - - assertEquals("my_other_db", printOutput); - } - - @Test - public void testShowCurrentDatabaseWithNoneSet() { - cli.execute("CREATE DATABASE should_not_show_up"); - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - cli.execute("USE CATALOG my_catalog"); - cli.execute("DROP DATABASE `default`"); - ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); - System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW CURRENT DATABASE"); - @SuppressWarnings("DefaultCharset") - String printOutput = outputStreamCaptor.toString().trim(); - - assertEquals("No database is currently set", printOutput); - } - - @Test - public void testShowDatabases() { - cli.execute("CREATE DATABASE should_not_show_up"); - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - cli.execute("USE CATALOG my_catalog"); - cli.execute("CREATE DATABASE my_db"); - cli.execute("CREATE DATABASE my_other_db"); - cli.execute("CREATE DATABASE my_database_that_has_a_very_long_name"); - ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); - System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW DATABASES"); - @SuppressWarnings("DefaultCharset") - String printOutput = outputStreamCaptor.toString().trim(); - - assertEquals( - "+-----------------------------------------+\n" - + "| Databases in my_catalog |\n" - + "+-----------------------------------------+\n" - + "| default |\n" - + "| my_database_that_has_a_very_long_name |\n" - + "| my_db |\n" - + "| my_other_db |\n" - + "+-----------------------------------------+", - printOutput); - } - - @Test - public void testShowDatabasesInOtherCatalog() { - cli.execute("CREATE DATABASE should_not_show_up"); - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - cli.execute("USE CATALOG my_catalog"); - cli.execute("CREATE DATABASE my_db"); - cli.execute("CREATE CATALOG my_other_catalog TYPE 'local'"); - cli.execute("CREATE DATABASE my_other_catalog.other_db"); - ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); - System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW DATABASES FROM my_other_catalog"); - @SuppressWarnings("DefaultCharset") - String printOutput = outputStreamCaptor.toString().trim(); - - assertEquals( - "+---------------------------------+\n" - + "| Databases in my_other_catalog |\n" - + "+---------------------------------+\n" - + "| default |\n" - + "| other_db |\n" - + "+---------------------------------+", - printOutput); - } - - @Test - public void testShowDatabasesWithPattern() { - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - cli.execute("CREATE DATABASE my_catalog.my_db"); - cli.execute("CREATE DATABASE my_catalog.other_db"); - cli.execute("CREATE DATABASE my_catalog.some_foo_db"); - cli.execute("CREATE DATABASE my_catalog.some_other_foo_db"); - ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); - System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW DATABASES FROM my_catalog LIKE '%foo%'"); - @SuppressWarnings("DefaultCharset") - String printOutput = outputStreamCaptor.toString().trim(); - - assertEquals( - "+---------------------------+\n" - + "| Databases in my_catalog |\n" - + "+---------------------------+\n" - + "| some_foo_db |\n" - + "| some_other_foo_db |\n" - + "+---------------------------+", - printOutput); - } } 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 06fec2e7a111..ffbdeb84f136 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 @@ -28,14 +28,11 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; import java.time.LocalDate; import java.time.LocalTime; import java.util.stream.Stream; import org.apache.beam.sdk.extensions.sql.impl.ParseException; import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; @@ -303,100 +300,4 @@ public void test_time_types() throws Exception { // test TIMESTAMP field assertEquals(parseTimestampWithUTCTimeZone("2018-07-01 21:26:07.123"), row.getDateTime("f_ts")); } - - @Test - public void testShowTables() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - catalogManager.registerTableProvider(new TextTableProvider()); - catalogManager.registerTableProvider(new TestTableProvider()); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - cli.execute("CREATE DATABASE random_db"); - cli.execute("USE DATABASE random_db"); - cli.execute("CREATE EXTERNAL TABLE should_not_show_up (id int, name varchar) TYPE 'text'"); - - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - cli.execute("USE CATALOG my_catalog"); - cli.execute("CREATE DATABASE my_db"); - cli.execute("USE DATABASE my_db"); - cli.execute("CREATE EXTERNAL TABLE my_table (id int, name varchar) TYPE 'text'"); - cli.execute("CREATE EXTERNAL TABLE my_other_table (col1 int, col2 timestamp) TYPE 'text'"); - cli.execute( - "CREATE EXTERNAL TABLE my_other_table_with_a_long_name " - + "(foo varchar, bar boolean) TYPE 'test'"); - ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); - System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW TABLES"); - @SuppressWarnings("DefaultCharset") - String printOutput = outputStreamCaptor.toString().trim(); - - assertEquals( - "+--------------------------------------------+\n" - + "| Tables in my_catalog.my_db | Type |\n" - + "+--------------------------------------------+\n" - + "| my_other_table | text |\n" - + "| my_other_table_with_a_long_name | test |\n" - + "| my_table | text |\n" - + "+--------------------------------------------+", - printOutput); - } - - @Test - public void testShowTablesInOtherDatabase() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - catalogManager.registerTableProvider(new TextTableProvider()); - catalogManager.registerTableProvider(new TestTableProvider()); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - cli.execute("CREATE DATABASE my_db"); - cli.execute( - "CREATE EXTERNAL TABLE my_db.should_not_show_up (id int, name varchar) TYPE 'text'"); - - cli.execute("CREATE CATALOG other_catalog TYPE 'local'"); - cli.execute("CREATE DATABASE other_catalog.other_db"); - cli.execute( - "CREATE EXTERNAL TABLE other_catalog.other_db.other_table (id int, name varchar) TYPE 'text'"); - ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); - System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW TABLES IN other_catalog.other_db"); - @SuppressWarnings("DefaultCharset") - String printOutput = outputStreamCaptor.toString().trim(); - - assertEquals( - "+---------------------------------------------+\n" - + "| Tables in other_catalog.other_db | Type |\n" - + "+---------------------------------------------+\n" - + "| other_table | text |\n" - + "+---------------------------------------------+", - printOutput); - } - - @Test - public void testShowTablesWithPattern() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - catalogManager.registerTableProvider(new TextTableProvider()); - catalogManager.registerTableProvider(new TestTableProvider()); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - cli.execute("CREATE DATABASE my_db"); - cli.execute("CREATE EXTERNAL TABLE my_db.my_table (id int, name varchar) TYPE 'text'"); - cli.execute("CREATE EXTERNAL TABLE my_db.my_table_2 (id int, name varchar) TYPE 'text'"); - cli.execute("CREATE EXTERNAL TABLE my_db.my_foo_table_1 (id int, name varchar) TYPE 'text'"); - cli.execute("CREATE EXTERNAL TABLE my_db.my_foo_table_2 (id int, name varchar) TYPE 'text'"); - - ByteArrayOutputStream outputStreamCaptor = new ByteArrayOutputStream(); - System.setOut(new PrintStream(outputStreamCaptor)); - cli.execute("SHOW TABLES IN my_db LIKE '%foo%'"); - @SuppressWarnings("DefaultCharset") - String printOutput = outputStreamCaptor.toString().trim(); - - assertEquals( - "+------------------------------------+\n" - + "| Tables in default.my_db | Type |\n" - + "+------------------------------------+\n" - + "| my_foo_table_1 | text |\n" - + "| my_foo_table_2 | text |\n" - + "+------------------------------------+", - printOutput); - } } From 4a867e7844bc8be92d75092d73e66875e79dfaa9 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 8 Dec 2025 14:25:00 -0500 Subject: [PATCH 12/14] fix name --- .../{BeamSqlLineShowTests.java => BeamSqlLineShowTest.java} | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) rename sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/{BeamSqlLineShowTests.java => BeamSqlLineShowTest.java} (98%) diff --git a/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineShowTests.java b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineShowTest.java similarity index 98% rename from sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineShowTests.java rename to sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineShowTest.java index 9511f52ed534..0b593a1b2cfb 100644 --- a/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineShowTests.java +++ b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineShowTest.java @@ -29,7 +29,7 @@ import java.util.List; import org.junit.Test; -public class BeamSqlLineShowTests { +public class BeamSqlLineShowTest { @Test public void testShowTables() throws IOException { ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); @@ -48,6 +48,7 @@ public void testShowTables() throws IOException { BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + System.out.println(byteArrayOutputStream.toString("UTF-8")); assertThat( Arrays.asList( "+------+------+", @@ -170,6 +171,7 @@ public void testShowDatabases() throws IOException { BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + System.out.println(byteArrayOutputStream.toString("UTF-8")); assertThat( Arrays.asList( "+------+", @@ -265,6 +267,7 @@ public void testShowCatalogs() throws IOException { BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null); List lines = Arrays.asList(byteArrayOutputStream.toString("UTF-8").split("\n")); + System.out.println(byteArrayOutputStream.toString("UTF-8")); assertThat( Arrays.asList( "+------+------+", From e3b40a012c9732bfd85b266b3f61e61a6042191a Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 9 Dec 2025 12:13:36 -0500 Subject: [PATCH 13/14] address comments --- .../sql/impl/CatalogManagerSchema.java | 17 +++++++++++++---- .../sql/impl/parser/SqlCreateExternalTable.java | 5 +---- .../sql/impl/parser/SqlDropDatabase.java | 5 +---- .../sql/impl/parser/SqlDropTable.java | 5 +---- .../sql/impl/parser/SqlUseDatabase.java | 5 +---- 5 files changed, 17 insertions(+), 20 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java index 5593331e21df..5e50e3489769 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java @@ -21,6 +21,7 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; +import com.google.common.collect.ImmutableSet; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -145,8 +146,7 @@ public void dropCatalog(SqlIdentifier identifier, boolean ifExists) { // will attempt to do so. public void maybeRegisterProvider(TableName path, String type) { type = type.toLowerCase(); - CatalogSchema catalogSchema = - path.catalog() != null ? getCatalogSchema(path) : getCurrentCatalogSchema(); + CatalogSchema catalogSchema = getCatalogSchema(path); BeamCalciteSchema beamCalciteSchema = catalogSchema.getDatabaseSchema(path); if (beamCalciteSchema.getTableProvider() instanceof MetaStore) { @@ -180,8 +180,14 @@ public Set getTableNames() { return getCurrentCatalogSchema().getTableNames(); } + /** + * Returns the {@link CatalogSchema} for the catalog referenced in this {@link TableName}. If the + * path does not reference a catalog, the currently use {@link CatalogSchema} will be returned. + */ public CatalogSchema getCatalogSchema(TableName tablePath) { - return getCatalogSchema(tablePath.catalog()); + return tablePath.catalog() != null + ? getCatalogSchema(tablePath.catalog()) + : getCurrentCatalogSchema(); } public CatalogSchema getCatalogSchema(@Nullable String catalog) { @@ -231,7 +237,10 @@ public CatalogSchema getCurrentCatalogSchema() { @Override public Set getSubSchemaNames() { - return catalogs().stream().map(Catalog::name).collect(Collectors.toSet()); + return ImmutableSet.builder() + .addAll(catalogs().stream().map(Catalog::name).collect(Collectors.toSet())) + .add(BeamSystemSchema.BEAMSYSTEM) + .build(); } public Collection catalogs() { 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 5adc88ccc407..de7903897b62 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 @@ -159,10 +159,7 @@ public void execute(CalcitePrepare.Context context) { CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; catalogManagerSchema.maybeRegisterProvider(pathOverride, SqlDdlNodes.getString(type)); - CatalogSchema catalogSchema = - pathOverride.catalog() != null - ? catalogManagerSchema.getCatalogSchema(pathOverride) - : catalogManagerSchema.getCurrentCatalogSchema(); + CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); beamCalciteSchema = catalogSchema.getDatabaseSchema(pathOverride); } else if (schema instanceof BeamCalciteSchema) { beamCalciteSchema = (BeamCalciteSchema) schema; 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 e3db29c4c14b..4b838c9f4182 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 @@ -84,10 +84,7 @@ public void execute(CalcitePrepare.Context context) { List components = Lists.newArrayList(Splitter.on(".").split(databaseName.toString())); TableName pathOverride = TableName.create(components, ""); - CatalogSchema catalogSchema = - pathOverride.catalog() != null - ? ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride) - : ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); + CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); catalogSchema.dropDatabase(databaseName, cascade, ifExists); } 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 b8cc5f4c72dc..0bc5cd911614 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 @@ -53,10 +53,7 @@ public void execute(CalcitePrepare.Context context) { BeamCalciteSchema beamCalciteSchema; if (schema instanceof CatalogManagerSchema) { - CatalogSchema catalogSchema = - pathOverride.catalog() != null - ? ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride) - : ((CatalogManagerSchema) schema).getCurrentCatalogSchema(); + CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); beamCalciteSchema = catalogSchema.getDatabaseSchema(pathOverride); } else if (schema instanceof BeamCalciteSchema) { beamCalciteSchema = (BeamCalciteSchema) schema; 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 da749c2ceb1e..f0e3fa59ddc7 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 @@ -78,10 +78,7 @@ public void execute(CalcitePrepare.Context context) { } CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; - CatalogSchema catalogSchema = - pathOverride.catalog() != null - ? catalogManagerSchema.getCatalogSchema(pathOverride) - : catalogManagerSchema.getCurrentCatalogSchema(); + CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); // if database exists in a different catalog, we need to also switch to that catalog if (pathOverride.catalog() != null && !pathOverride From c03253628011da1063d376c034df68aa4ab14595 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 9 Dec 2025 13:14:00 -0500 Subject: [PATCH 14/14] spotless --- .../beam/sdk/extensions/sql/impl/CatalogManagerSchema.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java index 5e50e3489769..098b72b28695 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java @@ -21,7 +21,6 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; -import com.google.common.collect.ImmutableSet; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -45,6 +44,7 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; 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.ImmutableSet; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory;