gaborgsomogyi commented on a change in pull request #2666: URL: https://github.com/apache/iceberg/pull/2666#discussion_r650113649
########## File path: flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java ########## @@ -0,0 +1,136 @@ +/* + * 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.iceberg.flink; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergConnector extends FlinkTestBase { + + private static TableEnvironment tEnv; + + @Rule + public final TemporaryFolder warehouse = new TemporaryFolder(); + Review comment: Nit: Extra newline can be removed. ########## File path: flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java ########## @@ -0,0 +1,136 @@ +/* + * 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.iceberg.flink; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergConnector extends FlinkTestBase { + + private static TableEnvironment tEnv; + + @Rule + public final TemporaryFolder warehouse = new TemporaryFolder(); + + + @BeforeClass + public static void beforeClass() { + EnvironmentSettings settings = EnvironmentSettings + .newInstance() + .useBlinkPlanner() + .inBatchMode() + .build(); + + tEnv = TableEnvironment.create(settings); + tEnv.getConfig().getConfiguration() + .set(CoreOptions.DEFAULT_PARALLELISM, 1) + .set(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + } + + @Test + public void testHadoopCatalog() { + Map<String, String> tableProps = Maps.newHashMap(); + tableProps.put("connector", "iceberg"); + tableProps.put("catalog-name", "test-hadoop"); + tableProps.put("catalog-type", "hadoop"); + tableProps.put("catalog-database", "local_db"); + tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseRoot()); + + sql("CREATE TABLE hadoop_table (id BIGINT, data STRING) WITH %s", toWithClause(tableProps)); + sql("INSERT INTO hadoop_table VALUES (1, 'aaa'), (2, 'bbb'), (3, 'ccc')"); + Assert.assertEquals("Should have expected rows", + Lists.newArrayList(Row.of(1L, "aaa"), Row.of(2L, "bbb"), Row.of(3L, "ccc")), + sql("SELECT * FROM hadoop_table")); + + // Drop and create it again. + sql("DROP TABLE hadoop_table"); + sql("CREATE TABLE hadoop_table (id BIGINT, data STRING) WITH %s", toWithClause(tableProps)); + Assert.assertEquals("Should have expected rows", + Lists.newArrayList(Row.of(1L, "aaa"), Row.of(2L, "bbb"), Row.of(3L, "ccc")), + sql("SELECT * FROM hadoop_table")); + + sql("DROP TABLE hadoop_table"); + } + + @Test + public void testHiveCatalog() { + Map<String, String> tableProps = Maps.newHashMap(); + tableProps.put("connector", "iceberg"); + tableProps.put("catalog-name", "test-hive"); + tableProps.put("catalog-type", "hive"); + tableProps.put("catalog-database", "default"); + tableProps.put(CatalogProperties.URI, FlinkCatalogTestBase.getURI(hiveConf)); + tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseRoot()); + + sql("CREATE TABLE hive_table(id BIGINT, data STRING) WITH %s", FlinkCatalogTestBase.toWithClause(tableProps)); + sql("INSERT INTO hive_table VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')"); + Assert.assertEquals("Should have the expected rows", + Lists.newArrayList(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), + sql("SELECT * FROM hive_table")); + + // Drop and create it again. + sql("DROP TABLE hive_table"); + sql("CREATE TABLE hive_table (id BIGINT, data STRING) WITH %s", toWithClause(tableProps)); + Assert.assertEquals("Should have expected rows", + Lists.newArrayList(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), + sql("SELECT * FROM hive_table")); + + sql("DROP TABLE hive_table"); + } + + private String toWithClause(Map<String, String> props) { + return FlinkCatalogTestBase.toWithClause(props); + } + + private String warehouseRoot() { + return String.format("file://%s", warehouse.getRoot().getAbsolutePath()); + } + + protected List<Row> sql(String query, Object... args) { Review comment: +1 on double checking whether it's needed at all. When I've removed the test still passed. If there is an objective reason why this needed the `@Override` must be added. ########## File path: flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java ########## @@ -0,0 +1,136 @@ +/* + * 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.iceberg.flink; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergConnector extends FlinkTestBase { + + private static TableEnvironment tEnv; + + @Rule + public final TemporaryFolder warehouse = new TemporaryFolder(); + + + @BeforeClass + public static void beforeClass() { + EnvironmentSettings settings = EnvironmentSettings + .newInstance() + .useBlinkPlanner() + .inBatchMode() Review comment: I've double checked it works with `inStreamingMode()`. It would be good to add parameterized tests which runs in both modes. ########## File path: flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java ########## @@ -19,56 +19,150 @@ package org.apache.iceberg.flink; +import java.util.Map; import java.util.Set; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.factories.DynamicTableSinkFactory; import org.apache.flink.table.factories.DynamicTableSourceFactory; import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; public class FlinkDynamicTableFactory implements DynamicTableSinkFactory, DynamicTableSourceFactory { + private static final String FACTORY_IDENTIFIER = "iceberg"; + + public static final ConfigOption<String> CATALOG_NAME = + ConfigOptions.key("catalog-name") + .stringType() + .noDefaultValue() + .withDescription("Catalog name"); + + public static final ConfigOption<String> CATALOG_TYPE = + ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE) + .stringType() + .noDefaultValue() + .withDescription("Catalog type."); + + public static final ConfigOption<String> CATALOG_DATABASE = + ConfigOptions.key("catalog-database") + .stringType() + .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_VALUE) + .withDeprecatedKeys("Catalog database"); + private final FlinkCatalog catalog; + public FlinkDynamicTableFactory() { Review comment: I've double checked and when default constructor removed the tests fail with the mentioned issue so it's needed. ########## File path: flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java ########## @@ -19,56 +19,143 @@ package org.apache.iceberg.flink; +import java.util.Map; import java.util.Set; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.factories.DynamicTableSinkFactory; import org.apache.flink.table.factories.DynamicTableSourceFactory; import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; public class FlinkDynamicTableFactory implements DynamicTableSinkFactory, DynamicTableSourceFactory { + private static final String FACTORY_IDENTIFIER = "iceberg"; + + private static final ConfigOption<String> CATALOG_NAME = + ConfigOptions.key("catalog-name") + .stringType() + .noDefaultValue() + .withDescription("Catalog name"); + + private static final ConfigOption<String> CATALOG_TYPE = + ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE) + .stringType() + .noDefaultValue() + .withDescription("Catalog type."); + + private static final ConfigOption<String> CATALOG_DATABASE = + ConfigOptions.key("catalog-database") + .stringType() + .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_VALUE) + .withDeprecatedKeys("Catalog database"); + private final FlinkCatalog catalog; + public FlinkDynamicTableFactory() { + this.catalog = null; + } + public FlinkDynamicTableFactory(FlinkCatalog catalog) { this.catalog = catalog; } @Override public DynamicTableSource createDynamicTableSource(Context context) { - ObjectPath objectPath = context.getObjectIdentifier().toObjectPath(); - TableLoader tableLoader = createTableLoader(objectPath); + ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); + Map<String, String> tableProps = context.getCatalogTable().getOptions(); + CatalogTable catalogTable = context.getCatalogTable(); TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); - return new IcebergTableSource(tableLoader, tableSchema, context.getCatalogTable().getOptions(), - context.getConfiguration()); + + TableLoader tableLoader; + if (catalog != null) { + tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); + } else { + tableLoader = createTableLoader(catalogTable, tableProps, objectIdentifier.getObjectName()); + } + + return new IcebergTableSource(tableLoader, tableSchema, tableProps, context.getConfiguration()); } @Override public DynamicTableSink createDynamicTableSink(Context context) { ObjectPath objectPath = context.getObjectIdentifier().toObjectPath(); - TableLoader tableLoader = createTableLoader(objectPath); + Map<String, String> tableProps = context.getCatalogTable().getOptions(); + CatalogTable catalogTable = context.getCatalogTable(); TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); + + TableLoader tableLoader; + if (catalog != null) { + tableLoader = createTableLoader(catalog, objectPath); + } else { + tableLoader = createTableLoader(catalogTable, tableProps, objectPath.getObjectName()); + } + return new IcebergTableSink(tableLoader, tableSchema); } @Override public Set<ConfigOption<?>> requiredOptions() { - throw new UnsupportedOperationException("Iceberg Table Factory can not be loaded from Java SPI"); + Set<ConfigOption<?>> options = Sets.newHashSet(); + options.add(CATALOG_TYPE); + options.add(CATALOG_NAME); + options.add(CATALOG_DATABASE); + return options; } @Override public Set<ConfigOption<?>> optionalOptions() { - throw new UnsupportedOperationException("Iceberg Table Factory can not be loaded from Java SPI"); + return Sets.newHashSet(); } @Override public String factoryIdentifier() { - throw new UnsupportedOperationException("Iceberg Table Factory can not be loaded from Java SPI"); + return FACTORY_IDENTIFIER; + } + + private static TableLoader createTableLoader(CatalogBaseTable catalogTable, + Map<String, String> tableProps, + String tableName) { + Configuration flinkConf = new Configuration(); + tableProps.forEach(flinkConf::setString); + + String catalogName = flinkConf.getString(CATALOG_NAME); + Preconditions.checkNotNull(catalogName, "Table property '%s' cannot be null", CATALOG_NAME.key()); + + String catalogDatabase = flinkConf.getString(CATALOG_DATABASE); + Preconditions.checkNotNull(catalogDatabase, "Table property '%s' cannot be null", CATALOG_DATABASE.key()); + + org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf(); + CatalogLoader catalogLoader = FlinkCatalogFactory.createCatalogLoader(catalogName, tableProps, hadoopConf); Review comment: @stevenzwu please see up. In this PR `createCatalogLoader` is made static. +1 not making static. ########## File path: flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java ########## @@ -0,0 +1,136 @@ +/* + * 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.iceberg.flink; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergConnector extends FlinkTestBase { + + private static TableEnvironment tEnv; + + @Rule + public final TemporaryFolder warehouse = new TemporaryFolder(); + + + @BeforeClass + public static void beforeClass() { + EnvironmentSettings settings = EnvironmentSettings + .newInstance() + .useBlinkPlanner() + .inBatchMode() + .build(); + + tEnv = TableEnvironment.create(settings); + tEnv.getConfig().getConfiguration() + .set(CoreOptions.DEFAULT_PARALLELISM, 1) + .set(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + } + + @Test + public void testHadoopCatalog() { + Map<String, String> tableProps = Maps.newHashMap(); + tableProps.put("connector", "iceberg"); + tableProps.put("catalog-name", "test-hadoop"); + tableProps.put("catalog-type", "hadoop"); + tableProps.put("catalog-database", "local_db"); + tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseRoot()); + + sql("CREATE TABLE hadoop_table (id BIGINT, data STRING) WITH %s", toWithClause(tableProps)); + sql("INSERT INTO hadoop_table VALUES (1, 'aaa'), (2, 'bbb'), (3, 'ccc')"); + Assert.assertEquals("Should have expected rows", + Lists.newArrayList(Row.of(1L, "aaa"), Row.of(2L, "bbb"), Row.of(3L, "ccc")), + sql("SELECT * FROM hadoop_table")); + + // Drop and create it again. + sql("DROP TABLE hadoop_table"); + sql("CREATE TABLE hadoop_table (id BIGINT, data STRING) WITH %s", toWithClause(tableProps)); + Assert.assertEquals("Should have expected rows", + Lists.newArrayList(Row.of(1L, "aaa"), Row.of(2L, "bbb"), Row.of(3L, "ccc")), + sql("SELECT * FROM hadoop_table")); + + sql("DROP TABLE hadoop_table"); + } + + @Test + public void testHiveCatalog() { + Map<String, String> tableProps = Maps.newHashMap(); + tableProps.put("connector", "iceberg"); + tableProps.put("catalog-name", "test-hive"); + tableProps.put("catalog-type", "hive"); + tableProps.put("catalog-database", "default"); + tableProps.put(CatalogProperties.URI, FlinkCatalogTestBase.getURI(hiveConf)); + tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseRoot()); + + sql("CREATE TABLE hive_table(id BIGINT, data STRING) WITH %s", FlinkCatalogTestBase.toWithClause(tableProps)); + sql("INSERT INTO hive_table VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')"); + Assert.assertEquals("Should have the expected rows", + Lists.newArrayList(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), + sql("SELECT * FROM hive_table")); + + // Drop and create it again. + sql("DROP TABLE hive_table"); + sql("CREATE TABLE hive_table (id BIGINT, data STRING) WITH %s", toWithClause(tableProps)); + Assert.assertEquals("Should have expected rows", + Lists.newArrayList(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), + sql("SELECT * FROM hive_table")); + + sql("DROP TABLE hive_table"); + } + + private String toWithClause(Map<String, String> props) { + return FlinkCatalogTestBase.toWithClause(props); + } + + private String warehouseRoot() { + return String.format("file://%s", warehouse.getRoot().getAbsolutePath()); + } + + protected List<Row> sql(String query, Object... args) { Review comment: +1 on double checking whether it's needed at all. When I've removed the test still passed. If there is an objective reason why this needed then `@Override` must be added. ########## File path: flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java ########## @@ -0,0 +1,136 @@ +/* + * 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.iceberg.flink; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergConnector extends FlinkTestBase { + + private static TableEnvironment tEnv; + + @Rule + public final TemporaryFolder warehouse = new TemporaryFolder(); + + + @BeforeClass + public static void beforeClass() { + EnvironmentSettings settings = EnvironmentSettings + .newInstance() + .useBlinkPlanner() + .inBatchMode() + .build(); + + tEnv = TableEnvironment.create(settings); + tEnv.getConfig().getConfiguration() + .set(CoreOptions.DEFAULT_PARALLELISM, 1) + .set(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + } + + @Test + public void testHadoopCatalog() { + Map<String, String> tableProps = Maps.newHashMap(); + tableProps.put("connector", "iceberg"); + tableProps.put("catalog-name", "test-hadoop"); + tableProps.put("catalog-type", "hadoop"); + tableProps.put("catalog-database", "local_db"); + tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseRoot()); + + sql("CREATE TABLE hadoop_table (id BIGINT, data STRING) WITH %s", toWithClause(tableProps)); + sql("INSERT INTO hadoop_table VALUES (1, 'aaa'), (2, 'bbb'), (3, 'ccc')"); + Assert.assertEquals("Should have expected rows", + Lists.newArrayList(Row.of(1L, "aaa"), Row.of(2L, "bbb"), Row.of(3L, "ccc")), + sql("SELECT * FROM hadoop_table")); + + // Drop and create it again. + sql("DROP TABLE hadoop_table"); + sql("CREATE TABLE hadoop_table (id BIGINT, data STRING) WITH %s", toWithClause(tableProps)); + Assert.assertEquals("Should have expected rows", + Lists.newArrayList(Row.of(1L, "aaa"), Row.of(2L, "bbb"), Row.of(3L, "ccc")), + sql("SELECT * FROM hadoop_table")); + + sql("DROP TABLE hadoop_table"); + } + + @Test + public void testHiveCatalog() { + Map<String, String> tableProps = Maps.newHashMap(); + tableProps.put("connector", "iceberg"); + tableProps.put("catalog-name", "test-hive"); + tableProps.put("catalog-type", "hive"); + tableProps.put("catalog-database", "default"); + tableProps.put(CatalogProperties.URI, FlinkCatalogTestBase.getURI(hiveConf)); + tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseRoot()); + + sql("CREATE TABLE hive_table(id BIGINT, data STRING) WITH %s", FlinkCatalogTestBase.toWithClause(tableProps)); + sql("INSERT INTO hive_table VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')"); + Assert.assertEquals("Should have the expected rows", + Lists.newArrayList(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), + sql("SELECT * FROM hive_table")); + + // Drop and create it again. + sql("DROP TABLE hive_table"); + sql("CREATE TABLE hive_table (id BIGINT, data STRING) WITH %s", toWithClause(tableProps)); + Assert.assertEquals("Should have expected rows", + Lists.newArrayList(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), + sql("SELECT * FROM hive_table")); + + sql("DROP TABLE hive_table"); + } + + private String toWithClause(Map<String, String> props) { + return FlinkCatalogTestBase.toWithClause(props); + } + + private String warehouseRoot() { + return String.format("file://%s", warehouse.getRoot().getAbsolutePath()); + } + + protected List<Row> sql(String query, Object... args) { Review comment: +1 on double checking whether it's needed at all. When I've removed the whole function then the test still passed. If there is an objective reason why this needed then `@Override` must be added. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
