stevenzwu commented on a change in pull request #2666:
URL: https://github.com/apache/iceberg/pull/2666#discussion_r657516562



##########
File path: 
flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.Map;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+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.After;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestIcebergConnector extends FlinkTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+
+  @Rule
+  public final TemporaryFolder warehouse = new TemporaryFolder();
+
+  private final boolean isStreaming;
+  private volatile TableEnvironment tEnv;
+
+  @Parameterized.Parameters(name = "isStreaming={0}")
+  public static Iterable<Object[]> parameters() {
+    return Lists.newArrayList(
+        new Object[] {true},
+        new Object[] {false}
+    );
+  }
+
+  public TestIcebergConnector(boolean isStreaming) {
+    this.isStreaming = isStreaming;
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    if (tEnv == null) {
+      synchronized (this) {
+        if (tEnv == null) {
+          EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings
+              .newInstance()
+              .useBlinkPlanner();
+          if (isStreaming) {
+            settingsBuilder.inStreamingMode();
+            StreamExecutionEnvironment env = StreamExecutionEnvironment
+                
.getExecutionEnvironment(MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG);
+            env.enableCheckpointing(400);
+            env.setMaxParallelism(2);
+            env.setParallelism(2);
+            tEnv = StreamTableEnvironment.create(env, settingsBuilder.build());
+          } else {
+            settingsBuilder.inBatchMode();
+            tEnv = TableEnvironment.create(settingsBuilder.build());
+          }
+          // Set only one parallelism.
+          tEnv.getConfig().getConfiguration()
+              .set(CoreOptions.DEFAULT_PARALLELISM, 1)
+              
.set(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false);
+        }
+      }
+    }
+    return tEnv;
+  }
+
+  @After
+  public void after() {
+    sql("DROP TABLE IF EXISTS %s", TABLE_NAME);
+  }
+
+  @Test
+  public void testHadoop() {
+    Map<String, String> tableProps = Maps.newHashMap();
+    tableProps.put("connector", "iceberg");
+    tableProps.put("catalog-name", "test-hadoop");
+    tableProps.put("catalog-type", "hadoop");
+    tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseRoot());
+
+    sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, 
toWithClause(tableProps));
+    sql("INSERT INTO %s VALUES (1, 'aaa'), (2, 'bbb'), (3, 'ccc')", 
TABLE_NAME);
+    Assert.assertEquals("Should have expected rows",
+        Lists.newArrayList(Row.of(1L, "aaa"), Row.of(2L, "bbb"), Row.of(3L, 
"ccc")),
+        sql("SELECT * FROM %s", TABLE_NAME));
+
+    // Drop and create it again.
+    sql("DROP TABLE %s", TABLE_NAME);
+    sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, 
toWithClause(tableProps));
+    Assert.assertEquals("Should have expected rows",

Review comment:
       drop table doesn't purge data and metadata files, right? table creation 
again will be able to see the old data. I guess it is to prevent the disaster 
from accidental "drop table"?




-- 
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]

Reply via email to