diff --git a/.github/trigger_files/beam_PostCommit_SQL.json b/.github/trigger_files/beam_PostCommit_SQL.json index 833fd9b0d174..6cc79a7a0325 100644 --- a/.github/trigger_files/beam_PostCommit_SQL.json +++ b/.github/trigger_files/beam_PostCommit_SQL.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run ", - "modification": 2 + "modification": 1 } diff --git a/.github/trigger_files/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.json b/.github/trigger_files/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.json index bb31ea07c195..ca2897e2eb2b 100644 --- a/.github/trigger_files/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.json @@ -1,3 +1,3 @@ { - "modification": 1 + "modification": 2 } \ No newline at end of file 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 1209d2b4663d..0ca38824204b 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 @@ -17,10 +17,11 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import java.util.HashMap; import java.util.Map; -import java.util.Set; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalog; -import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; 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; @@ -29,7 +30,7 @@ public class IcebergCatalog extends InMemoryCatalog { // TODO(ahmedabu98): extend this to the IO implementation so // other SDKs can make use of it too private static final String BEAM_HADOOP_PREFIX = "beam.catalog.hadoop"; - private final InMemoryMetaStore metaStore = new InMemoryMetaStore(); + private final Map metaStores = new HashMap<>(); @VisibleForTesting final IcebergCatalogConfig catalogConfig; public IcebergCatalog(String name, Map properties) { @@ -52,12 +53,12 @@ public IcebergCatalog(String name, Map properties) { .setCatalogProperties(catalogProps.build()) .setConfigProperties(hadoopProps.build()) .build(); - metaStore.registerProvider(new IcebergTableProvider(catalogConfig)); } @Override - public InMemoryMetaStore metaStore() { - return metaStore; + public IcebergMetastore metaStore(String db) { + metaStores.putIfAbsent(db, new IcebergMetastore(db, catalogConfig)); + return metaStores.get(db); } @Override @@ -70,17 +71,24 @@ public boolean createDatabase(String database) { return catalogConfig.createNamespace(database); } + @Override + public void useDatabase(String database) { + checkArgument(databaseExists(database), "Database '%s' does not exist."); + currentDatabase = database; + } + + @Override + public boolean databaseExists(String db) { + return catalogConfig.namespaceExists(db); + } + @Override public boolean dropDatabase(String database, boolean cascade) { boolean removed = catalogConfig.dropNamespace(database, cascade); + metaStores.remove(database); if (database.equals(currentDatabase)) { currentDatabase = null; } return removed; } - - @Override - public Set listDatabases() { - return catalogConfig.listNamespaces(); - } } diff --git a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java new file mode 100644 index 000000000000..b73aa25c7a2b --- /dev/null +++ b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java @@ -0,0 +1,154 @@ +/* + * 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.provider.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.extensions.sql.TableUtils; +import org.apache.beam.sdk.extensions.sql.impl.TableName; +import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; +import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; +import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; +import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig.IcebergTableInfo; +import org.apache.beam.sdk.io.iceberg.TableAlreadyExistsException; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class IcebergMetastore extends InMemoryMetaStore { + private static final Logger LOG = LoggerFactory.getLogger(IcebergMetastore.class); + @VisibleForTesting final IcebergCatalogConfig catalogConfig; + private final Map cachedTables = new HashMap<>(); + private final String database; + + public IcebergMetastore(String db, IcebergCatalogConfig catalogConfig) { + this.database = db; + this.catalogConfig = catalogConfig; + } + + @Override + public String getTableType() { + return "iceberg"; + } + + @Override + public void createTable(Table table) { + if (!table.getType().equals("iceberg")) { + getProvider(table.getType()).createTable(table); + } else { + String identifier = getIdentifier(table); + try { + catalogConfig.createTable(identifier, table.getSchema(), table.getPartitionFields()); + } catch (TableAlreadyExistsException e) { + LOG.info( + "Iceberg table '{}' already exists at location '{}'.", table.getName(), identifier); + } + } + cachedTables.put(table.getName(), table); + } + + @Override + public void dropTable(String tableName) { + String identifier = getIdentifier(tableName); + if (catalogConfig.dropTable(identifier)) { + LOG.info("Dropped table '{}' (path: '{}').", tableName, identifier); + } else { + LOG.info( + "Ignoring DROP TABLE call for '{}' (path: '{}') because it does not exist.", + tableName, + identifier); + } + cachedTables.remove(tableName); + } + + @Override + public Map getTables() { + for (String id : catalogConfig.listTables(database)) { + String name = TableName.create(id).getTableName(); + @Nullable Table cachedTable = cachedTables.get(name); + if (cachedTable == null) { + Table table = checkStateNotNull(loadTable(id)); + cachedTables.put(name, table); + } + } + return ImmutableMap.copyOf(cachedTables); + } + + @Override + public @Nullable Table getTable(String name) { + if (cachedTables.containsKey(name)) { + return cachedTables.get(name); + } + @Nullable Table table = loadTable(getIdentifier(name)); + if (table != null) { + cachedTables.put(name, table); + } + return table; + } + + private String getIdentifier(String name) { + return database + "." + name; + } + + private String getIdentifier(Table table) { + checkArgument( + table.getLocation() == null, "Cannot create Iceberg tables using LOCATION property."); + return getIdentifier(table.getName()); + } + + private @Nullable Table loadTable(String identifier) { + @Nullable IcebergTableInfo tableInfo = catalogConfig.loadTable(identifier); + if (tableInfo == null) { + return null; + } + return Table.builder() + .type(getTableType()) + .name(identifier) + .schema(tableInfo.getSchema()) + .properties(TableUtils.parseProperties(tableInfo.getProperties())) + .build(); + } + + @Override + public BeamSqlTable buildBeamSqlTable(Table table) { + if (table.getType().equals("iceberg")) { + return new IcebergTable(getIdentifier(table), table, catalogConfig); + } + return getProvider(table.getType()).buildBeamSqlTable(table); + } + + @Override + public boolean supportsPartitioning(Table table) { + if (table.getType().equals("iceberg")) { + return true; + } + return getProvider(table.getType()).supportsPartitioning(table); + } + + @Override + public void registerProvider(TableProvider provider) { + super.registerProvider(provider); + } +} diff --git a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java index 000ca50e4309..b68aa34a1777 100644 --- a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java +++ b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import com.fasterxml.jackson.databind.node.ObjectNode; @@ -66,10 +65,10 @@ class IcebergTable extends SchemaBaseBeamTable { @VisibleForTesting @Nullable Integer triggeringFrequency; @VisibleForTesting final @Nullable List partitionFields; - IcebergTable(Table table, IcebergCatalogConfig catalogConfig) { + IcebergTable(String tableIdentifier, Table table, IcebergCatalogConfig catalogConfig) { super(table.getSchema()); this.schema = table.getSchema(); - this.tableIdentifier = checkArgumentNotNull(table.getLocation()); + this.tableIdentifier = tableIdentifier; this.catalogConfig = catalogConfig; ObjectNode properties = table.getProperties(); if (properties.has(TRIGGERING_FREQUENCY_FIELD)) { diff --git a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java deleted file mode 100644 index 568893716581..000000000000 --- a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java +++ /dev/null @@ -1,96 +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.meta.provider.iceberg; - -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; -import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; - -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; -import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; -import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; -import org.apache.beam.sdk.io.iceberg.TableAlreadyExistsException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A table provider for Iceberg tables. CREATE and DROP operations are performed on real external - * tables. - */ -public class IcebergTableProvider implements TableProvider { - private static final Logger LOG = LoggerFactory.getLogger(IcebergTableProvider.class); - @VisibleForTesting final IcebergCatalogConfig catalogConfig; - private final Map tables = new HashMap<>(); - - public IcebergTableProvider(IcebergCatalogConfig catalogConfig) { - this.catalogConfig = catalogConfig; - } - - @Override - public String getTableType() { - return "iceberg"; - } - - @Override - public void createTable(Table table) { - try { - catalogConfig.createTable( - checkStateNotNull(table.getLocation()), table.getSchema(), table.getPartitionFields()); - } catch (TableAlreadyExistsException e) { - LOG.info( - "Iceberg table '{}' already exists at location '{}'.", - table.getName(), - table.getLocation()); - } - tables.put(table.getName(), table); - } - - @Override - public void dropTable(String tableName) { - Table table = - checkArgumentNotNull(getTable(tableName), "Table '%s' is not registered.", tableName); - String location = checkStateNotNull(table.getLocation()); - if (catalogConfig.dropTable(location)) { - LOG.info("Dropped table '{}' (location: '{}').", tableName, location); - } else { - LOG.info( - "Ignoring DROP TABLE call for '{}' (location: '{}') because it does not exist.", - tableName, - location); - } - tables.remove(tableName); - } - - @Override - public Map getTables() { - return tables; - } - - @Override - public BeamSqlTable buildBeamSqlTable(Table table) { - return new IcebergTable(table, catalogConfig); - } - - @Override - public boolean supportsPartitioning(Table table) { - return true; - } -} diff --git a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java index 0c51b31f1927..9ac96652d340 100644 --- a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java +++ b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java @@ -18,19 +18,29 @@ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; import static java.lang.String.format; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; import java.util.UUID; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.sql.BeamSqlCli; +import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.CalciteContextException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; -import org.junit.Assert; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.DateTime; import org.junit.Before; import org.junit.ClassRule; import org.junit.Rule; @@ -43,6 +53,7 @@ public class BeamSqlCliIcebergTest { @Rule public transient ExpectedException thrown = ExpectedException.none(); private InMemoryCatalogManager catalogManager; private BeamSqlCli cli; + private BeamSqlEnv sqlEnv; private String warehouse; @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -50,17 +61,26 @@ public class BeamSqlCliIcebergTest { public void setup() throws IOException { catalogManager = new InMemoryCatalogManager(); cli = new BeamSqlCli().catalogManager(catalogManager); + sqlEnv = + BeamSqlEnv.builder(catalogManager) + .setPipelineOptions(PipelineOptionsFactory.create()) + .build(); File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue(warehouseFile.delete()); + assertTrue(warehouseFile.delete()); warehouse = "file:" + warehouseFile + "/" + UUID.randomUUID(); } private String createCatalog(String name) { + return createCatalog(name, null); + } + + private String createCatalog(String name, @Nullable String warehouseOverride) { + String ware = warehouseOverride != null ? warehouseOverride : warehouse; return format("CREATE CATALOG %s \n", name) + "TYPE iceberg \n" + "PROPERTIES (\n" + " 'type' = 'hadoop', \n" - + format(" 'warehouse' = '%s')", warehouse); + + format(" 'warehouse' = '%s')", ware); } @Test @@ -68,7 +88,6 @@ public void testCreateCatalog() { assertEquals("default", catalogManager.currentCatalog().name()); cli.execute(createCatalog("my_catalog")); - assertNotNull(catalogManager.getCatalog("my_catalog")); assertEquals("default", catalogManager.currentCatalog().name()); cli.execute("USE CATALOG my_catalog"); @@ -83,11 +102,11 @@ public void testCreateNamespace() { IcebergCatalog catalog = (IcebergCatalog) catalogManager.currentCatalog(); assertEquals("default", catalog.currentDatabase()); cli.execute("CREATE DATABASE new_namespace"); - assertEquals("new_namespace", Iterables.getOnlyElement(catalog.listDatabases())); + assertTrue(catalog.databaseExists("new_namespace")); // Specifies IF NOT EXISTS, so should be a no-op cli.execute("CREATE DATABASE IF NOT EXISTS new_namespace"); - assertEquals("new_namespace", Iterables.getOnlyElement(catalog.listDatabases())); + assertTrue(catalog.databaseExists("new_namespace")); // This one doesn't, so it should throw an error. thrown.expect(CalciteContextException.class); @@ -126,7 +145,7 @@ public void testDropNamespace() { cli.execute("USE DATABASE new_namespace"); assertEquals("new_namespace", catalog.currentDatabase()); cli.execute("DROP DATABASE new_namespace"); - assertTrue(catalog.listDatabases().isEmpty()); + assertFalse(catalog.databaseExists("new_namespace")); assertNull(catalog.currentDatabase()); // Drop non-existent namespace with IF EXISTS @@ -137,4 +156,83 @@ public void testDropNamespace() { thrown.expectMessage("Database 'new_namespace' does not exist."); cli.execute("DROP DATABASE new_namespace"); } + + @Test + public void testCrossCatalogTableWriteAndRead() throws IOException { + // create and use catalog 1 + sqlEnv.executeDdl(createCatalog("catalog_1")); + sqlEnv.executeDdl("USE CATALOG catalog_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + // create and use database inside catalog 1 + IcebergCatalog catalog = (IcebergCatalog) catalogManager.currentCatalog(); + sqlEnv.executeDdl("CREATE DATABASE my_namespace"); + sqlEnv.executeDdl("USE DATABASE my_namespace"); + assertEquals("my_namespace", catalog.currentDatabase()); + // create and write to table inside database + String tableIdentifier = "my_namespace.my_table"; + sqlEnv.executeDdl( + format("CREATE EXTERNAL TABLE %s( \n", tableIdentifier) + + " c_integer INTEGER, \n" + + " c_boolean BOOLEAN, \n" + + " c_timestamp TIMESTAMP, \n" + + " c_varchar VARCHAR \n " + + ") \n" + + "TYPE 'iceberg'\n"); + BeamRelNode insertNode = + sqlEnv.parseQuery( + format("INSERT INTO %s VALUES (", tableIdentifier) + + "2147483647, " + + "TRUE, " + + "TIMESTAMP '2025-07-31 20:17:40.123', " + + "'varchar' " + + ")"); + Pipeline p1 = Pipeline.create(); + BeamSqlRelUtils.toPCollection(p1, insertNode); + p1.run().waitUntilFinish(); + + // create and use a new catalog, with a new database + File warehouseFile2 = TEMPORARY_FOLDER.newFolder(); + assertTrue(warehouseFile2.delete()); + String warehouse2 = "file:" + warehouseFile2 + "/" + UUID.randomUUID(); + sqlEnv.executeDdl(createCatalog("catalog_2", warehouse2)); + sqlEnv.executeDdl("USE CATALOG catalog_2"); + sqlEnv.executeDdl("CREATE DATABASE other_namespace"); + sqlEnv.executeDdl("USE DATABASE other_namespace"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + assertEquals("other_namespace", catalogManager.currentCatalog().currentDatabase()); + + // insert from old catalog to new table in new catalog + sqlEnv.executeDdl( + "CREATE EXTERNAL TABLE other_table( \n" + + " c_integer INTEGER, \n" + + " c_boolean BOOLEAN, \n" + + " c_timestamp TIMESTAMP, \n" + + " c_varchar VARCHAR) \n" + + "TYPE 'iceberg'\n"); + BeamRelNode insertNode2 = + sqlEnv.parseQuery("INSERT INTO other_table SELECT * FROM catalog_1.my_namespace.my_table"); + Pipeline p2 = Pipeline.create(); + BeamSqlRelUtils.toPCollection(p2, insertNode2); + p2.run().waitUntilFinish(); + + // switch over to catalog 1 and read table inside catalog 2 + sqlEnv.executeDdl("USE DATABASE catalog_1.my_namespace"); + BeamRelNode insertNode3 = + sqlEnv.parseQuery("SELECT * FROM catalog_2.other_namespace.other_table"); + Pipeline p3 = Pipeline.create(); + PCollection output = BeamSqlRelUtils.toPCollection(p3, insertNode3); + + // validate read contents + Schema expectedSchema = + checkStateNotNull(catalog.catalogConfig.loadTable(tableIdentifier)).getSchema(); + assertEquals(expectedSchema, output.getSchema()); + PAssert.that(output) + .containsInAnyOrder( + Row.withSchema(expectedSchema) + .addValues(2147483647, true, DateTime.parse("2025-07-31T20:17:40.123Z"), "varchar") + .build()); + p3.run().waitUntilFinish(); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + assertEquals("my_namespace", catalogManager.currentCatalog().currentDatabase()); + } } diff --git a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java new file mode 100644 index 000000000000..a7baf1191d15 --- /dev/null +++ b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java @@ -0,0 +1,97 @@ +/* + * 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.provider.iceberg; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.util.UUID; +import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +/** UnitTest for {@link IcebergMetastore}. */ +public class IcebergMetastoreTest { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + private IcebergCatalog catalog; + + @Before + public void setup() throws IOException { + File warehouseFile = TEMPORARY_FOLDER.newFolder(); + assertTrue(warehouseFile.delete()); + String warehouse = "file:" + warehouseFile + "/" + UUID.randomUUID(); + catalog = + new IcebergCatalog( + "test_catalog", ImmutableMap.of("type", "hadoop", "warehouse", warehouse)); + } + + private IcebergMetastore metastore() { + return catalog.metaStore(catalog.currentDatabase()); + } + + @Test + public void testGetTableType() { + assertEquals("iceberg", metastore().getTableType()); + } + + @Test + public void testBuildBeamSqlTable() { + Table table = Table.builder().name("my_table").schema(Schema.of()).type("iceberg").build(); + BeamSqlTable sqlTable = metastore().buildBeamSqlTable(table); + + assertNotNull(sqlTable); + assertTrue(sqlTable instanceof IcebergTable); + + IcebergTable icebergTable = (IcebergTable) sqlTable; + assertEquals(catalog.currentDatabase() + ".my_table", icebergTable.tableIdentifier); + assertEquals(catalog.catalogConfig, icebergTable.catalogConfig); + } + + @Test + public void testCreateTable() { + Table table = Table.builder().name("my_table").schema(Schema.of()).type("iceberg").build(); + metastore().createTable(table); + + assertNotNull(catalog.catalogConfig.loadTable(catalog.currentDatabase() + ".my_table")); + } + + @Test + public void testGetTables() { + Table table1 = Table.builder().name("my_table_1").schema(Schema.of()).type("iceberg").build(); + Table table2 = Table.builder().name("my_table_2").schema(Schema.of()).type("iceberg").build(); + metastore().createTable(table1); + metastore().createTable(table2); + + assertEquals(ImmutableSet.of("my_table_1", "my_table_2"), metastore().getTables().keySet()); + } + + @Test + public void testSupportsPartitioning() { + Table table = Table.builder().name("my_table_1").schema(Schema.of()).type("iceberg").build(); + assertTrue(metastore().supportsPartitioning(table)); + } +} diff --git a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java index a7b128b2bca3..417db09a2210 100644 --- a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java +++ b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -43,6 +43,7 @@ import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; +import org.apache.beam.sdk.extensions.sql.impl.TableName; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamPushDownIOSourceRel; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; @@ -140,6 +141,7 @@ public void runSqlWriteAndRead(boolean withPartitionFields) .setPipelineOptions(PipelineOptionsFactory.create()) .build(); String tableIdentifier = DATASET + "." + testName.getMethodName(); + String tableName = TableName.create(tableIdentifier).getTableName(); // 1) create Iceberg catalog String createCatalog = @@ -153,9 +155,9 @@ public void runSqlWriteAndRead(boolean withPartitionFields) + " 'gcp_region' = 'us-central1')"; sqlEnv.executeDdl(createCatalog); - // 2) use the catalog we just created - String setCatalog = "USE CATALOG my_catalog"; - sqlEnv.executeDdl(setCatalog); + // 2) use the catalog we just created and dataset + sqlEnv.executeDdl("USE CATALOG my_catalog"); + sqlEnv.executeDdl("USE DATABASE " + DATASET); // 3) create beam table String partitionFields = @@ -163,7 +165,7 @@ public void runSqlWriteAndRead(boolean withPartitionFields) ? "PARTITIONED BY ('bucket(c_integer, 5)', 'c_boolean', 'hour(c_timestamp)', 'truncate(c_varchar, 3)') \n" : ""; String createTableStatement = - "CREATE EXTERNAL TABLE TEST( \n" + format("CREATE EXTERNAL TABLE %s( \n", tableName) + " c_bigint BIGINT, \n" + " c_integer INTEGER, \n" + " c_float FLOAT, \n" @@ -176,17 +178,13 @@ public void runSqlWriteAndRead(boolean withPartitionFields) + " c_arr_struct ARRAY, c_arr_struct_integer INTEGER>> \n" + ") \n" + "TYPE 'iceberg' \n" - + partitionFields - + "LOCATION '" - + tableIdentifier - + "'"; + + partitionFields; sqlEnv.executeDdl(createTableStatement); // 3) verify a real Iceberg table was created, with the right partition spec IcebergCatalog catalog = (IcebergCatalog) catalogManager.currentCatalog(); - IcebergTableProvider provider = - (IcebergTableProvider) catalog.metaStore().getProvider("iceberg"); - Catalog icebergCatalog = provider.catalogConfig.catalog(); + IcebergMetastore metastore = catalog.metaStore(DATASET); + Catalog icebergCatalog = metastore.catalogConfig.catalog(); PartitionSpec expectedSpec = PartitionSpec.unpartitioned(); if (withPartitionFields) { expectedSpec = @@ -202,12 +200,12 @@ public void runSqlWriteAndRead(boolean withPartitionFields) assertEquals("my_catalog." + tableIdentifier, icebergTable.name()); assertTrue(icebergTable.location().startsWith(warehouse)); assertEquals(expectedSpec, icebergTable.spec()); - Schema expectedSchema = checkStateNotNull(provider.getTable("TEST")).getSchema(); + Schema expectedSchema = checkStateNotNull(metastore.getTable(tableName)).getSchema(); assertEquals(expectedSchema, IcebergUtils.icebergSchemaToBeamSchema(icebergTable.schema())); // 4) write to underlying Iceberg table String insertStatement = - "INSERT INTO TEST VALUES (" + format("INSERT INTO %s VALUES (", tableName) + "9223372036854775807, " + "2147483647, " + "1.0, " @@ -252,7 +250,7 @@ public void runSqlWriteAndRead(boolean withPartitionFields) assertEquals(expectedRow, beamRow); // 6) read using Beam SQL and verify - String selectTableStatement = "SELECT * FROM TEST"; + String selectTableStatement = "SELECT * FROM " + tableName; PCollection output = BeamSqlRelUtils.toPCollection(readPipeline, sqlEnv.parseQuery(selectTableStatement)); PAssert.that(output).containsInAnyOrder(expectedRow); @@ -260,7 +258,7 @@ public void runSqlWriteAndRead(boolean withPartitionFields) assertThat(state, equalTo(PipelineResult.State.DONE)); // 7) cleanup - sqlEnv.executeDdl("DROP TABLE TEST"); + sqlEnv.executeDdl("DROP TABLE " + tableName); assertFalse(icebergCatalog.tableExists(TableIdentifier.parse(tableIdentifier))); } @@ -271,6 +269,7 @@ public void testSQLReadWithProjectAndFilterPushDown() { .setPipelineOptions(PipelineOptionsFactory.create()) .build(); String tableIdentifier = DATASET + "." + testName.getMethodName(); + String tableName = TableName.create(tableIdentifier).getTableName(); // 1) create Iceberg catalog String createCatalog = @@ -284,28 +283,25 @@ public void testSQLReadWithProjectAndFilterPushDown() { + " 'gcp_region' = 'us-central1')"; sqlEnv.executeDdl(createCatalog); - // 2) use the catalog we just created - String setCatalog = "USE CATALOG my_catalog"; - sqlEnv.executeDdl(setCatalog); + // 2) use the catalog we just created and the dataset + sqlEnv.executeDdl("USE CATALOG my_catalog"); + sqlEnv.executeDdl("USE DATABASE " + DATASET); // 3) create Beam table String createTableStatement = - "CREATE EXTERNAL TABLE TEST( \n" + format("CREATE EXTERNAL TABLE %s( \n", tableName) + " c_integer INTEGER, \n" + " c_float FLOAT, \n" + " c_boolean BOOLEAN, \n" + " c_timestamp TIMESTAMP, \n" + " c_varchar VARCHAR \n " + ") \n" - + "TYPE 'iceberg' \n" - + "LOCATION '" - + tableIdentifier - + "'"; + + "TYPE 'iceberg'"; sqlEnv.executeDdl(createTableStatement); // 4) insert some data) String insertStatement = - "INSERT INTO TEST VALUES " + format("INSERT INTO %s VALUES ", tableName) + "(123, 1.23, TRUE, TIMESTAMP '2025-05-22 20:17:40.123', 'a'), " + "(456, 4.56, FALSE, TIMESTAMP '2025-05-25 20:17:40.123', 'b'), " + "(789, 7.89, TRUE, TIMESTAMP '2025-05-28 20:17:40.123', 'c')"; @@ -314,7 +310,7 @@ public void testSQLReadWithProjectAndFilterPushDown() { // 5) read with a filter String selectTableStatement = - "SELECT c_integer, c_varchar FROM TEST where " + format("SELECT c_integer, c_varchar FROM %s where ", tableName) + "(c_boolean=TRUE and c_varchar in ('a', 'b')) or c_float > 5"; BeamRelNode relNode = sqlEnv.parseQuery(selectTableStatement); PCollection output = BeamSqlRelUtils.toPCollection(readPipeline, relNode); diff --git a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java deleted file mode 100644 index cf066b1abed8..000000000000 --- a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java +++ /dev/null @@ -1,83 +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.meta.provider.iceberg; - -import static org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTable.TRIGGERING_FREQUENCY_FIELD; -import static org.apache.beam.sdk.schemas.Schema.toSchema; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - -import java.util.stream.Stream; -import org.apache.beam.sdk.extensions.sql.TableUtils; -import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; -import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_40_0.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.junit.Test; - -/** UnitTest for {@link IcebergTableProvider}. */ -public class IcebergTableProviderTest { - private final IcebergCatalog catalog = - new IcebergCatalog( - "test_catalog", - ImmutableMap.of( - "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", - "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", - "warehouse", "gs://bucket/warehouse", - "beam.catalog.test_catalog.hadoop.fs.gs.project.id", "apache-beam-testing", - "beam.catalog.test_catalog.hadoop.foo", "bar")); - - @Test - public void testGetTableType() { - assertNotNull(catalog.metaStore().getProvider("iceberg")); - } - - @Test - public void testBuildBeamSqlTable() throws Exception { - ImmutableMap properties = ImmutableMap.of(TRIGGERING_FREQUENCY_FIELD, 30); - - ObjectMapper mapper = new ObjectMapper(); - String propertiesString = mapper.writeValueAsString(properties); - Table table = - fakeTableBuilder("my_table") - .properties(TableUtils.parseProperties(propertiesString)) - .build(); - BeamSqlTable sqlTable = catalog.metaStore().buildBeamSqlTable(table); - - assertNotNull(sqlTable); - assertTrue(sqlTable instanceof IcebergTable); - - IcebergTable icebergTable = (IcebergTable) sqlTable; - assertEquals("namespace.my_table", icebergTable.tableIdentifier); - assertEquals(catalog.catalogConfig, icebergTable.catalogConfig); - } - - private static Table.Builder fakeTableBuilder(String name) { - return Table.builder() - .name(name) - .location("namespace." + name) - .schema( - Stream.of( - Schema.Field.nullable("id", Schema.FieldType.INT32), - Schema.Field.nullable("name", Schema.FieldType.STRING)) - .collect(toSchema())) - .type("iceberg"); - } -} diff --git a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java index bdd710c861e0..900fdae743a1 100644 --- a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java +++ b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java @@ -150,18 +150,15 @@ public void testSimpleInsertWithPartitionedFields() throws Exception { + "' \n" + "TBLPROPERTIES '{ \"timestampAttributeKey\" : \"ts\" }'"; String icebergTableString = - "CREATE EXTERNAL TABLE iceberg_table( \n" + format("CREATE EXTERNAL TABLE %s( \n", tableIdentifier) + " id BIGINT, \n" + " name VARCHAR \n " + ") \n" + "TYPE 'iceberg' \n" + "PARTITIONED BY('id', 'truncate(name, 3)') \n" - + "LOCATION '" - + tableIdentifier - + "' \n" + "TBLPROPERTIES '{ \"triggering_frequency_seconds\" : 10 }'"; String insertStatement = - "INSERT INTO iceberg_table \n" + format("INSERT INTO %s \n", tableIdentifier) + "SELECT \n" + " pubsub_topic.payload.id, \n" + " pubsub_topic.payload.name \n" @@ -208,18 +205,15 @@ public void testSimpleInsertFlat() throws Exception { + pubsub.topicPath() + "' \n" + "TBLPROPERTIES '{ \"timestampAttributeKey\" : \"ts\" }'"; - String bqTableString = - "CREATE EXTERNAL TABLE iceberg_table( \n" + String icebergTableString = + format("CREATE EXTERNAL TABLE %s( \n", tableIdentifier) + " id BIGINT, \n" + " name VARCHAR \n " + ") \n" + "TYPE 'iceberg' \n" - + "LOCATION '" - + tableIdentifier - + "' \n" + "TBLPROPERTIES '{ \"triggering_frequency_seconds\" : 10 }'"; String insertStatement = - "INSERT INTO iceberg_table \n" + format("INSERT INTO %s \n", tableIdentifier) + "SELECT \n" + " id, \n" + " name \n" @@ -230,7 +224,7 @@ public void testSimpleInsertFlat() throws Exception { .withDdlString(createCatalogDdl) .withDdlString(setCatalogDdl) .withDdlString(pubsubTableString) - .withDdlString(bqTableString)); + .withDdlString(icebergTableString)); pipeline.run(); // Block until a subscription for this topic exists 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 470cbb443895..46102c7b92fe 100644 --- a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl +++ b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl @@ -265,12 +265,12 @@ SqlDrop SqlDropCatalog(Span s, boolean replace) : } /** - * CREATE DATABASE ( IF NOT EXISTS )? database_name + * CREATE DATABASE ( IF NOT EXISTS )? ( catalog_name '.' )? database_name */ SqlCreate SqlCreateDatabase(Span s, boolean replace) : { final boolean ifNotExists; - final SqlNode databaseName; + final SqlIdentifier databaseName; } { { @@ -278,11 +278,7 @@ SqlCreate SqlCreateDatabase(Span s, boolean replace) : } ifNotExists = IfNotExistsOpt() - ( - databaseName = StringLiteral() - | - databaseName = SimpleIdentifier() - ) + databaseName = CompoundIdentifier() { return new SqlCreateDatabase( @@ -294,22 +290,18 @@ SqlCreate SqlCreateDatabase(Span s, boolean replace) : } /** - * USE DATABASE database_name + * USE DATABASE ( catalog_name '.' )? database_name */ SqlCall SqlUseDatabase(Span s, String scope) : { - final SqlNode databaseName; + final SqlIdentifier databaseName; } { { s.add(this); } - ( - databaseName = StringLiteral() - | - databaseName = SimpleIdentifier() - ) + databaseName = CompoundIdentifier() { return new SqlUseDatabase( s.end(this), @@ -324,17 +316,13 @@ SqlCall SqlUseDatabase(Span s, String scope) : SqlDrop SqlDropDatabase(Span s, boolean replace) : { final boolean ifExists; - final SqlNode databaseName; + final SqlIdentifier databaseName; final boolean cascade; } { ifExists = IfExistsOpt() - ( - databaseName = StringLiteral() - | - databaseName = SimpleIdentifier() - ) + databaseName = CompoundIdentifier() cascade = CascadeOpt() @@ -363,7 +351,7 @@ SqlNodeList PartitionFieldList() : * Note: This example is probably out of sync with the code. * * CREATE EXTERNAL TABLE ( IF NOT EXISTS )? - * ( database_name '.' )? table_name '(' column_def ( ',' column_def )* ')' + * ( catalog_name '.' )? ( database_name '.' )? table_name '(' column_def ( ',' column_def )* ')' * TYPE type_name * ( PARTITIONED BY '(' partition_field ( ',' partition_field )* ')' )? * ( COMMENT comment_string )? diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java index f9cc1fd9d482..8365f56e27de 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.impl.schema.BeamPCollectionTable; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; +import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; import org.apache.beam.sdk.extensions.sql.meta.provider.ReadOnlyTableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; @@ -136,8 +137,8 @@ public abstract class SqlTransform extends PTransform> public PCollection expand(PInput input) { TableProvider inputTableProvider = new ReadOnlyTableProvider(PCOLLECTION_NAME, toTableMap(input)); - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - catalogManager.registerTableProvider(PCOLLECTION_NAME, inputTableProvider); + CatalogManager catalogManager = new InMemoryCatalogManager(); + catalogManager.registerTableProvider(inputTableProvider); BeamSqlEnvBuilder sqlEnvBuilder = BeamSqlEnv.builder(catalogManager); // TODO: validate duplicate functions. diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableUtils.java index 2e52a1bbf422..5285999f3292 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableUtils.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableUtils.java @@ -63,6 +63,10 @@ public static ObjectNode parseProperties(String json) { } } + public static ObjectNode parseProperties(Map map) { + return objectMapper.valueToTree(map); + } + public static Map convertNode2Map(JsonNode jsonNode) { return objectMapper.convertValue(jsonNode, new TypeReference>() {}); } 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 d684c72b2e69..c76ae79dd45d 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 @@ -17,8 +17,6 @@ */ 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.HashMap; @@ -37,36 +35,32 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schemas; import org.checkerframework.checker.nullness.qual.Nullable; -/** Adapter from {@link TableProvider} to {@link Schema}. */ +/** + * A Calcite {@link Schema} that corresponds to a {@link TableProvider} or {@link + * org.apache.beam.sdk.extensions.sql.meta.store.MetaStore}. In Beam SQL, a DATABASE refers to a + * {@link BeamCalciteSchema}. + */ @SuppressWarnings({"keyfor", "nullness"}) // TODO(https://github.com/apache/beam/issues/20497) public class BeamCalciteSchema implements Schema { private JdbcConnection connection; - private @Nullable TableProvider tableProvider; - private @Nullable CatalogManager catalogManager; + private TableProvider tableProvider; private Map subSchemas; + private final String name; - BeamCalciteSchema(JdbcConnection jdbcConnection, TableProvider tableProvider) { + /** Creates a {@link BeamCalciteSchema} representing a {@link TableProvider}. */ + BeamCalciteSchema(String name, JdbcConnection jdbcConnection, TableProvider tableProvider) { this.connection = jdbcConnection; this.tableProvider = tableProvider; this.subSchemas = new HashMap<>(); + this.name = name; } - /** - * Creates a {@link BeamCalciteSchema} representing a {@link CatalogManager}. This will typically - * be the root node of a pipeline. - */ - BeamCalciteSchema(JdbcConnection jdbcConnection, CatalogManager catalogManager) { - this.connection = jdbcConnection; - this.catalogManager = catalogManager; - this.subSchemas = new HashMap<>(); + public String name() { + return name; } public TableProvider getTableProvider() { - return resolveMetastore(); - } - - public @Nullable CatalogManager getCatalogManager() { - return catalogManager; + return tableProvider; } public Map getPipelineOptions() { @@ -106,7 +100,7 @@ public Expression getExpression(SchemaPlus parentSchema, String name) { @Override public Set getTableNames() { - return resolveMetastore().getTables().keySet(); + return tableProvider.getTables().keySet(); } @Override @@ -122,13 +116,13 @@ public Set getTypeNames() { @Override public org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Table getTable( String name) { - Table table = resolveMetastore().getTable(name); + Table table = tableProvider.getTable(name); if (table == null) { return null; } return new BeamCalciteTable( - resolveMetastore().buildBeamSqlTable(table), - getPipelineOptions(), + tableProvider.buildBeamSqlTable(table), + connection.getPipelineOptionsMap(), connection.getPipelineOptions()); } @@ -144,7 +138,7 @@ public Collection getFunctions(String name) { @Override public Set getSubSchemaNames() { - return resolveMetastore().getSubProviders(); + return tableProvider.getSubProviders(); } /** @@ -157,23 +151,11 @@ public Set getSubSchemaNames() { public Schema getSubSchema(String name) { if (!subSchemas.containsKey(name)) { BeamCalciteSchema subSchema; - if (tableProvider != null) { - @Nullable TableProvider subProvider = tableProvider.getSubProvider(name); - subSchema = subProvider != null ? new BeamCalciteSchema(connection, subProvider) : null; - } else { - @Nullable Catalog catalog = checkStateNotNull(catalogManager).getCatalog(name); - subSchema = catalog != null ? new BeamCalciteSchema(connection, catalog.metaStore()) : null; - } + @Nullable TableProvider subProvider = tableProvider.getSubProvider(name); + subSchema = subProvider != null ? new BeamCalciteSchema(name, connection, subProvider) : null; subSchemas.put(name, subSchema); } return subSchemas.get(name); } - - public TableProvider resolveMetastore() { - if (tableProvider != null) { - return tableProvider; - } - return checkStateNotNull(catalogManager).currentCatalog().metaStore(); - } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java index 73193f58f131..c77ff1f85b79 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java @@ -41,6 +41,7 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.ReadOnlyTableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.UdfUdafProvider; +import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; @@ -51,7 +52,6 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.tools.RuleSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; -import org.checkerframework.checker.nullness.qual.Nullable; /** * Contains the metadata of tables/UDF functions, and exposes APIs to @@ -150,7 +150,6 @@ public static class BeamSqlEnvBuilder { private static final String CALCITE_PLANNER = "org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner"; private String queryPlannerClassName; - private @Nullable TableProvider defaultTableProvider; private CatalogManager catalogManager; private String currentSchemaName; private Map schemaMap; @@ -162,8 +161,12 @@ public static class BeamSqlEnvBuilder { private BeamSqlEnvBuilder(TableProvider tableProvider) { checkNotNull(tableProvider, "Table provider for the default schema must be sets."); - defaultTableProvider = tableProvider; - catalogManager = new InMemoryCatalogManager(); + if (tableProvider instanceof MetaStore) { + catalogManager = new InMemoryCatalogManager((MetaStore) tableProvider); + } else { + catalogManager = new InMemoryCatalogManager(); + catalogManager.registerTableProvider(tableProvider); + } queryPlannerClassName = CALCITE_PLANNER; schemaMap = new HashMap<>(); functionSet = new HashSet<>(); @@ -264,12 +267,7 @@ public BeamSqlEnvBuilder setUseCatalog(String name) { public BeamSqlEnv build() { checkStateNotNull(pipelineOptions); - JdbcConnection jdbcConnection; - if (defaultTableProvider != null) { - jdbcConnection = JdbcDriver.connect(defaultTableProvider, pipelineOptions); - } else { - jdbcConnection = JdbcDriver.connect(catalogManager, pipelineOptions); - } + JdbcConnection jdbcConnection = JdbcDriver.connect(catalogManager, pipelineOptions); configureSchemas(jdbcConnection); 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 new file mode 100644 index 000000000000..ec225efc1c39 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java @@ -0,0 +1,278 @@ +/* + * 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 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.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes; +import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; +import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; +import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; +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.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; +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.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A Calcite {@link Schema} that corresponds to a {@link CatalogManager}. This is typically the root + * node of a pipeline. Child schemas are of type {@link CatalogSchema}. + */ +public class CatalogManagerSchema implements Schema { + private static final Logger LOG = LoggerFactory.getLogger(CatalogManagerSchema.class); + private final JdbcConnection connection; + private final CatalogManager catalogManager; + private final Map catalogSubSchemas = new HashMap<>(); + + CatalogManagerSchema(JdbcConnection jdbcConnection, CatalogManager catalogManager) { + this.connection = jdbcConnection; + this.catalogManager = catalogManager; + } + + @VisibleForTesting + public JdbcConnection connection() { + return connection; + } + + public void createCatalog( + SqlIdentifier catalogIdentifier, + String type, + Map properties, + boolean replace, + boolean ifNotExists) { + String name = SqlDdlNodes.name(catalogIdentifier); + if (catalogManager.getCatalog(name) != null) { + if (replace) { + LOG.info("Replacing existing catalog '{}'", name); + catalogManager.dropCatalog(name); + } else if (!ifNotExists) { + throw SqlUtil.newContextException( + catalogIdentifier.getParserPosition(), + RESOURCE.internal(String.format("Catalog '%s' already exists.", name))); + } else { + LOG.info("Catalog '{}' already exists", name); + return; + } + } + + catalogManager.createCatalog(name, type, properties); + CatalogSchema catalogSchema = + new CatalogSchema(connection, checkStateNotNull(catalogManager.getCatalog(name))); + catalogSubSchemas.put(name, catalogSchema); + } + + public void useCatalog(SqlIdentifier catalogIdentifier) { + String name = catalogIdentifier.toString(); + if (catalogManager.getCatalog(catalogIdentifier.toString()) == null) { + throw SqlUtil.newContextException( + catalogIdentifier.getParserPosition(), + RESOURCE.internal(String.format("Cannot use catalog: '%s' not found.", name))); + } + + if (catalogManager.currentCatalog().name().equals(name)) { + LOG.info("Catalog '{}' is already in use.", name); + return; + } + + catalogManager.useCatalog(name); + LOG.info("Switched to catalog '{}' (type: {})", name, catalogManager.currentCatalog().type()); + } + + public void dropCatalog(SqlIdentifier identifier, boolean ifExists) { + String name = SqlDdlNodes.name(identifier); + if (catalogManager.getCatalog(name) == null) { + if (!ifExists) { + throw SqlUtil.newContextException( + identifier.getParserPosition(), + RESOURCE.internal(String.format("Cannot drop catalog: '%s' not found.", name))); + } + LOG.info("Ignoring 'DROP CATALOG` call for non-existent catalog: {}", name); + return; + } + + if (catalogManager.currentCatalog().name().equals(name)) { + throw SqlUtil.newContextException( + identifier.getParserPosition(), + RESOURCE.internal( + String.format( + "Unable to drop active catalog '%s'. Please switch to another catalog first.", + name))); + } + + catalogManager.dropCatalog(name); + LOG.info("Successfully dropped catalog '{}'", name); + catalogSubSchemas.remove(name); + } + + // A BeamCalciteSchema may be used to interact with multiple TableProviders. + // If such a TableProvider is not registered in the BeamCalciteSchema, this method + // will attempt to do so. + public void maybeRegisterProvider(TableName path, String type) { + type = type.toLowerCase(); + CatalogSchema catalogSchema = getCatalogSchema(path); + BeamCalciteSchema beamCalciteSchema = catalogSchema.getDatabaseSchema(path); + + if (beamCalciteSchema.getTableProvider() instanceof MetaStore) { + MetaStore metaStore = (MetaStore) beamCalciteSchema.getTableProvider(); + if (metaStore.tableProviders().containsKey(type)) { + return; + } + + // Start with the narrowest scope. + // Attempt to fetch provider from Catalog first, then CatalogManager. + @Nullable TableProvider provider = catalogSchema.getCatalog().tableProviders().get(type); + if (provider == null) { + provider = catalogManager.tableProviders().get(type); + } + // register provider + if (provider != null) { + metaStore.registerProvider(provider); + } + } + } + + @Override + public @Nullable Table getTable(String table) { + @Nullable + CatalogSchema catalogSchema = catalogSubSchemas.get(catalogManager.currentCatalog().name()); + return catalogSchema != null ? catalogSchema.getTable(table) : null; + } + + @Override + public Set getTableNames() { + return getCurrentCatalogSchema().getTableNames(); + } + + public CatalogSchema getCatalogSchema(TableName tablePath) { + @Nullable Schema catalogSchema = getSubSchema(tablePath.catalog()); + if (catalogSchema == null) { + catalogSchema = getCurrentCatalogSchema(); + } + Preconditions.checkState( + catalogSchema instanceof CatalogSchema, + "Unexpected Schema type for Catalog '%s': %s", + tablePath.catalog(), + catalogSchema.getClass()); + return (CatalogSchema) catalogSchema; + } + + public CatalogSchema getCurrentCatalogSchema() { + return (CatalogSchema) + checkStateNotNull( + getSubSchema(catalogManager.currentCatalog().name()), + "Could not find Calcite Schema for active catalog '%s'.", + catalogManager.currentCatalog().name()); + } + + @Override + public @Nullable Schema getSubSchema(@Nullable String name) { + if (name == null) { + return null; + } + @Nullable CatalogSchema catalogSchema = catalogSubSchemas.get(name); + if (catalogSchema == null) { + @Nullable Catalog catalog = catalogManager.getCatalog(name); + if (catalog != null) { + catalogSchema = new CatalogSchema(connection, catalog); + catalogSubSchemas.put(name, catalogSchema); + } + } + if (catalogSchema != null) { + return catalogSchema; + } + + // ** Backwards compatibility ** + // Name could be referring to a BeamCalciteSchema. + // Attempt to fetch from current catalog + return getCurrentCatalogSchema().getSubSchema(name); + } + + @Override + public Set getSubSchemaNames() { + return catalogManager.catalogs().stream().map(Catalog::name).collect(Collectors.toSet()); + } + + public void setPipelineOption(String key, String value) { + Map options = new HashMap<>(connection.getPipelineOptionsMap()); + options.put(key, value); + connection.setPipelineOptionsMap(options); + } + + public void removePipelineOption(String key) { + Map options = new HashMap<>(connection.getPipelineOptionsMap()); + options.remove(key); + connection.setPipelineOptionsMap(options); + } + + public void removeAllPipelineOptions() { + connection.setPipelineOptionsMap(Collections.emptyMap()); + } + + @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/CatalogSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java new file mode 100644 index 000000000000..792e5b98bcd3 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java @@ -0,0 +1,229 @@ +/* + * 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 java.lang.String.format; +import static org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog.DEFAULT; +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 java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes; +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.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.SqlUtil; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A Calcite {@link Schema} that corresponds to a {@link Catalog}. Child schemas are of type {@link + * BeamCalciteSchema}. + */ +public class CatalogSchema implements Schema { + private static final Logger LOG = LoggerFactory.getLogger(CatalogSchema.class); + private final JdbcConnection connection; + private final Catalog catalog; + private final Map subSchemas = new HashMap<>(); + /** + * Creates a Calcite {@link Schema} representing a {@link CatalogManager}. This will typically be + * the root node of a pipeline. + */ + CatalogSchema(JdbcConnection jdbcConnection, Catalog catalog) { + this.connection = jdbcConnection; + this.catalog = catalog; + // should always have a "default" sub-schema available + subSchemas.put(DEFAULT, new BeamCalciteSchema(DEFAULT, connection, catalog.metaStore(DEFAULT))); + } + + public Catalog getCatalog() { + return catalog; + } + + public @Nullable BeamCalciteSchema getCurrentDatabaseSchema() { + return getSubSchema(catalog.currentDatabase()); + } + + public BeamCalciteSchema getDatabaseSchema(TableName tablePath) { + @Nullable BeamCalciteSchema beamCalciteSchema = getSubSchema(tablePath.database()); + if (beamCalciteSchema == null) { + beamCalciteSchema = getCurrentDatabaseSchema(); + } + return checkStateNotNull( + beamCalciteSchema, "Could not find BeamCalciteSchema for table: '%s'", tablePath); + } + + public void createDatabase(SqlIdentifier databaseIdentifier, boolean ifNotExists) { + String name = SqlDdlNodes.name(databaseIdentifier); + boolean alreadyExists = subSchemas.containsKey(name); + + if (!alreadyExists || name.equals(DEFAULT)) { + try { + LOG.info("Creating database '{}'", name); + if (catalog.createDatabase(name)) { + LOG.info("Successfully created database '{}'", name); + } else { + alreadyExists = true; + } + } catch (Exception e) { + throw SqlUtil.newContextException( + databaseIdentifier.getParserPosition(), + RESOURCE.internal( + format("Encountered an error when creating database '%s': %s", name, e))); + } + } + + if (alreadyExists) { + String message = format("Database '%s' already exists.", name); + if (ifNotExists || name.equals(DEFAULT)) { + LOG.info(message); + } else { + throw SqlUtil.newContextException( + databaseIdentifier.getParserPosition(), RESOURCE.internal(message)); + } + } + + subSchemas.put(name, new BeamCalciteSchema(name, connection, catalog.metaStore(name))); + } + + public void useDatabase(SqlIdentifier identifier) { + String name = SqlDdlNodes.name(identifier); + if (!subSchemas.containsKey(name)) { + if (!catalog.databaseExists(name)) { + throw SqlUtil.newContextException( + identifier.getParserPosition(), + RESOURCE.internal(String.format("Cannot use database: '%s' not found.", name))); + } + subSchemas.put(name, new BeamCalciteSchema(name, connection, catalog.metaStore(name))); + } + + if (name.equals(catalog.currentDatabase())) { + LOG.info("Database '{}' is already in use.", name); + return; + } + + catalog.useDatabase(name); + LOG.info("Switched to database '{}'.", name); + } + + public void dropDatabase(SqlIdentifier identifier, boolean cascade, boolean ifExists) { + String name = SqlDdlNodes.name(identifier); + try { + LOG.info("Dropping database '{}'", name); + boolean dropped = catalog.dropDatabase(name, cascade); + + if (dropped) { + LOG.info("Successfully dropped database '{}'", name); + } else if (ifExists) { + LOG.info("Database '{}' does not exist.", name); + } else { + throw SqlUtil.newContextException( + identifier.getParserPosition(), + RESOURCE.internal(String.format("Database '%s' does not exist.", name))); + } + } catch (Exception e) { + throw SqlUtil.newContextException( + identifier.getParserPosition(), + RESOURCE.internal( + format("Encountered an error when dropping database '%s': %s", name, e))); + } + + subSchemas.remove(name); + } + + @Override + public @Nullable Table getTable(String s) { + @Nullable BeamCalciteSchema beamCalciteSchema = currentDatabase(); + return beamCalciteSchema != null ? beamCalciteSchema.getTable(s) : null; + } + + @Override + public Set getTableNames() { + @Nullable BeamCalciteSchema beamCalciteSchema = currentDatabase(); + return beamCalciteSchema != null ? beamCalciteSchema.getTableNames() : Collections.emptySet(); + } + + @Override + public @Nullable BeamCalciteSchema getSubSchema(@Nullable String name) { + if (name == null) { + return null; + } + + if (!subSchemas.containsKey(name) && catalog.databaseExists(name)) { + subSchemas.put(name, new BeamCalciteSchema(name, connection, catalog.metaStore(name))); + } + return subSchemas.get(name); + } + + private @Nullable BeamCalciteSchema currentDatabase() { + return getSubSchema(catalog.currentDatabase()); + } + + @Override + public Set getSubSchemaNames() { + return subSchemas.keySet(); + } + + @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/JdbcConnection.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java index f9d7eddbc687..baea5ae155b4 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java @@ -136,13 +136,13 @@ public SchemaPlus getCurrentSchemaPlus() { *

Overrides the schema if it exists. */ void setSchema(String name, TableProvider tableProvider) { - BeamCalciteSchema beamCalciteSchema = new BeamCalciteSchema(this, tableProvider); + BeamCalciteSchema beamCalciteSchema = new BeamCalciteSchema(name, this, tableProvider); getRootSchema().add(name, beamCalciteSchema); } /** Like {@link #setSchema(String, TableProvider)} but using a {@link CatalogManager}. */ void setSchema(String name, CatalogManager catalogManager) { - BeamCalciteSchema beamCalciteSchema = new BeamCalciteSchema(this, catalogManager); - getRootSchema().add(name, beamCalciteSchema); + CatalogManagerSchema catalogManagerSchema = new CatalogManagerSchema(this, catalogManager); + getRootSchema().add(name, catalogManagerSchema); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java index f69918e2c58c..53d8debaaf95 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java @@ -25,6 +25,12 @@ import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.dataflow.qual.Pure; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -60,6 +66,12 @@ public abstract class TableName { /** Table name, the last element of the fully-specified table name with path. */ public abstract String getTableName(); + /** Splits the input String by "." separator and returns a new {@link TableName}. */ + public static TableName create(String path) { + List components = Lists.newArrayList(Splitter.on(".").split(path)); + return create(components); + } + /** Full table name with path. */ public static TableName create(List fullPath) { checkNotNull(fullPath, "Full table path cannot be null"); @@ -97,4 +109,22 @@ public TableName removePrefix() { List pathPostfix = getPath().stream().skip(1).collect(toList()); return TableName.create(pathPostfix, getTableName()); } + + /** Returns the database name in this table path. */ + @Pure + public @Nullable String database() { + return isCompound() ? Iterables.getLast(getPath()) : null; + } + + @Pure + public @Nullable String catalog() { + return getPath().size() > 1 ? getPath().get(0) : null; + } + + @Override + public final String toString() { + List components = + ImmutableList.builder().addAll(getPath()).add(getTableName()).build(); + return String.join(".", components); + } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java index 5626520f21dd..f4a90177bb52 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java @@ -26,8 +26,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; @@ -43,12 +42,8 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SqlCreateCatalog extends SqlCreate implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlCreateCatalog.class); private final SqlIdentifier catalogName; private final SqlNode type; private final SqlNodeList properties; @@ -118,42 +113,20 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, catalogName); Schema schema = pair.left.schema; - String name = pair.right; String typeStr = checkArgumentNotNull(SqlDdlNodes.getString(type)); - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } - - @Nullable CatalogManager catalogManager = ((BeamCalciteSchema) schema).getCatalogManager(); - if (catalogManager == null) { + if (!(schema instanceof CatalogManagerSchema)) { throw SqlUtil.newContextException( catalogName.getParserPosition(), RESOURCE.internal( - String.format( - "Unexpected 'CREATE CATALOG' call for Schema '%s' that is not a Catalog.", - name))); - } - - // check if catalog already exists - if (catalogManager.getCatalog(name) != null) { - if (getReplace()) { - LOG.info("Replacing existing catalog '{}'", name); - catalogManager.dropCatalog(name); - } else if (!ifNotExists) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal(String.format("Catalog '%s' already exists.", name))); - } else { - return; - } + "Attempting to create catalog '" + + SqlDdlNodes.name(catalogName) + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } - // create the catalog - catalogManager.createCatalog(name, typeStr, parseProperties()); - LOG.info("Catalog '{}' (type: {}) successfully created", name, typeStr); + ((CatalogManagerSchema) schema) + .createCatalog(catalogName, typeStr, parseProperties(), getReplace(), ifNotExists); } private Map parseProperties() { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java index c2524e3c9867..877b6721152c 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java @@ -17,13 +17,13 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import static java.lang.String.format; +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 static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.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.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; @@ -37,21 +37,20 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SqlCreateDatabase extends SqlCreate implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlCreateDatabase.class); private final SqlIdentifier databaseName; private static final SqlOperator OPERATOR = new SqlSpecialOperator("CREATE DATABASE", SqlKind.OTHER_DDL); public SqlCreateDatabase( - SqlParserPos pos, boolean replace, boolean ifNotExists, SqlNode databaseName) { + SqlParserPos pos, boolean replace, boolean ifNotExists, SqlIdentifier databaseName) { super(OPERATOR, pos, replace, ifNotExists); - this.databaseName = SqlDdlNodes.getIdentifier(databaseName, pos); + this.databaseName = databaseName; } @Override @@ -78,44 +77,39 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, databaseName); Schema schema = pair.left.schema; - String name = pair.right; - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } + List components = Lists.newArrayList(Splitter.on('.').split(databaseName.toString())); + @Nullable + String catalogName = components.size() > 1 ? components.get(components.size() - 2) : null; - @Nullable CatalogManager catalogManager = ((BeamCalciteSchema) schema).getCatalogManager(); - if (catalogManager == null) { + if (!(schema instanceof CatalogManagerSchema)) { throw SqlUtil.newContextException( databaseName.getParserPosition(), RESOURCE.internal( - format( - "Unexpected 'CREATE DATABASE' call using Schema '%s' that is not a Catalog.", - name))); + "Attempting to create database '" + + databaseName + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } - // Attempt to create the database. - Catalog catalog = catalogManager.currentCatalog(); - try { - LOG.info("Creating database '{}'", name); - boolean created = catalog.createDatabase(name); - - if (created) { - LOG.info("Successfully created database '{}'", name); - } else if (ifNotExists) { - LOG.info("Database '{}' already exists.", name); - } else { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal(format("Database '%s' already exists.", name))); - } - } catch (Exception e) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal( - format("Encountered an error when creating database '%s': %s", name, e))); + CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; + CatalogSchema catalogSchema = catalogManagerSchema.getCurrentCatalogSchema(); + // override if a catalog name is present + if (catalogName != null) { + Schema overridden = + checkStateNotNull( + catalogManagerSchema.getSubSchema(catalogName), + "Could not find Calcite Schema for catalog '%s'.", + catalogName); + checkState( + overridden instanceof CatalogSchema, + "Catalog '%s' had unexpected Calcite Schema of type %s. Expected type: %s.", + catalogName, + overridden.getClass(), + CatalogSchema.class.getSimpleName()); + catalogSchema = (CatalogSchema) overridden; } + + catalogSchema.createDatabase(databaseName, ifNotExists); } } 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 96b534e36d25..ab644145b4f7 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; +import static org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes.name; +import static org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes.schema; import static org.apache.beam.sdk.schemas.Schema.toSchema; import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -26,11 +28,15 @@ import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.sql.TableUtils; 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.impl.utils.CalciteUtils; import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlCreate; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; @@ -50,7 +56,7 @@ }) public class SqlCreateExternalTable extends SqlCreate implements BeamSqlParser.ExecutableStatement { private final SqlIdentifier name; - private final List columnList; + private final List columnList; private final SqlNode type; private final SqlNode comment; private final SqlNode location; @@ -66,7 +72,7 @@ public SqlCreateExternalTable( boolean replace, boolean ifNotExists, SqlIdentifier name, - List columnList, + List columnList, SqlNode type, SqlNodeList partitionFields, SqlNode comment, @@ -144,28 +150,41 @@ public void execute(CalcitePrepare.Context context) { } return; } - // Table does not exist. Create it. - if (!(pair.left.schema instanceof BeamCalciteSchema)) { + + Schema schema = pair.left.schema; + + BeamCalciteSchema beamCalciteSchema; + if (schema instanceof CatalogManagerSchema) { + TableName pathOverride = TableName.create(name.toString()); + CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; + catalogManagerSchema.maybeRegisterProvider(pathOverride, SqlDdlNodes.getString(type)); + + CatalogSchema catalogSchema = catalogManagerSchema.getCatalogSchema(pathOverride); + beamCalciteSchema = catalogSchema.getDatabaseSchema(pathOverride); + } else if (schema instanceof BeamCalciteSchema) { + beamCalciteSchema = (BeamCalciteSchema) schema; + } else { throw SqlUtil.newContextException( name.getParserPosition(), - RESOURCE.internal("Schema is not instanceof BeamCalciteSchema")); + RESOURCE.internal( + "Attempting to create a table with unexpected Calcite Schema of type " + + schema.getClass())); } - - BeamCalciteSchema schema = (BeamCalciteSchema) pair.left.schema; Table table = toTable(); + if (partitionFields != null) { checkArgument( - schema.resolveMetastore().supportsPartitioning(table), + beamCalciteSchema.getTableProvider().supportsPartitioning(table), "Invalid use of 'PARTITIONED BY()': Table '%s' of type '%s' " + "does not support partitioning.", - SqlDdlNodes.name(name), + name(name), SqlDdlNodes.getString(type)); } - schema.resolveMetastore().createTable(table); + beamCalciteSchema.getTableProvider().createTable(table); } - private void unparseColumn(SqlWriter writer, Schema.Field column) { + private void unparseColumn(SqlWriter writer, Field column) { writer.sep(","); writer.identifier(column.getName(), false); writer.identifier(CalciteUtils.toSqlTypeName(column.getType()).name(), false); @@ -190,11 +209,12 @@ private void unparseColumn(SqlWriter writer, Schema.Field column) { private Table toTable() { return Table.builder() .type(SqlDdlNodes.getString(type)) - .name(SqlDdlNodes.name(name)) + .name(name(name)) .schema(columnList.stream().collect(toSchema())) .partitionFields(parsePartitionFields()) .comment(SqlDdlNodes.getString(comment)) .location(SqlDdlNodes.getString(location)) + // .path(path) .properties( (tblProperties == null) ? TableUtils.emptyProperties() 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 4c99b3aa3518..c5d162ebbb68 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 @@ -18,6 +18,7 @@ package org.apache.beam.sdk.extensions.sql.impl.parser; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import java.util.List; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; @@ -50,23 +51,33 @@ public static SqlNode column( /** Returns the schema in which to create an object. */ static Pair schema( CalcitePrepare.Context context, boolean mutable, SqlIdentifier id) { - final List path; - if (id.isSimple()) { - path = context.getDefaultSchemaPath(); - } else { + CalciteSchema rootSchema = mutable ? context.getMutableRootSchema() : context.getRootSchema(); + @Nullable CalciteSchema schema = null; + List path = null; + if (!id.isSimple()) { path = Util.skipLast(id.names); + schema = childSchema(rootSchema, path); + } + // if id isSimple or if the above returned a null schema, use default schema path + if (schema == null) { + path = context.getDefaultSchemaPath(); + schema = childSchema(rootSchema, path); } - CalciteSchema schema = mutable ? context.getMutableRootSchema() : context.getRootSchema(); + return Pair.of(checkStateNotNull(schema, "Got null sub-schema for path '%s'", path), name(id)); + } + + private static @Nullable CalciteSchema childSchema(CalciteSchema rootSchema, List path) { + @Nullable CalciteSchema schema = rootSchema; for (String p : path) { - schema = schema.getSubSchema(p, true); if (schema == null) { - throw new AssertionError(String.format("Got null sub-schema for path '%s' in %s", p, path)); + break; } + schema = schema.getSubSchema(p, true); } - return Pair.of(schema, name(id)); + return schema; } - static String name(SqlIdentifier id) { + public static String name(SqlIdentifier id) { if (id.isSimple()) { return id.getSimple(); } else { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java index 8985484128cf..7a8ccdf7d435 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java @@ -20,8 +20,7 @@ import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; @@ -36,12 +35,8 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SqlDropCatalog extends SqlDrop implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlDropCatalog.class); private static final SqlOperator OPERATOR = new SqlSpecialOperator("DROP CATALOG", SqlKind.OTHER_DDL); private final SqlIdentifier catalogName; @@ -64,45 +59,18 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, catalogName); Schema schema = pair.left.schema; - String name = pair.right; - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } - - BeamCalciteSchema beamCalciteSchema = (BeamCalciteSchema) schema; - @Nullable CatalogManager catalogManager = beamCalciteSchema.getCatalogManager(); - if (catalogManager == null) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal( - String.format( - "Unexpected 'DROP CATALOG' call for Schema '%s' that is not a Catalog.", name))); - } - - if (catalogManager.getCatalog(name) == null) { - if (!ifExists) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal(String.format("Cannot drop catalog: '%s' not found.", name))); - } - LOG.info("Ignoring 'DROP CATALOG` call for non-existent catalog: {}", name); - return; - } - - if (catalogManager.currentCatalog().name().equals(name)) { + if (!(schema instanceof CatalogManagerSchema)) { throw SqlUtil.newContextException( catalogName.getParserPosition(), RESOURCE.internal( - String.format( - "Unable to drop active catalog '%s'. Please switch to another catalog first.", - name))); + "Attempting to drop a catalog '" + + SqlDdlNodes.name(catalogName) + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } - catalogManager.dropCatalog(name); - LOG.info("Successfully dropped catalog '{}'", name); + ((CatalogManagerSchema) schema).dropCatalog(catalogName, ifExists); } @Override 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 f4938b5fff45..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 @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import static java.lang.String.format; import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.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.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; @@ -37,22 +36,20 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlWriter; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; public class SqlDropDatabase extends SqlDrop implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlDropDatabase.class); private static final SqlOperator OPERATOR = new SqlSpecialOperator("DROP DATABASE", SqlKind.OTHER_DDL); private final SqlIdentifier databaseName; private final boolean cascade; public SqlDropDatabase( - SqlParserPos pos, boolean ifExists, SqlNode databaseName, boolean cascade) { + SqlParserPos pos, boolean ifExists, SqlIdentifier databaseName, boolean cascade) { super(OPERATOR, pos, ifExists); - this.databaseName = SqlDdlNodes.getIdentifier(databaseName, pos); + this.databaseName = databaseName; this.cascade = cascade; } @@ -74,45 +71,21 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, databaseName); Schema schema = pair.left.schema; - String name = pair.right; - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } - - BeamCalciteSchema beamCalciteSchema = (BeamCalciteSchema) schema; - @Nullable CatalogManager catalogManager = beamCalciteSchema.getCatalogManager(); - if (catalogManager == null) { + if (!(schema instanceof CatalogManagerSchema)) { throw SqlUtil.newContextException( databaseName.getParserPosition(), RESOURCE.internal( - String.format( - "Unexpected 'DROP DATABASE' call using Schema '%s' that is not a Catalog.", - name))); + "Attempting to drop database '" + + databaseName + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } - Catalog catalog = catalogManager.currentCatalog(); - try { - LOG.info("Dropping database '{}'", name); - boolean dropped = catalog.dropDatabase(name, cascade); - - if (dropped) { - LOG.info("Successfully dropped database '{}'", name); - } else if (ifExists) { - LOG.info("Database '{}' does not exist.", name); - } else { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal(String.format("Database '%s' does not exist.", name))); - } - } catch (Exception e) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal( - format("Encountered an error when dropping database '%s': %s", name, e))); - } + List components = Lists.newArrayList(Splitter.on(".").split(databaseName.toString())); + TableName pathOverride = TableName.create(components, ""); + CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); + catalogSchema.dropDatabase(databaseName, cascade, ifExists); } @Override 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 18d06ef8aebc..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 @@ -17,11 +17,23 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; +import static org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes.name; +import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; + +import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; +import org.apache.beam.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.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.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.calcite.v1_40_0.org.apache.calcite.util.Pair; /** Parse tree for {@code DROP TABLE} statement. */ public class SqlDropTable extends SqlDropObject { @@ -32,6 +44,39 @@ public class SqlDropTable extends SqlDropObject { SqlDropTable(SqlParserPos pos, boolean ifExists, SqlIdentifier name) { super(OPERATOR, pos, ifExists, name); } + + @Override + public void execute(CalcitePrepare.Context context) { + final Pair pair = SqlDdlNodes.schema(context, true, name); + TableName pathOverride = TableName.create(name.toString()); + Schema schema = pair.left.schema; + + BeamCalciteSchema beamCalciteSchema; + if (schema instanceof CatalogManagerSchema) { + CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); + beamCalciteSchema = catalogSchema.getDatabaseSchema(pathOverride); + } else if (schema instanceof BeamCalciteSchema) { + beamCalciteSchema = (BeamCalciteSchema) schema; + } else { + throw SqlUtil.newContextException( + name.getParserPosition(), + RESOURCE.internal( + "Attempting to drop a table using unexpected Calcite Schema of type " + + schema.getClass())); + } + + if (beamCalciteSchema.getTable(pair.right) == null) { + // Table does not exist. + if (!ifExists) { + // They did not specify IF EXISTS, so give error. + throw SqlUtil.newContextException( + name.getParserPosition(), RESOURCE.tableNotFound(name.toString())); + } + return; + } + + beamCalciteSchema.getTableProvider().dropTable(pair.right); + } } // End SqlDropTable.java diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java index f949a1fc9ae7..338ae8baeb6b 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java @@ -20,8 +20,10 @@ import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSetOption; @@ -44,20 +46,29 @@ public void execute(CalcitePrepare.Context context) { final SqlIdentifier name = getName(); final SqlNode value = getValue(); final Pair pair = SqlDdlNodes.schema(context, true, name); - if (!(pair.left.schema instanceof BeamCalciteSchema)) { + Schema schema = pair.left.schema; + if (schema instanceof CatalogManagerSchema) { + CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; + if (value != null) { + catalogManagerSchema.setPipelineOption(pair.right, SqlDdlNodes.getString(value)); + } else if ("ALL".equals(pair.right)) { + catalogManagerSchema.removeAllPipelineOptions(); + } else { + catalogManagerSchema.removePipelineOption(pair.right); + } + } else if (schema instanceof BeamCalciteSchema) { + BeamCalciteSchema beamCalciteSchema = (BeamCalciteSchema) schema; + if (value != null) { + beamCalciteSchema.setPipelineOption(pair.right, SqlDdlNodes.getString(value)); + } else if ("ALL".equals(pair.right)) { + beamCalciteSchema.removeAllPipelineOptions(); + } else { + beamCalciteSchema.removePipelineOption(pair.right); + } + } else { throw SqlUtil.newContextException( name.getParserPosition(), - RESOURCE.internal("Schema is not instanceof BeamCalciteSchema")); - } - - BeamCalciteSchema schema = (BeamCalciteSchema) pair.left.schema; - - if (value != null) { - schema.setPipelineOption(pair.right, SqlDdlNodes.getString(value)); - } else if ("ALL".equals(pair.right)) { - schema.removeAllPipelineOptions(); - } else { - schema.removePipelineOption(pair.right); + RESOURCE.internal("Schema is not instanceof CatalogManagerSchema or BeamCalciteSchema")); } } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java index 1e96e3799ad1..52884f0ccdf3 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java @@ -21,8 +21,7 @@ import java.util.Collections; import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; @@ -35,12 +34,8 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SqlUseCatalog extends SqlSetOption implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlUseCatalog.class); private final SqlIdentifier catalogName; private static final SqlOperator OPERATOR = new SqlSpecialOperator("USE CATALOG", SqlKind.OTHER); @@ -64,36 +59,17 @@ public List getOperandList() { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, catalogName); Schema schema = pair.left.schema; - String name = pair.right; - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } - - BeamCalciteSchema beamCalciteSchema = (BeamCalciteSchema) schema; - @Nullable CatalogManager catalogManager = beamCalciteSchema.getCatalogManager(); - if (catalogManager == null) { + if (!(schema instanceof CatalogManagerSchema)) { throw SqlUtil.newContextException( catalogName.getParserPosition(), RESOURCE.internal( - String.format( - "Unexpected 'USE CATALOG' call for Schema '%s' that is not a Catalog.", name))); - } - - if (catalogManager.getCatalog(name) == null) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal(String.format("Cannot use catalog: '%s' not found.", name))); - } - - if (catalogManager.currentCatalog().name().equals(name)) { - LOG.info("Catalog '{}' is already in use.", name); - return; + "Attempting to 'USE CATALOG' " + + catalogName + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } - catalogManager.useCatalog(name); - LOG.info("Switched to catalog '{}' (type: {})", name, catalogManager.currentCatalog().type()); + ((CatalogManagerSchema) schema).useCatalog(catalogName); } } 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 b3bf122cadbf..9d06e471dbbe 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 @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import java.util.Collections; import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.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.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; @@ -37,19 +36,17 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlUtil; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +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; public class SqlUseDatabase extends SqlSetOption implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlUseDatabase.class); private final SqlIdentifier databaseName; private static final SqlOperator OPERATOR = new SqlSpecialOperator("USE DATABASE", SqlKind.OTHER); - public SqlUseDatabase(SqlParserPos pos, String scope, SqlNode databaseName) { + public SqlUseDatabase(SqlParserPos pos, String scope, SqlIdentifier databaseName) { super(pos, scope, SqlDdlNodes.getIdentifier(databaseName, pos), null); - this.databaseName = SqlDdlNodes.getIdentifier(databaseName, pos); + this.databaseName = databaseName; } @Override @@ -66,38 +63,32 @@ public List getOperandList() { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, databaseName); Schema schema = pair.left.schema; - String name = checkStateNotNull(pair.right); + String path = databaseName.toString(); + List components = Lists.newArrayList(Splitter.on(".").split(path)); + TableName pathOverride = TableName.create(components, ""); - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } - - BeamCalciteSchema beamCalciteSchema = (BeamCalciteSchema) schema; - @Nullable CatalogManager catalogManager = beamCalciteSchema.getCatalogManager(); - if (catalogManager == null) { + if (!(schema instanceof CatalogManagerSchema)) { throw SqlUtil.newContextException( databaseName.getParserPosition(), RESOURCE.internal( - String.format( - "Unexpected 'USE DATABASE' call using Schema '%s' that is not a Catalog.", - name))); - } - - Catalog catalog = catalogManager.currentCatalog(); - if (!catalog.listDatabases().contains(name)) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal(String.format("Cannot use database: '%s' not found.", name))); + "Attempting to create database '" + + path + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } - if (name.equals(catalog.currentDatabase())) { - LOG.info("Database '{}' is already in use.", name); - return; + CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; + CatalogSchema catalogSchema = catalogManagerSchema.getCatalogSchema(pathOverride); + // if database exists in a different catalog, we need to also switch to that catalog + if (pathOverride.catalog() != null + && !pathOverride + .catalog() + .equals(catalogManagerSchema.getCurrentCatalogSchema().getCatalog().name())) { + SqlIdentifier catalogIdentifier = + new SqlIdentifier(pathOverride.catalog(), databaseName.getParserPosition()); + catalogManagerSchema.useCatalog(catalogIdentifier); } - catalog.useDatabase(name); - LOG.info("Switched to database '{}'.", name); + catalogSchema.useDatabase(databaseName); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/Table.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/Table.java index 3b72baa9b38e..5c03a2b20b25 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/Table.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/Table.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.extensions.sql.TableUtils; import org.apache.beam.sdk.schemas.Schema; import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.dataflow.qual.Pure; /** Represents the metadata of a {@code BeamSqlTable}. */ @AutoValue @@ -39,7 +40,7 @@ public abstract class Table implements Serializable { public abstract @Nullable String getComment(); - public abstract @Nullable String getLocation(); + public abstract @Pure @Nullable String getLocation(); public abstract ObjectNode getProperties(); 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 e347584654cd..db7724a4809d 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 @@ -18,8 +18,8 @@ package org.apache.beam.sdk.extensions.sql.meta.catalog; import java.util.Map; -import java.util.Set; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; import org.checkerframework.checker.nullness.qual.Nullable; @@ -36,8 +36,11 @@ public interface Catalog { /** A type that defines this catalog. */ String type(); - /** The underlying {@link MetaStore} that actually manages tables. */ - MetaStore metaStore(); + /** + * Returns the underlying {@link MetaStore} for this database. Creates a new {@link MetaStore} if + * one does not exist yet. + */ + MetaStore metaStore(String database); /** * Produces the currently active database. Can be null if no database is active. @@ -55,12 +58,8 @@ public interface Catalog { */ boolean createDatabase(String databaseName); - /** - * Returns a set of existing databases accessible to this catalog. - * - * @return a set of existing database names - */ - Set listDatabases(); + /** Returns true if the database exists. */ + boolean databaseExists(String db); /** * Switches to use the specified database. @@ -84,4 +83,12 @@ public interface Catalog { /** User-specified configuration properties. */ Map properties(); + + /** Registers this {@link TableProvider} and propagates it to underlying {@link MetaStore}s. */ + void registerTableProvider(TableProvider provider); + + /** + * Returns all the {@link TableProvider}s available to this {@link Catalog}, organized by type. + */ + Map tableProviders(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java index 4654f0dd1b0d..808449de5d54 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.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.impl.BeamCalciteSchema; @@ -55,9 +56,13 @@ public interface CatalogManager { * Registers a {@link TableProvider} and propagates it to all the {@link Catalog} instances * available to this manager. */ - void registerTableProvider(String name, TableProvider tableProvider); + void registerTableProvider(TableProvider tableProvider); - default void registerTableProvider(TableProvider tp) { - registerTableProvider(tp.getTableType(), tp); - } + /** + * Returns all the {@link TableProvider}s available to this {@link CatalogManager}, organized by + * type. + */ + Map tableProviders(); + + Collection catalogs(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java index 71bcd0b58af3..0fa3dd4d01c1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java @@ -17,9 +17,11 @@ */ package org.apache.beam.sdk.extensions.sql.meta.catalog; +import java.util.Collection; import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.checkerframework.checker.nullness.qual.Nullable; public class EmptyCatalogManager implements CatalogManager { @@ -49,14 +51,24 @@ public void dropCatalog(String name) { } @Override - public void registerTableProvider(String name, TableProvider tableProvider) { + public void registerTableProvider(TableProvider tableProvider) { throw new UnsupportedOperationException( "ReadOnlyCatalogManager does not support registering a table provider"); } + @Override + public Map tableProviders() { + return EMPTY.tableProviders; + } + @Override public void createCatalog(String name, String type, Map properties) { throw new UnsupportedOperationException( "ReadOnlyCatalogManager does not support catalog creation"); } + + @Override + public Collection catalogs() { + return ImmutableSet.of(EMPTY); + } } 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 64d2fefe2f63..a52699306fb5 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 @@ -21,24 +21,32 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Map; -import java.util.Set; +import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.checkerframework.checker.nullness.qual.Nullable; public class InMemoryCatalog implements Catalog { private final String name; private final Map properties; - private final InMemoryMetaStore metaStore = new InMemoryMetaStore(); + protected final Map tableProviders = new HashMap<>(); + private final Map metaStores = new HashMap<>(); private final HashSet databases = new HashSet<>(Collections.singleton(DEFAULT)); protected @Nullable String currentDatabase = DEFAULT; public InMemoryCatalog(String name, Map properties) { + this(name, new InMemoryMetaStore(), properties); + } + + public InMemoryCatalog(String name, MetaStore defaultMetastore, Map properties) { this.name = name; this.properties = properties; + metaStores.put(DEFAULT, defaultMetastore); } @Override @@ -53,7 +61,13 @@ public String name() { } @Override - public MetaStore metaStore() { + public MetaStore metaStore(String db) { + @Nullable MetaStore metaStore = metaStores.get(db); + if (metaStore == null) { + metaStore = new InMemoryMetaStore(); + tableProviders.values().forEach(metaStore::registerProvider); + metaStores.put(db, metaStore); + } return metaStore; } @@ -67,9 +81,14 @@ public boolean createDatabase(String database) { return databases.add(database); } + @Override + public boolean databaseExists(String db) { + return databases.contains(db); + } + @Override public void useDatabase(String database) { - checkArgument(listDatabases().contains(database), "Database '%s' does not exist."); + checkArgument(databaseExists(database), "Database '%s' does not exist."); currentDatabase = database; } @@ -90,7 +109,22 @@ public boolean dropDatabase(String database, boolean cascade) { } @Override - public Set listDatabases() { - return databases; + public void registerTableProvider(TableProvider provider) { + tableProviders.put(provider.getTableType(), provider); + metaStores.values().forEach(m -> m.registerProvider(provider)); + } + + @Override + public Map tableProviders() { + return tableProviders; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(InMemoryCatalog.class) + .add("name", name) + .add("currentDatabase", currentDatabase) + .add("databases", databases) + .toString(); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java index 84deeb96436a..2cbcb56c49ed 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java @@ -19,19 +19,21 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.ServiceLoader; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; +import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; public class InMemoryCatalogManager implements CatalogManager { private final Map catalogs = new HashMap<>(); - private final Map tableProviderMap = new HashMap<>(); + private final Map tableProviders = new HashMap<>(); private String currentCatalogName; public InMemoryCatalogManager() { @@ -39,13 +41,20 @@ public InMemoryCatalogManager() { this.currentCatalogName = "default"; } + /** To keep backwards compatibility, extends an option to set a default metastore. */ + public InMemoryCatalogManager(MetaStore defaultMetastore) { + this.catalogs.put( + "default", new InMemoryCatalog("default", defaultMetastore, Collections.emptyMap())); + this.currentCatalogName = "default"; + } + @Override public void createCatalog(String name, String type, Map properties) { Preconditions.checkState( !catalogs.containsKey(name), "Catalog with name '%s' already exists.", name); Catalog catalog = findAndCreateCatalog(name, type, properties); - tableProviderMap.values().forEach(catalog.metaStore()::registerProvider); + tableProviders.values().forEach(catalog::registerTableProvider); catalogs.put(name, catalog); } @@ -73,9 +82,14 @@ public void dropCatalog(String name) { } @Override - public void registerTableProvider(String name, TableProvider tableProvider) { - tableProviderMap.put(name, tableProvider); - catalogs.values().forEach(catalog -> catalog.metaStore().registerProvider(tableProvider)); + public void registerTableProvider(TableProvider tableProvider) { + catalogs.values().forEach(catalog -> catalog.registerTableProvider(tableProvider)); + tableProviders.put(tableProvider.getTableType(), tableProvider); + } + + @Override + public Map tableProviders() { + return tableProviders; } private Catalog findAndCreateCatalog(String name, String type, Map properties) { @@ -115,4 +129,9 @@ private Catalog createCatalogInstance( String.format("Encountered an error when constructing Catalog '%s'", name), e); } } + + @Override + public Collection catalogs() { + return catalogs.values(); + } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java index d3a8f9920c4a..83b8685c3fe9 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.extensions.sql.meta.store; -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; - import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link MetaStore} which stores the meta info in memory. @@ -55,7 +54,7 @@ public void createTable(Table table) { } // invoke the provider's create - providers.get(table.getType()).createTable(table); + getProvider(table.getType()).createTable(table); // store to the global metastore tables.put(table.getName(), table); @@ -68,7 +67,7 @@ public void dropTable(String tableName) { } Table table = tables.get(tableName); - providers.get(table.getType()).dropTable(tableName); + getProvider(table.getType()).dropTable(tableName); tables.remove(tableName); } @@ -79,26 +78,34 @@ public Map getTables() { @Override public BeamSqlTable buildBeamSqlTable(Table table) { - TableProvider provider = providers.get(table.getType()); + TableProvider provider = getProvider(table.getType()); return provider.buildBeamSqlTable(table); } - private void validateTableType(Table table) { - if (!providers.containsKey(table.getType())) { + protected void validateTableType(Table table) { + if (providers.containsKey(table.getType().toLowerCase())) { + return; + } + // check if there is a nested metastore that supports this table + @Nullable + InMemoryMetaStore nestedMemoryMetastore = (InMemoryMetaStore) providers.get(getTableType()); + if (nestedMemoryMetastore != null) { + nestedMemoryMetastore.validateTableType(table); + } else { throw new IllegalArgumentException("Table type: " + table.getType() + " not supported!"); } } @Override public void registerProvider(TableProvider provider) { - if (providers.containsKey(provider.getTableType())) { - throw new IllegalArgumentException( - "Provider is already registered for table type: " + provider.getTableType()); + String type = provider.getTableType().toLowerCase(); + if (providers.containsKey(type)) { + throw new IllegalArgumentException("Provider is already registered for table type: " + type); } initTablesFromProvider(provider); - this.providers.put(provider.getTableType(), provider); + this.providers.put(type, provider); } private void initTablesFromProvider(TableProvider provider) { @@ -112,22 +119,35 @@ private void initTablesFromProvider(TableProvider provider) { this.tables.putAll(tables); } - Map getProviders() { + @Override + public Map tableProviders() { return providers; } @Override public boolean supportsPartitioning(Table table) { - TableProvider provider = providers.get(table.getType()); - if (provider == null) { - throw new IllegalArgumentException( - "No TableProvider registered for table type: " + table.getType()); - } - return provider.supportsPartitioning(table); + return getProvider(table.getType()).supportsPartitioning(table); } + /** + * Fetches a {@link TableProvider} for this type. This provider can exist in the current {@link + * InMemoryMetaStore} or a nested {@link InMemoryMetaStore}. + * + * @param type + * @return + */ public TableProvider getProvider(String type) { - return checkArgumentNotNull( - providers.get(type), "No TableProvider registered for table type: " + type); + @Nullable TableProvider provider = providers.get(type.toLowerCase()); + if (provider != null) { + return provider; + } + + // check nested InMemoryMetaStore + provider = providers.get(getTableType()); + if (provider != null && (provider instanceof InMemoryMetaStore)) { + return ((InMemoryMetaStore) provider).getProvider(type); + } + + throw new IllegalStateException("No TableProvider registered for table type: " + type); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java index 39ad6d3dfb54..0315d45420be 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.meta.store; +import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; /** The interface to handle CRUD of {@code BeamSql} table metadata. */ @@ -27,4 +28,9 @@ public interface MetaStore extends TableProvider { * @param provider */ void registerProvider(TableProvider provider); + + /** + * Returns all the registered {@link TableProvider}s in this {@link MetaStore}, organized by type. + */ + Map tableProviders(); } 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 new file mode 100644 index 000000000000..0164c634814b --- /dev/null +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java @@ -0,0 +1,333 @@ +/* + * 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; + +import static org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog.DEFAULT; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.util.Map; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; +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.store.MetaStore; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.CalciteContextException; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +/** UnitTest for {@link BeamSqlCli} using catalogs. */ +public class BeamSqlCliCatalogTest { + @Rule public transient ExpectedException thrown = ExpectedException.none(); + private InMemoryCatalogManager catalogManager; + private BeamSqlCli cli; + + @Before + public void setupCli() { + catalogManager = new InMemoryCatalogManager(); + cli = new BeamSqlCli().catalogManager(catalogManager); + } + + @Test + public void testExecute_createCatalog_invalidTypeError() { + thrown.expect(UnsupportedOperationException.class); + thrown.expectMessage("Could not find type 'abcdef' for catalog 'invalid_catalog'."); + cli.execute("CREATE CATALOG invalid_catalog TYPE abcdef"); + } + + @Test + public void testExecute_createCatalog_duplicateCatalogError() { + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + + // this should be fine. + cli.execute("CREATE CATALOG IF NOT EXISTS my_catalog TYPE 'local'"); + + // without "IF NOT EXISTS", Beam will throw an error + thrown.expect(CalciteContextException.class); + thrown.expectMessage("Catalog 'my_catalog' already exists."); + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + } + + @Test + public void testExecute_createCatalog() { + assertNull(catalogManager.getCatalog("my_catalog")); + cli.execute( + "CREATE CATALOG my_catalog \n" + + "TYPE 'local' \n" + + "PROPERTIES (\n" + + " 'foo' = 'bar', \n" + + " 'abc' = 'xyz', \n" + + " 'beam.test.prop' = '123'\n" + + ")"); + assertNotNull(catalogManager.getCatalog("my_catalog")); + // we only created the catalog, but have not switched to it + assertNotEquals("my_catalog", catalogManager.currentCatalog().name()); + + Map expectedProps = + ImmutableMap.of( + "foo", "bar", + "abc", "xyz", + "beam.test.prop", "123"); + Catalog catalog = catalogManager.getCatalog("my_catalog"); + + assertEquals("my_catalog", catalog.name()); + assertEquals("local", catalog.type()); + assertEquals(expectedProps, catalog.properties()); + } + + @Test + public void testExecute_setCatalog_doesNotExistError() { + thrown.expect(CalciteContextException.class); + thrown.expectMessage("Cannot use catalog: 'my_catalog' not found."); + cli.execute("USE CATALOG my_catalog"); + } + + @Test + public void testExecute_setCatalog() { + assertNull(catalogManager.getCatalog("catalog_1")); + assertNull(catalogManager.getCatalog("catalog_2")); + Map catalog1Props = + ImmutableMap.of("foo", "bar", "abc", "xyz", "beam.test.prop", "123"); + Map catalog2Props = ImmutableMap.of("a", "b", "c", "d"); + cli.execute( + "CREATE CATALOG catalog_1 \n" + + "TYPE 'local' \n" + + "PROPERTIES (\n" + + " 'foo' = 'bar', \n" + + " 'abc' = 'xyz', \n" + + " 'beam.test.prop' = '123'\n" + + ")"); + cli.execute( + "CREATE CATALOG catalog_2 \n" + + "TYPE 'local' \n" + + "PROPERTIES (\n" + + " 'a' = 'b', \n" + + " 'c' = 'd' \n" + + ")"); + assertNotNull(catalogManager.getCatalog("catalog_1")); + assertNotNull(catalogManager.getCatalog("catalog_2")); + + // catalog manager always starts with a "default" catalog + assertEquals("default", catalogManager.currentCatalog().name()); + cli.execute("USE CATALOG catalog_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + assertEquals(catalog1Props, catalogManager.currentCatalog().properties()); + cli.execute("USE CATALOG catalog_2"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + assertEquals(catalog2Props, catalogManager.currentCatalog().properties()); + + // DEFAULT is a reserved keyword, so need to encapsulate in backticks + cli.execute("USE CATALOG 'default'"); + assertEquals("default", catalogManager.currentCatalog().name()); + } + + @Test + public void testExecute_dropCatalog_doesNotExistError() { + thrown.expect(CalciteContextException.class); + thrown.expectMessage("Cannot drop catalog: 'my_catalog' not found."); + cli.execute("DROP CATALOG 'my_catalog'"); + } + + @Test + public void testExecute_dropCatalog_activelyUsedError() { + thrown.expect(CalciteContextException.class); + thrown.expectMessage( + "Unable to drop active catalog 'default'. Please switch to another catalog first."); + cli.execute("DROP CATALOG 'default'"); + } + + @Test + public void testExecute_dropCatalog() { + assertNull(catalogManager.getCatalog("my_catalog")); + cli.execute( + "CREATE CATALOG my_catalog \n" + + "TYPE 'local' \n" + + "PROPERTIES (\n" + + " 'foo' = 'bar', \n" + + " 'abc' = 'xyz', \n" + + " 'beam.test.prop' = '123'\n" + + ")"); + assertNotNull(catalogManager.getCatalog("my_catalog")); + + assertNotEquals("my_catalog", catalogManager.currentCatalog().name()); + cli.execute("DROP CATALOG my_catalog"); + assertNull(catalogManager.getCatalog("my_catalog")); + } + + @Test + public void testCreateUseDropDatabaseWithSameCatalogScope() { + // create Catalog catalog_1 and create Database db_1 inside of it + cli.execute("CREATE CATALOG catalog_1 TYPE 'local'"); + cli.execute("USE CATALOG catalog_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); + cli.execute("CREATE DATABASE db_1"); + assertTrue(catalogManager.currentCatalog().databaseExists("db_1")); + cli.execute("USE DATABASE db_1"); + assertEquals("db_1", catalogManager.currentCatalog().currentDatabase()); + + // create new Catalog catalog_2 and switch to it + cli.execute("CREATE CATALOG catalog_2 TYPE 'local'"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + cli.execute("USE CATALOG catalog_2"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); + + // confirm that database 'db_1' from catalog_1 is not leaked to catalog_2 + assertFalse(catalogManager.currentCatalog().databaseExists("db_1")); + + // switch back and drop database + cli.execute("USE CATALOG catalog_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + cli.execute("DROP DATABASE db_1"); + assertFalse(catalogManager.currentCatalog().databaseExists("db_1")); + } + + @Test + public void testCreateWriteDropTableWithSameCatalogScope() { + // create and use catalog + cli.execute("CREATE CATALOG catalog_1 TYPE 'local'"); + cli.execute("USE CATALOG catalog_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); + + // create new database + cli.execute("CREATE DATABASE db_1"); + cli.execute("USE DATABASE db_1"); + assertTrue(catalogManager.currentCatalog().databaseExists("db_1")); + MetaStore metastoreDb1 = + checkStateNotNull(catalogManager.getCatalog("catalog_1")).metaStore("db_1"); + + // create new table in catalog_1, db_1 + TestTableProvider testTableProvider = new TestTableProvider(); + catalogManager.registerTableProvider(testTableProvider); + cli.execute("CREATE EXTERNAL TABLE person(id int, name varchar, age int) TYPE 'test'"); + Table table = metastoreDb1.getTable("person"); + assertNotNull(table); + + // write to table + cli.execute("INSERT INTO person VALUES(123, 'John', 34)"); + TestTableProvider.TableWithRows tableWithRows = testTableProvider.tables().get(table.getName()); + assertEquals(1, tableWithRows.getRows().size()); + Row row = tableWithRows.getRows().get(0); + Row expectedRow = + Row.withSchema( + Schema.builder() + .addNullableInt32Field("id") + .addNullableStringField("name") + .addNullableInt32Field("age") + .build()) + .addValues(123, "John", 34) + .build(); + assertEquals(expectedRow, row); + + // drop the table + cli.execute("DROP TABLE person"); + assertNull(metastoreDb1.getTable("person")); + } + + @Test + public void testCreateUseDropDatabaseWithOtherCatalogScope() { + // create two catalogs + cli.execute("CREATE CATALOG catalog_1 TYPE 'local'"); + cli.execute("CREATE CATALOG catalog_2 TYPE 'local'"); + // set default catalog_2 + cli.execute("USE CATALOG catalog_2"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); + // while using catalog_2, create new database in catalog_1 + cli.execute("CREATE DATABASE catalog_1.db_1"); + assertTrue(checkStateNotNull(catalogManager.getCatalog("catalog_1")).databaseExists("db_1")); + + // use database in catalog_2. this will override both current database (to 'deb_1') + // and current catalog (to 'catalog_1') + cli.execute("USE DATABASE catalog_1.db_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + assertEquals("db_1", catalogManager.currentCatalog().currentDatabase()); + assertTrue(catalogManager.currentCatalog().databaseExists("db_1")); + + // switch back to catalog_2 and drop + cli.execute("USE CATALOG catalog_2"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + // confirm that database 'db_1' created in catalog_1 was not leaked to catalog_2 + assertFalse(catalogManager.currentCatalog().databaseExists("db_1")); + // drop and validate + assertTrue(checkStateNotNull(catalogManager.getCatalog("catalog_1")).databaseExists("db_1")); + cli.execute("DROP DATABASE catalog_1.db_1"); + assertFalse(checkStateNotNull(catalogManager.getCatalog("catalog_1")).databaseExists("db_1")); + } + + @Test + public void testCreateWriteDropTableWithOtherCatalogScope() { + // create two catalogs + cli.execute("CREATE CATALOG catalog_1 TYPE 'local'"); + cli.execute("CREATE CATALOG catalog_2 TYPE 'local'"); + // set default catalog_2 + cli.execute("USE CATALOG catalog_2"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); + + // while using catalog_2, create new database in catalog_1 + cli.execute("CREATE DATABASE catalog_1.db_1"); + assertTrue(checkStateNotNull(catalogManager.getCatalog("catalog_1")).databaseExists("db_1")); + MetaStore metastoreDb1 = + checkStateNotNull(catalogManager.getCatalog("catalog_1")).metaStore("db_1"); + + // while using catalog_2, create new table in catalog_1, db_1 + TestTableProvider testTableProvider = new TestTableProvider(); + catalogManager.registerTableProvider(testTableProvider); + cli.execute( + "CREATE EXTERNAL TABLE catalog_1.db_1.person(id int, name varchar, age int) TYPE 'test'"); + Table table = metastoreDb1.getTable("person"); + assertNotNull(table); + // confirm we are still using catalog_2 + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + + // write to table while using catalog_2 + cli.execute("INSERT INTO catalog_1.db_1.person VALUES(123, 'John', 34)"); + TestTableProvider.TableWithRows tableWithRows = testTableProvider.tables().get(table.getName()); + assertEquals(1, tableWithRows.getRows().size()); + Row row = tableWithRows.getRows().get(0); + Row expectedRow = + Row.withSchema( + Schema.builder() + .addNullableInt32Field("id") + .addNullableStringField("name") + .addNullableInt32Field("age") + .build()) + .addValues(123, "John", 34) + .build(); + assertEquals(expectedRow, row); + // confirm we are still using catalog_2 + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + + // drop the table while using catalog_2 + cli.execute("DROP TABLE catalog_1.db_1.person"); + assertNull(metastoreDb1.getTable("person")); + } +} 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 0d93792bcad2..cca1bfd93f27 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 @@ -17,9 +17,20 @@ */ package org.apache.beam.sdk.extensions.sql; +import static org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog.DEFAULT; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; 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.schemas.Schema; +import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.CalciteContextException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.junit.Before; @@ -42,8 +53,7 @@ public void setupCli() { @Test public void testCreateDatabase() { cli.execute("CREATE DATABASE my_database"); - assertEquals( - ImmutableSet.of("default", "my_database"), catalogManager.currentCatalog().listDatabases()); + assertTrue(catalogManager.currentCatalog().databaseExists("my_database")); } @Test @@ -58,16 +68,15 @@ public void testCreateDuplicateDatabase_error() { public void testCreateDuplicateDatabase_ifNotExists() { cli.execute("CREATE DATABASE my_database"); cli.execute("CREATE DATABASE IF NOT EXISTS my_database"); - assertEquals( - ImmutableSet.of("default", "my_database"), catalogManager.currentCatalog().listDatabases()); + assertTrue(catalogManager.currentCatalog().databaseExists("my_database")); } @Test public void testUseDatabase() { - assertEquals("default", catalogManager.currentCatalog().currentDatabase()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); cli.execute("CREATE DATABASE my_database"); cli.execute("CREATE DATABASE my_database2"); - assertEquals("default", catalogManager.currentCatalog().currentDatabase()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); cli.execute("USE DATABASE my_database"); assertEquals("my_database", catalogManager.currentCatalog().currentDatabase()); cli.execute("USE DATABASE my_database2"); @@ -76,26 +85,128 @@ public void testUseDatabase() { @Test public void testUseDatabase_doesNotExist() { - assertEquals("default", catalogManager.currentCatalog().currentDatabase()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); thrown.expect(CalciteContextException.class); thrown.expectMessage("Cannot use database: 'non_existent' not found."); cli.execute("USE DATABASE non_existent"); } @Test - public void testDropDatabase() { + public void testUseDatabaseWithDeletedCatalog_notFound() { + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + cli.execute("USE CATALOG my_catalog"); cli.execute("CREATE DATABASE my_database"); + cli.execute("USE CATALOG 'default'"); + assertEquals("default", catalogManager.currentCatalog().name()); assertEquals( - ImmutableSet.of("default", "my_database"), catalogManager.currentCatalog().listDatabases()); + ImmutableSet.of("default", "my_catalog"), + catalogManager.catalogs().stream().map(Catalog::name).collect(Collectors.toSet())); + cli.execute("DROP CATALOG my_catalog"); + 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."); + cli.execute("USE DATABASE my_catalog.my_database"); + } + + @Test + public void testDropDatabase() { + cli.execute("CREATE DATABASE my_database"); + assertTrue(catalogManager.currentCatalog().databaseExists("my_database")); cli.execute("DROP DATABASE my_database"); - assertEquals(ImmutableSet.of("default"), catalogManager.currentCatalog().listDatabases()); + assertFalse(catalogManager.currentCatalog().databaseExists("my_database")); } @Test public void testDropDatabase_nonexistent() { - assertEquals(ImmutableSet.of("default"), catalogManager.currentCatalog().listDatabases()); + assertFalse(catalogManager.currentCatalog().databaseExists("my_database")); thrown.expect(CalciteContextException.class); thrown.expectMessage("Database 'my_database' does not exist."); cli.execute("DROP DATABASE my_database"); } + + @Test + public void testCreateInsertDropTableUsingDefaultDatabase() { + Catalog catalog = catalogManager.currentCatalog(); + // create new database db_1 + cli.execute("CREATE DATABASE db_1"); + assertTrue(catalog.databaseExists("db_1")); + cli.execute("USE DATABASE db_1"); + assertEquals("db_1", catalog.currentDatabase()); + + // create new table + TestTableProvider testTableProvider = new TestTableProvider(); + catalogManager.registerTableProvider(testTableProvider); + cli.execute("CREATE EXTERNAL TABLE person(id int, name varchar, age int) TYPE 'test'"); + // table should be inside the currently used database + Table table = catalog.metaStore("db_1").getTable("person"); + assertNotNull(table); + + // write to the table + cli.execute("INSERT INTO person VALUES(123, 'John', 34)"); + TestTableProvider.TableWithRows tableWithRows = testTableProvider.tables().get(table.getName()); + assertEquals(1, tableWithRows.getRows().size()); + Row row = tableWithRows.getRows().get(0); + Row expectedRow = + Row.withSchema( + Schema.builder() + .addNullableInt32Field("id") + .addNullableStringField("name") + .addNullableInt32Field("age") + .build()) + .addValues(123, "John", 34) + .build(); + assertEquals(expectedRow, row); + + // drop table, using the current database + cli.execute("DROP TABLE person"); + assertNull(catalogManager.currentCatalog().metaStore("db_1").getTable("person")); + } + + @Test + public void testCreateInsertDropTableUsingOtherDatabase() { + Catalog catalog = catalogManager.currentCatalog(); + // create database db_1 + cli.execute("CREATE DATABASE db_1"); + cli.execute("USE DATABASE db_1"); + assertEquals("db_1", catalog.currentDatabase()); + assertTrue(catalog.databaseExists("db_1")); + + // switch to other database db_2 + cli.execute("CREATE DATABASE db_2"); + cli.execute("USE DATABASE db_2"); + assertEquals("db_2", catalog.currentDatabase()); + + // create table from another database + TestTableProvider testTableProvider = new TestTableProvider(); + catalogManager.registerTableProvider(testTableProvider); + cli.execute("CREATE EXTERNAL TABLE db_1.person(id int, name varchar, age int) TYPE 'test'"); + // current database should not have the table + assertNull(catalog.metaStore("db_2").getTable("person")); + + // other database should have the table + Table table = catalog.metaStore("db_1").getTable("person"); + assertNotNull(table); + + // write to table from another database + cli.execute("INSERT INTO db_1.person VALUES(123, 'John', 34)"); + TestTableProvider.TableWithRows tableWithRows = testTableProvider.tables().get(table.getName()); + assertEquals(1, tableWithRows.getRows().size()); + Row row = tableWithRows.getRows().get(0); + Row expectedRow = + Row.withSchema( + Schema.builder() + .addNullableInt32Field("id") + .addNullableStringField("name") + .addNullableInt32Field("age") + .build()) + .addValues(123, "John", 34) + .build(); + assertEquals(expectedRow, row); + + // drop table, overriding the current database + cli.execute("DROP TABLE db_1.person"); + assertNull(catalogManager.currentCatalog().metaStore("db_1").getTable("person")); + } } 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 b8e6e90d680c..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 @@ -25,26 +25,20 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import java.time.LocalDate; import java.time.LocalTime; -import java.util.Map; 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.Catalog; -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; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.CalciteContextException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -180,7 +174,7 @@ public void testExecute_createTableWithRowField() throws Exception { + " >, \n" + "isRobot BOOLEAN" + ") \n" - + "TYPE 'text' \n" + + "TYPE 'teXt' \n" + "COMMENT '' LOCATION '/home/admin/orders'"); Table table = metaStore.getTables().get("person"); assertNotNull(table); @@ -221,7 +215,7 @@ public void testExecute_dropTable() throws Exception { + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name', \n" + "age int COMMENT 'age') \n" - + "TYPE 'text' \n" + + "TYPE 'TExt' \n" + "COMMENT '' LOCATION '/home/admin/orders'"); Table table = metaStore.getTables().get("person"); assertNotNull(table); @@ -242,182 +236,12 @@ public void testExecute_dropTable_assertTableRemovedFromPlanner() throws Excepti + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name', \n" + "age int COMMENT 'age') \n" - + "TYPE 'text' \n" + + "TYPE 'TEXT' \n" + "COMMENT '' LOCATION '/home/admin/orders'"); cli.execute("drop table person"); cli.explainQuery("select * from person"); } - @Test - public void testExecute_createCatalog_invalidTypeError() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - thrown.expect(UnsupportedOperationException.class); - thrown.expectMessage("Could not find type 'abcdef' for catalog 'invalid_catalog'."); - cli.execute("CREATE CATALOG invalid_catalog TYPE abcdef"); - } - - @Test - public void testExecute_createCatalog_duplicateCatalogError() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - - // this should be fine. - cli.execute("CREATE CATALOG IF NOT EXISTS my_catalog TYPE 'local'"); - - // without "IF NOT EXISTS", Beam will throw an error - thrown.expect(CalciteContextException.class); - thrown.expectMessage("Catalog 'my_catalog' already exists."); - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - } - - @Test - public void testExecute_createCatalog() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - assertNull(catalogManager.getCatalog("my_catalog")); - cli.execute( - "CREATE CATALOG my_catalog \n" - + "TYPE 'local' \n" - + "PROPERTIES (\n" - + " 'foo' = 'bar', \n" - + " 'abc' = 'xyz', \n" - + " 'beam.test.prop' = '123'\n" - + ")"); - assertNotNull(catalogManager.getCatalog("my_catalog")); - // we only created the catalog, but have not switched to it - assertNotEquals("my_catalog", catalogManager.currentCatalog().name()); - - Map expectedProps = - ImmutableMap.of( - "foo", "bar", - "abc", "xyz", - "beam.test.prop", "123"); - Catalog catalog = catalogManager.getCatalog("my_catalog"); - - assertEquals("my_catalog", catalog.name()); - assertEquals("local", catalog.type()); - assertEquals(expectedProps, catalog.properties()); - } - - @Test - public void testExecute_setCatalog_doesNotExistError() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - thrown.expect(CalciteContextException.class); - thrown.expectMessage("Cannot use catalog: 'my_catalog' not found."); - cli.execute("USE CATALOG my_catalog"); - } - - @Test - public void testExecute_setCatalog() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - assertNull(catalogManager.getCatalog("catalog_1")); - assertNull(catalogManager.getCatalog("catalog_2")); - Map catalog1Props = - ImmutableMap.of("foo", "bar", "abc", "xyz", "beam.test.prop", "123"); - Map catalog2Props = ImmutableMap.of("a", "b", "c", "d"); - cli.execute( - "CREATE CATALOG catalog_1 \n" - + "TYPE 'local' \n" - + "PROPERTIES (\n" - + " 'foo' = 'bar', \n" - + " 'abc' = 'xyz', \n" - + " 'beam.test.prop' = '123'\n" - + ")"); - cli.execute( - "CREATE CATALOG catalog_2 \n" - + "TYPE 'local' \n" - + "PROPERTIES (\n" - + " 'a' = 'b', \n" - + " 'c' = 'd' \n" - + ")"); - assertNotNull(catalogManager.getCatalog("catalog_1")); - assertNotNull(catalogManager.getCatalog("catalog_2")); - - // catalog manager always starts with a "default" catalog - assertEquals("default", catalogManager.currentCatalog().name()); - cli.execute("USE CATALOG catalog_1"); - assertEquals("catalog_1", catalogManager.currentCatalog().name()); - assertEquals(catalog1Props, catalogManager.currentCatalog().properties()); - cli.execute("USE CATALOG catalog_2"); - assertEquals("catalog_2", catalogManager.currentCatalog().name()); - assertEquals(catalog2Props, catalogManager.currentCatalog().properties()); - - // DEFAULT is a reserved keyword, so need to encapsulate in backticks - cli.execute("USE CATALOG 'default'"); - assertEquals("default", catalogManager.currentCatalog().name()); - } - - @Test - public void testExecute_dropCatalog_doesNotExistError() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - thrown.expect(CalciteContextException.class); - thrown.expectMessage("Cannot drop catalog: 'my_catalog' not found."); - cli.execute("DROP CATALOG 'my_catalog'"); - } - - @Test - public void testExecute_dropCatalog_activelyUsedError() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - thrown.expect(CalciteContextException.class); - thrown.expectMessage( - "Unable to drop active catalog 'default'. Please switch to another catalog first."); - cli.execute("DROP CATALOG 'default'"); - } - - @Test - public void testExecute_dropCatalog() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - assertNull(catalogManager.getCatalog("my_catalog")); - cli.execute( - "CREATE CATALOG my_catalog \n" - + "TYPE 'local' \n" - + "PROPERTIES (\n" - + " 'foo' = 'bar', \n" - + " 'abc' = 'xyz', \n" - + " 'beam.test.prop' = '123'\n" - + ")"); - assertNotNull(catalogManager.getCatalog("my_catalog")); - - assertNotEquals("my_catalog", catalogManager.currentCatalog().name()); - cli.execute("DROP CATALOG my_catalog"); - assertNull(catalogManager.getCatalog("my_catalog")); - } - - @Test - public void testExecute_tableScopeAcrossCatalogs() throws Exception { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - catalogManager.registerTableProvider(new TextTableProvider()); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - cli.execute("USE CATALOG my_catalog"); - cli.execute( - "CREATE EXTERNAL TABLE person (\n" + "id int, name varchar, age int) \n" + "TYPE 'text'"); - - assertEquals("my_catalog", catalogManager.currentCatalog().name()); - assertNotNull(catalogManager.currentCatalog().metaStore().getTables().get("person")); - - cli.execute("CREATE CATALOG my_other_catalog TYPE 'local'"); - cli.execute("USE CATALOG my_other_catalog"); - assertEquals("my_other_catalog", catalogManager.currentCatalog().name()); - assertNull(catalogManager.currentCatalog().metaStore().getTables().get("person")); - } - @Test public void testExplainQuery() throws Exception { InMemoryMetaStore metaStore = new InMemoryMetaStore(); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java index b9aa4ae2ecc7..2265759126fb 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java @@ -116,9 +116,9 @@ public void testDriverManager_simple() throws Exception { public void testDriverManager_defaultUserAgent() throws Exception { Connection connection = DriverManager.getConnection(JdbcDriver.CONNECT_STRING_PREFIX); SchemaPlus rootSchema = ((CalciteConnection) connection).getRootSchema(); - BeamCalciteSchema beamSchema = - (BeamCalciteSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; - Map pipelineOptions = beamSchema.getPipelineOptions(); + CatalogManagerSchema catalogManagerSchema = + (CatalogManagerSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; + Map pipelineOptions = catalogManagerSchema.connection().getPipelineOptionsMap(); assertThat(pipelineOptions.get("userAgent"), containsString("BeamSQL")); } @@ -127,9 +127,9 @@ public void testDriverManager_defaultUserAgent() throws Exception { public void testDriverManager_hasUserAgent() throws Exception { JdbcConnection connection = (JdbcConnection) DriverManager.getConnection(JdbcDriver.CONNECT_STRING_PREFIX); - BeamCalciteSchema schema = connection.getCurrentBeamSchema(); + CatalogManagerSchema schema = connection.getCurrentBeamSchema(); assertThat( - schema.getPipelineOptions().get("userAgent"), + schema.connection().getPipelineOptionsMap().get("userAgent"), equalTo("BeamSQL/" + ReleaseInfo.getReleaseInfo().getVersion())); } @@ -140,9 +140,9 @@ public void testDriverManager_setUserAgent() throws Exception { DriverManager.getConnection( JdbcDriver.CONNECT_STRING_PREFIX + "beam.userAgent=Secret Agent"); SchemaPlus rootSchema = ((CalciteConnection) connection).getRootSchema(); - BeamCalciteSchema beamSchema = - (BeamCalciteSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; - Map pipelineOptions = beamSchema.getPipelineOptions(); + CatalogManagerSchema catalogManagerSchema = + (CatalogManagerSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; + Map pipelineOptions = catalogManagerSchema.connection().getPipelineOptionsMap(); assertThat(pipelineOptions.get("userAgent"), equalTo("Secret Agent")); } @@ -154,9 +154,9 @@ public void testDriverManager_pipelineOptionsPlumbing() throws Exception { JdbcDriver.CONNECT_STRING_PREFIX + "beam.foo=baz;beam.foobizzle=mahshizzle;other=smother"); SchemaPlus rootSchema = ((CalciteConnection) connection).getRootSchema(); - BeamCalciteSchema beamSchema = - (BeamCalciteSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; - Map pipelineOptions = beamSchema.getPipelineOptions(); + CatalogManagerSchema catalogManagerSchema = + (CatalogManagerSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; + Map pipelineOptions = catalogManagerSchema.connection().getPipelineOptionsMap(); assertThat(pipelineOptions.get("foo"), equalTo("baz")); assertThat(pipelineOptions.get("foobizzle"), equalTo("mahshizzle")); assertThat(pipelineOptions.get("other"), nullValue()); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java index e9daf57816bf..83d97bda2e91 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java @@ -75,7 +75,7 @@ private Table executeCreateTableWith(String fieldType) throws SqlParseException + "fieldName " + fieldType + " ) " - + "TYPE 'text' " + + "TYPE 'test' " + "LOCATION '/home/admin/person'\n"; System.out.println(createTable); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java index 518a830041e2..e465ce44d056 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java @@ -61,13 +61,13 @@ public void testParseCreateExternalTable_full() throws Exception { "CREATE EXTERNAL TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "COMMENT 'person table' \n" + "LOCATION '/home/admin/person'\n" + "TBLPROPERTIES '{\"hello\": [\"james\", \"bond\"]}'"); assertEquals( - mockTable("person", "text", "person table", properties), + mockTable("person", "test", "person table", properties), tableProvider.getTables().get("person")); } @@ -80,7 +80,7 @@ public void testParseCreateExternalTable_WithComplexFields() { "CREATE EXTERNAL TABLE PersonDetails" + " ( personInfo MAP> , " + " additionalInfo ROW )" - + " TYPE 'text'" + + " TYPE 'test'" + " LOCATION '/home/admin/person'"); assertNotNull(tableProvider.getTables().get("PersonDetails")); @@ -105,7 +105,7 @@ public void testParseCreateTable() throws Exception { "CREATE TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "COMMENT 'person table' \n" + "LOCATION '/home/admin/person'\n" + "TBLPROPERTIES '{\"hello\": [\"james\", \"bond\"]}'"); @@ -126,11 +126,11 @@ public void testParseCreateExternalTable_withoutTableComment() throws Exception "CREATE EXTERNAL TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "LOCATION '/home/admin/person'\n" + "TBLPROPERTIES '{\"hello\": [\"james\", \"bond\"]}'"); assertEquals( - mockTable("person", "text", null, properties), tableProvider.getTables().get("person")); + mockTable("person", "test", null, properties), tableProvider.getTables().get("person")); } @Test @@ -142,11 +142,11 @@ public void testParseCreateExternalTable_withoutTblProperties() throws Exception "CREATE EXTERNAL TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "COMMENT 'person table' \n" + "LOCATION '/home/admin/person'\n"); assertEquals( - mockTable("person", "text", "person table", TableUtils.emptyProperties()), + mockTable("person", "test", "person table", TableUtils.emptyProperties()), tableProvider.getTables().get("person")); } @@ -159,11 +159,11 @@ public void testParseCreateExternalTable_withoutLocation() throws Exception { "CREATE EXTERNAL TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "COMMENT 'person table' \n"); assertEquals( - mockTable("person", "text", "person table", TableUtils.emptyProperties(), null), + mockTable("person", "test", "person table", TableUtils.emptyProperties(), null), tableProvider.getTables().get("person")); } @@ -172,12 +172,12 @@ public void testParseCreateExternalTable_minimal() throws Exception { TestTableProvider tableProvider = new TestTableProvider(); BeamSqlEnv env = BeamSqlEnv.withTableProvider(tableProvider); - env.executeDdl("CREATE EXTERNAL TABLE person (id INT) TYPE text"); + env.executeDdl("CREATE EXTERNAL TABLE person (id INT) TYPE test"); assertEquals( Table.builder() .name("person") - .type("text") + .type("test") .schema( Stream.of(Schema.Field.of("id", CalciteUtils.INTEGER).withNullable(true)) .collect(toSchema())) @@ -197,7 +197,7 @@ public void testParseCreateExternalTable_withDatabase() throws Exception { .setPipelineOptions(PipelineOptionsFactory.create()) .build(); assertNull(testProvider.getTables().get("person")); - env.executeDdl("CREATE EXTERNAL TABLE test.person (id INT) TYPE text"); + env.executeDdl("CREATE EXTERNAL TABLE test.person (id INT) TYPE test"); assertNotNull(testProvider.getTables().get("person")); } @@ -212,7 +212,7 @@ public void testParseDropTable() throws Exception { "CREATE EXTERNAL TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "COMMENT 'person table' \n"); assertNotNull(tableProvider.getTables().get("person")); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java index 5ba74e88acc3..e964ec0a992a 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java @@ -31,11 +31,13 @@ public abstract class BaseRelTest { protected static BeamSqlEnv env = BeamSqlEnv.readOnly("test", tables); protected static PCollection compilePipeline(String sql, Pipeline pipeline) { + env = BeamSqlEnv.readOnly("test", tables); return BeamSqlRelUtils.toPCollection(pipeline, env.parseQuery(sql)); } protected static void registerTable(String tableName, BeamSqlTable table) { tables.put(tableName, table); + env = BeamSqlEnv.readOnly("test", tables); } protected static BeamSqlTable getTable(String tableName) { diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java index 92b77ec9efbd..71b12145e81c 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java @@ -327,20 +327,20 @@ private void assertTopTableInJoins(RelNode parsedQuery, String expectedTableName private void createThreeTables(TestTableProvider tableProvider) { BeamSqlEnv env = BeamSqlEnv.withTableProvider(tableProvider); - env.executeDdl("CREATE EXTERNAL TABLE small_table (id INTEGER, medium_key INTEGER) TYPE text"); + env.executeDdl("CREATE EXTERNAL TABLE small_table (id INTEGER, medium_key INTEGER) TYPE test"); env.executeDdl( "CREATE EXTERNAL TABLE medium_table (" + "id INTEGER," + "small_key INTEGER," + "large_key INTEGER" - + ") TYPE text"); + + ") TYPE test"); env.executeDdl( "CREATE EXTERNAL TABLE large_table (" + "id INTEGER," + "medium_key INTEGER" - + ") TYPE text"); + + ") TYPE test"); Row row = Row.withSchema(tableProvider.getTable("small_table").getSchema()).addValues(1, 1).build(); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java index 825f3ed06485..ea41490c8d00 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java @@ -94,10 +94,10 @@ public void testBuildBeamSqlTable() throws Exception { @Test public void testRegisterProvider() throws Exception { store.registerProvider(new MockTableProvider("mock", "hello", "world")); - assertNotNull(store.getProviders()); - assertEquals(2, store.getProviders().size()); - assertEquals("text", store.getProviders().get("text").getTableType()); - assertEquals("mock", store.getProviders().get("mock").getTableType()); + assertNotNull(store.tableProviders()); + assertEquals(2, store.tableProviders().size()); + assertEquals("text", store.tableProviders().get("text").getTableType()); + assertEquals("mock", store.tableProviders().get("mock").getTableType()); assertEquals(2, store.getTables().size()); } @@ -119,6 +119,7 @@ private static Table mockTable(String name, String type) { .name(name) .comment(name + " table") .location("/home/admin/" + name) + // .path("default.default." + name) .schema( Stream.of( Schema.Field.nullable("id", Schema.FieldType.INT32), diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java index 96357b44e54b..7603e2c6259f 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java @@ -32,11 +32,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchTableException; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; @@ -109,6 +111,11 @@ public boolean createNamespace(String namespace) { } } + public boolean namespaceExists(String namespace) { + checkSupportsNamespaces(); + return ((SupportsNamespaces) catalog()).namespaceExists(Namespace.of(namespace)); + } + public Set listNamespaces() { checkSupportsNamespaces(); @@ -141,17 +148,46 @@ public void createTable( org.apache.iceberg.Schema icebergSchema = IcebergUtils.beamSchemaToIcebergSchema(tableSchema); PartitionSpec icebergSpec = PartitionUtils.toPartitionSpec(partitionFields, tableSchema); try { - catalog().createTable(icebergIdentifier, icebergSchema, icebergSpec); LOG.info( - "Created table '{}' with schema: {}\n, partition spec: {}", + "Attempting to create table '{}', with schema: {}, partition spec: {}.", icebergIdentifier, icebergSchema, icebergSpec); + catalog().createTable(icebergIdentifier, icebergSchema, icebergSpec); + LOG.info("Successfully created table '{}'.", icebergIdentifier); } catch (AlreadyExistsException e) { throw new TableAlreadyExistsException(e); } } + public @Nullable IcebergTableInfo loadTable(String tableIdentifier) { + TableIdentifier icebergIdentifier = TableIdentifier.parse(tableIdentifier); + try { + Table table = catalog().loadTable(icebergIdentifier); + return IcebergTableInfo.create( + tableIdentifier, + IcebergUtils.icebergSchemaToBeamSchema(table.schema()), + table.properties()); + } catch (NoSuchTableException ignored) { + return null; + } + } + + // Helper class to pass information to Beam SQL module without relying on Iceberg deps + @AutoValue + public abstract static class IcebergTableInfo { + public abstract String getIdentifier(); + + public abstract Schema getSchema(); + + public abstract Map getProperties(); + + static IcebergTableInfo create( + String identifier, Schema schema, Map properties) { + return new AutoValue_IcebergCatalogConfig_IcebergTableInfo(identifier, schema, properties); + }; + } + public boolean dropTable(String tableIdentifier) { TableIdentifier icebergIdentifier = TableIdentifier.parse(tableIdentifier); return catalog().dropTable(icebergIdentifier); diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java index 9f3b68afc451..fe8db05d2be7 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.tpcds; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import com.fasterxml.jackson.databind.node.ObjectNode; import java.util.ArrayList; @@ -35,6 +36,7 @@ import org.apache.beam.sdk.extensions.sql.impl.BeamSqlPipelineOptions; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider; import org.apache.beam.sdk.io.TextIO; @@ -117,7 +119,8 @@ private static void registerAllTablesByInMemoryMetaStore( .properties(properties) .type("text") .build(); - inMemoryCatalogManager.currentCatalog().metaStore().createTable(table); + Catalog catalog = inMemoryCatalogManager.currentCatalog(); + catalog.metaStore(checkStateNotNull(catalog.currentDatabase())).createTable(table); } }