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



##########
File path: 
flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.List;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.SqlDialect;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.hive.HiveCatalog;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+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 TestMigrateAction extends FlinkCatalogTestBase {
+
+  private static final String TABLE_NAME_UNPARTITIONED = 
"test_table_unpartitioned";
+  private static final String SOURCE_HIVE_CATALOG_NAME = "myhive";
+  private static final String SOURCE_HIVE_DB_NAME = "test_hive_db";
+  private static final String SOURCE_HIVE_TABLE_NAME = "test_hive_table";
+
+  private final FileFormat format;
+  private static HiveMetaStoreClient metastoreClient;
+  private HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      TestMigrateAction.metastoreClient = new HiveMetaStoreClient(hiveConf);
+      String dbPath = metastore.getDatabasePath(SOURCE_HIVE_DB_NAME);
+      Database db = new Database(SOURCE_HIVE_DB_NAME, "description", dbPath, 
Maps.newHashMap());
+      metastoreClient.createDatabase(db);
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Rule
+  public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, 
SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    getTableEnv().registerCatalog(SOURCE_HIVE_CATALOG_NAME, flinkHiveCatalog);
+    exec("USE CATALOG %s", SOURCE_HIVE_CATALOG_NAME);
+    exec("USE %s", SOURCE_HIVE_DB_NAME);
+  }
+
+  @After
+  public void clean() {
+    // drop iceberg db and table
+    exec("DROP TABLE IF EXISTS %s.%s", flinkDatabase, 
TABLE_NAME_UNPARTITIONED);
+    exec("DROP DATABASE IF EXISTS %s", flinkDatabase);
+    super.clean();
+
+    // drop hive db and table
+    try {
+      metastoreClient.dropTable(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME);
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+    exec("DROP CATALOG IF EXISTS %s", SOURCE_HIVE_CATALOG_NAME);
+  }
+
+  @Override
+  protected StreamTableEnvironment getTableEnv() {
+    if (tEnv == null) {
+      synchronized (this) {
+        if (tEnv == null) {
+          env.setParallelism(1);
+          env.enableCheckpointing(100);
+          this.tEnv = StreamTableEnvironment.create(env);
+        }
+      }
+    }
+    return tEnv;
+  }
+
+  @Parameterized.Parameters(name = "catalogName={0}, baseNamespace={1}, 
format={2}")
+  public static Iterable<Object[]> parameters() {
+    List<Object[]> parameters = Lists.newArrayList();
+    for (FileFormat format : new FileFormat[] {FileFormat.ORC, 
FileFormat.PARQUET}) {
+      for (Object[] catalogParams : FlinkCatalogTestBase.parameters()) {
+        String catalogName = (String) catalogParams[0];
+        Namespace baseNamespace = (Namespace) catalogParams[1];
+        parameters.add(new Object[] {catalogName, baseNamespace, format});
+      }
+    }
+    return parameters;
+  }
+
+  public TestMigrateAction(String catalogName, Namespace baseNamespace, 
FileFormat format) {
+    super(catalogName, baseNamespace);
+    this.format = format;
+  }
+
+  @Test
+  public void testMigrateUnpartition() throws IOException, 
TableNotExistException {
+    getTableEnv().getConfig().setSqlDialect(SqlDialect.HIVE);
+    sql("CREATE TABLE %s (id INT, data STRING) stored as %s", 
SOURCE_HIVE_TABLE_NAME, format.name());
+    getTableEnv().getConfig().setSqlDialect(SqlDialect.DEFAULT);
+
+    String location = flinkHiveCatalog.getHiveTable(new 
ObjectPath(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME)).getSd()
+        .getLocation();
+
+    Schema schema = new Schema(
+        Types.NestedField.required(1, "id", Types.IntegerType.get()),
+        Types.NestedField.optional(2, "data", Types.StringType.get()));
+    GenericAppenderFactory genericAppenderFactory = new 
GenericAppenderFactory(schema);
+
+    URL url = new URL(location + File.separator + "test." + format.name());
+    File dataFile = new File(url.getPath());
+
+    List<Object[]> expected = Lists.newArrayList();
+    try (FileAppender<Record> fileAppender = 
genericAppenderFactory.newAppender(Files.localOutput(dataFile), format)) {
+      for (int i = 0; i < 10; i++) {
+        Record record = SimpleDataUtil.createRecord(i, "iceberg");
+        fileAppender.add(record);
+        expected.add(new Object[] {i, "iceberg"});
+      }
+    }
+
+    int migrateFileCount =
+        Actions.migrateHive2Iceberg(env, flinkHiveCatalog, 
SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME,
+            validationCatalog, baseNamespace, DATABASE, 
TABLE_NAME_UNPARTITIONED).execute();
+    Assert.assertEquals("Should produce the expected file count.", 1, 
migrateFileCount);
+
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+    List<Object[]> list = sql("SELECT * FROM %s", TABLE_NAME_UNPARTITIONED);
+    Assert.assertEquals("Should produce the expected records count.", 10, 
list.size());
+    Assert.assertArrayEquals("Should produce the expected records.", 
expected.toArray(), list.toArray());
+  }
+
+  @Test
+  public void testMigratePartition() throws IOException, TException {
+    getTableEnv().getConfig().setSqlDialect(SqlDialect.HIVE);
+    sql("CREATE TABLE %s (id INT, data STRING) PARTITIONED BY (p STRING) 
STORED AS %s", SOURCE_HIVE_TABLE_NAME,
+        format.name());
+
+    Partition hivePartition = createHivePartition(format);
+    getTableEnv().getConfig().setSqlDialect(SqlDialect.DEFAULT);
+
+    metastoreClient.add_partition(hivePartition);
+
+    Partition partition = metastoreClient.getPartition(SOURCE_HIVE_DB_NAME, 
SOURCE_HIVE_TABLE_NAME, "p=iceberg");
+    String location = partition.getSd().getLocation();
+
+    Schema schema = new Schema(
+        Types.NestedField.required(1, "id", Types.IntegerType.get()),
+        Types.NestedField.optional(2, "data", Types.StringType.get()));
+    GenericAppenderFactory genericAppenderFactory = new 
GenericAppenderFactory(schema);
+
+    URL url = new URL(location + File.separator + "test." + format.name());
+    File dataFile = new File(url.getPath());
+
+    List<Object[]> expected = Lists.newArrayList();
+    try (FileAppender<Record> fileAppender = 
genericAppenderFactory.newAppender(Files.localOutput(dataFile), format)) {
+      for (int i = 0; i < 10; i++) {
+        Record record = SimpleDataUtil.createRecord(i, "iceberg" + i);
+        fileAppender.add(record);
+        expected.add(new Object[] {i, "iceberg" + i, "iceberg"});
+      }
+    }
+
+    int migrateFileCount =
+        Actions.migrateHive2Iceberg(env, flinkHiveCatalog, 
SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME,
+            validationCatalog, baseNamespace, DATABASE, 
TABLE_NAME_UNPARTITIONED).execute();
+    Assert.assertEquals("Should produce the expected file count.", 1, 
migrateFileCount);

Review comment:
       if we are going to create 1 manifest per partition, then we can also add 
an assertion on `allManifests`




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