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



##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,423 @@
+/*
+ * 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.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.runtime.util.HadoopUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+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.flink.util.ArrayUtils;
+import org.apache.flink.util.Collector;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestWriter;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.Action;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.flink.FlinkCatalogFactory;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.orc.OrcMetrics;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<List<ManifestFile>> {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(MigrateAction.class);
+
+  private static final String PATQUET_INPUT_FORMAT = 
"org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat";
+  private static final String ORC_INPUT_FORMAT = 
"org.apache.hadoop.hive.ql.io.orc.OrcInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+  private static final String ICEBERG_METADATA_FOLDER = "metadata";
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDatabaseName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDatabaseName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog 
flinkHiveCatalog, String hiveSourceDatabaseName,
+                       String hiveSourceTableName, Catalog icebergCatalog, 
Namespace baseNamespace,
+                       String icebergDatabaseName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDatabaseName = hiveSourceDatabaseName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDatabaseName = icebergDatabaseName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public List<ManifestFile> execute() {
+    flinkHiveCatalog.open();
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDatabaseName, 
hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    try {
+      hiveTable = flinkHiveCatalog.getHiveTable(tableSource);
+    } catch (TableNotExistException e) {
+      throw new RuntimeException(String.format("The source table %s not exists 
! ", hiveSourceTableName));
+    }
+
+    List<FieldSchema> fieldSchemaList = hiveTable.getSd().getCols();
+    List<FieldSchema> partitionList = hiveTable.getPartitionKeys();
+    fieldSchemaList.addAll(partitionList);
+    Schema icebergSchema = HiveSchemaUtil.convert(fieldSchemaList);
+    PartitionSpec spec = toPartitionSpec(partitionList, icebergSchema);
+
+    FileFormat fileFormat = getHiveFileFormat(hiveTable);
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(), 
icebergTableName);
+    String hiveLocation = hiveTable.getSd().getLocation();
+
+    Map<String, String> properties = Maps.newHashMap();
+    properties.put(TableProperties.DEFAULT_FILE_FORMAT, fileFormat.name());
+    if (!baseNamespace.isEmpty()) {
+      properties.put(FlinkCatalogFactory.BASE_NAMESPACE, 
baseNamespace.toString());
+    }
+
+    Table icebergTable;
+    if (icebergCatalog instanceof HadoopCatalog) {
+      icebergTable = icebergCatalog.createTable(identifier, icebergSchema, 
spec, properties);
+    } else {
+      icebergTable = icebergCatalog.createTable(identifier, icebergSchema, 
spec, hiveLocation, properties);
+    }
+
+    String metadataLocation = getMetadataLocation(icebergTable);
+
+    String nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), 
TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = 
MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<ManifestFile> manifestFiles = null;
+    try {
+      if (spec.isUnpartitioned()) {
+        manifestFiles =
+            migrateUnpartitionedTable(spec, fileFormat, hiveLocation, 
nameMapping, metricsConfig, metadataLocation);
+      } else {
+        manifestFiles =
+            migratePartitionedTable(spec, tableSource, nameMapping, 
fileFormat, metricsConfig, metadataLocation);
+      }
+
+      AppendFiles append = icebergTable.newAppend();
+      manifestFiles.forEach(append::appendManifest);
+      append.commit();
+    } catch (Exception e) {
+      LOGGER.error("Migrate hive table to iceberg table failed.", e);

Review comment:
       yes,I add the `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:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to