You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/02/05 08:44:58 UTC

[GitHub] [iceberg] zhangjun0x01 opened a new pull request #2217: Flink : migrate hive table to iceberg table

zhangjun0x01 opened a new pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217


   related to https://github.com/apache/iceberg/issues/2162


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571645531



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.flink.api.common.functions.RichMapFunction;
+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.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;
+
+public class MigrateAction implements Action<List<ManifestFile>> {
+
+  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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  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, icebergTable, nameMapping, metricsConfig,
+                metadataLocation);
+      } else {
+        manifestFiles =
+            migratePartitionedTable(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name(),
+                metadataLocation);
+
+      }
+    } catch (Exception e) {
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Migrate", e);
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    manifestFiles.forEach(append::appendManifest);
+    append.commit();
+
+    flinkHiveCatalog.close();
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,
+                                                       Table icebergTable, String nameMapping,
+                                                       MetricsConfig metricsConfig, String metadataLocation)
+      throws Exception {
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<PartitionAndLocation> dataStream =
+        env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+    DataStream<List<ManifestFile>> ds = dataStream.map(migrateMapper);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+        .flatMap(Collection::stream).collect(Collectors.toList());
+  }
+
+  private Namespace toNamespace() {
+    return Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));
+  }
+
+  private List<ManifestFile> migratePartitionedTable(PartitionSpec spec, ObjectPath tableSource,
+                                                     String nameMapping, FileFormat fileFormat,
+                                                     MetricsConfig metricsConfig, String hiveTableName,
+                                                     String metadataLocation)
+      throws Exception {
+    List<CatalogPartitionSpec> partitionSpecs = flinkHiveCatalog.listPartitions(tableSource);
+    List<PartitionAndLocation> inputs = Lists.newArrayList();
+    for (CatalogPartitionSpec partitionSpec : partitionSpecs) {
+      Partition partition =
+          flinkHiveCatalog.getHivePartition(flinkHiveCatalog.getHiveTable(tableSource), partitionSpec);
+      inputs.add(
+          new PartitionAndLocation(partition.getSd().getLocation(), Maps.newHashMap(partitionSpec.getPartitionSpec())));
+    }
+
+    int size = partitionSpecs.size();
+    int parallelism = Math.min(size, maxParallelism);
+
+    DataStream<PartitionAndLocation> dataStream = env.fromCollection(inputs);
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<List<ManifestFile>> ds = dataStream.map(migrateMapper).setParallelism(parallelism);

Review comment:
       This is probably better a `DataStream<ManifestFile>`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571674095



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);

Review comment:
       After the refactoring of emitting ManifestFile per partition, we are only listing the files under one partition. Then I won't be concerned for the list. Previously, we are collecting all files under a 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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-773890650


   @openinx @stevenzwu could you help me review it ? thanks 


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578667299



##########
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);

Review comment:
       This can also lead to memory pressure issues as it collects all manifest information for the table at the same time.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] MichaelTiemannOSC commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
MichaelTiemannOSC commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-977718918


   Has this PR gone dormant due to acceptance of [#2210?](https://github.com/apache/iceberg/pull/2210)
   
   I'm using a Trino environment where it's easy to write Hive parquet files from pandas, but I haven't found the fast path from pandas to Iceberg.  (Row-by-row insertion is too slow.)  I'm here because I thought table migration might be an answer...


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571517626



##########
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() {

Review comment:
       because we use `StreamExecutionEnvironment` in the MigrateAction, so I use `StreamExecutionEnvironment` to construct the `StreamTableEnvironment` .
   
   the `parallelism` and `enableCheckpointing ` is my mistake ,I remove it




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571645037



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.flink.api.common.functions.RichMapFunction;
+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.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;
+
+public class MigrateAction implements Action<List<ManifestFile>> {
+
+  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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  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, icebergTable, nameMapping, metricsConfig,
+                metadataLocation);
+      } else {
+        manifestFiles =
+            migratePartitionedTable(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name(),
+                metadataLocation);
+
+      }
+    } catch (Exception e) {
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Migrate", e);
+    }
+
+    AppendFiles append = icebergTable.newAppend();

Review comment:
       nit: move this into the try block?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571127314



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return files.size();
+  }
+
+  private Namespace toNamespace(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Namespace.of(icebergDbName);
+    } else {
+      String[] namespaces = new String[baseNamespace.levels().length + 1];
+      System.arraycopy(baseNamespace.levels(), 0, namespaces, 0, baseNamespace.levels().length);
+      namespaces[baseNamespace.levels().length] = icebergDbName;
+      return Namespace.of(namespaces);
+    }
+  }
+
+  private List<DataFile> importPartitions(PartitionSpec spec, ObjectPath tableSource,
+                                          String nameMapping, FileFormat fileFormat,
+                                          MetricsConfig metricsConfig, String hiveTableName)
+      throws Exception {
+    List<CatalogPartitionSpec> partitionSpecs = flinkHiveCatalog.listPartitions(tableSource);
+    List<PartitionAndLocation> inputs = Lists.newArrayList();
+    for (CatalogPartitionSpec partitionSpec : partitionSpecs) {
+      Partition partition =
+          flinkHiveCatalog.getHivePartition(flinkHiveCatalog.getHiveTable(tableSource), partitionSpec);
+      inputs.add(
+          new PartitionAndLocation(partition.getSd().getLocation(), Maps.newHashMap(partitionSpec.getPartitionSpec())));
+    }
+
+    int size = partitionSpecs.size();
+    int parallelism = Math.min(size, maxParallelism);
+
+    DataStream<PartitionAndLocation> dataStream = env.fromCollection(inputs);
+    MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+    DataStream<List<DataFile>> ds = dataStream.map(migrateMap).setParallelism(parallelism);

Review comment:
       As mentioned in the another comment, we can parallelize the processing of each Hive partition and generate a manifest file for each partition on taskmanagers. That is probably the more expensive part of migration. Then on the jobmanager, we can just keep `appendManifest` and `commit` once.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578082619



##########
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 delete the `throw`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r577633327



##########
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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Failed to migrate hive table", e);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:

Review comment:
       nit: Typo




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578672027



##########
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:
       Do we need to clean up the Table as well here? Or is deleting the manifests enough? 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r618122251



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java
##########
@@ -50,4 +53,34 @@ public RewriteDataFilesAction rewriteDataFiles() {
     return new RewriteDataFilesAction(env, table);
   }
 
+  /**
+   * Migrate an exist hive table into iceberg table
+   *
+   * @param flinkHiveCatalog    the HiveCatalog of flink, we get hive table info from this
+   * @param hiveSourceDatabaseName    source hive database name
+   * @param hiveSourceTableName source hive table name
+   * @param icebergCatalog      target iceberg catalog
+   * @param baseNamespace       target iceberg Namespace
+   * @param icebergDbName       target iceberg database name
+   * @param icebergTableName    target iceberg table name
+   * @return the MigrateAction
+   */
+  public static MigrateAction migrateHive2Iceberg(HiveCatalog flinkHiveCatalog, String hiveSourceDatabaseName,
+                                                  String hiveSourceTableName, Catalog icebergCatalog,
+                                                  Namespace baseNamespace,
+                                                  String icebergDbName, String icebergTableName) {
+    return migrateHive2Iceberg(StreamExecutionEnvironment.getExecutionEnvironment(), flinkHiveCatalog,
+        hiveSourceDatabaseName,
+        hiveSourceTableName, icebergCatalog, baseNamespace,
+        icebergDbName, icebergTableName);
+  }
+
+  public static MigrateAction migrateHive2Iceberg(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog,
+                                                  String hiveSourceDatabaseName, String hiveSourceTableName,
+                                                  Catalog icebergCatalog, Namespace baseNamespace,
+                                                  String icebergDbName, String icebergTableName) {

Review comment:
       I think the `database`,`table` may be more clear for new iceberg user ,because the `TableIdentifier ` is iceberg internal class,the user need to know how to construct the `TableIdentifier`,what do you think?




-- 
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r618127756



##########
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) {

Review comment:
       I think if we add a new Catalog,We should consider whether we should support migration to the new catalog. For example, in the process of doing this migration action, I encountered many problems with the avro format, which was not easy to handle. In addition, I think that avro as a data file format should not be common, so I do not support the avro format .
   
   but I added a verification to make sure `icebergCatalog` is a `hadoopcatalog` or `hivecatalog`.
   ```
     if (icebergCatalog instanceof HadoopCatalog) {
         icebergTable = icebergCatalog.createTable(identifier, icebergSchema, spec, properties);
       } else if (icebergCatalog instanceof org.apache.iceberg.hive.HiveCatalog) {
         icebergTable = icebergCatalog.createTable(identifier, icebergSchema, spec, hiveLocation, properties);
       }
   ```
   
   if we add a new Catalog,I think we may need to modify a lot of code,for example : 
   https://github.com/apache/iceberg/blob/master/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java#L104
   
   what do you think ?




-- 
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-780974521


   > Sounds good. My only concern is the dependency on `StreamExecutionEnvironment` and other Flink constructs. That would cause Flink dependency, even if we do not use Flink for a minimal local migration. Is there a way to separate the logic and the parallel execution?
   
   Are you concern the conflict between `StreamExecutionEnvironment` and other flink constructs? `StreamExecutionEnvironment` is the basic environment for flink program execution. it will not conflict with other construction methods.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578059084



##########
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);

Review comment:
       for partition table ,if we add all datafile into one manifest, it may be affect read performance
   https://github.com/apache/iceberg/pull/2217#discussion_r571085366




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578668982



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,406 @@
+/*
+ * 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.hive.common.FileUtils;
+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 PARQUET_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 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 = HiveSchemaUtil.spec(icebergSchema, partitionList);
+
+    FileFormat fileFormat = getHiveFileFormat(hiveTable);
+
+    Namespace namespace = Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));
+    TableIdentifier identifier = TableIdentifier.of(namespace, 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);

Review comment:
       in the Spark path we default to adding a mapping if none is specified. I think this is probably a good idea here as well.
   
   https://github.com/apache/iceberg/blob/04f4a309240ef52faaebe4eef5820a682906b780/spark3/src/main/java/org/apache/iceberg/actions/Spark3CreateAction.java#L174-L180




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r577610516



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return files.size();
+  }
+
+  private Namespace toNamespace(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Namespace.of(icebergDbName);
+    } else {
+      String[] namespaces = new String[baseNamespace.levels().length + 1];
+      System.arraycopy(baseNamespace.levels(), 0, namespaces, 0, baseNamespace.levels().length);
+      namespaces[baseNamespace.levels().length] = icebergDbName;
+      return Namespace.of(namespaces);
+    }
+  }
+
+  private List<DataFile> importPartitions(PartitionSpec spec, ObjectPath tableSource,
+                                          String nameMapping, FileFormat fileFormat,
+                                          MetricsConfig metricsConfig, String hiveTableName)
+      throws Exception {
+    List<CatalogPartitionSpec> partitionSpecs = flinkHiveCatalog.listPartitions(tableSource);
+    List<PartitionAndLocation> inputs = Lists.newArrayList();
+    for (CatalogPartitionSpec partitionSpec : partitionSpecs) {
+      Partition partition =
+          flinkHiveCatalog.getHivePartition(flinkHiveCatalog.getHiveTable(tableSource), partitionSpec);
+      inputs.add(
+          new PartitionAndLocation(partition.getSd().getLocation(), Maps.newHashMap(partitionSpec.getPartitionSpec())));
+    }
+
+    int size = partitionSpecs.size();
+    int parallelism = Math.min(size, maxParallelism);
+
+    DataStream<PartitionAndLocation> dataStream = env.fromCollection(inputs);
+    MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+    DataStream<List<DataFile>> ds = dataStream.map(migrateMap).setParallelism(parallelism);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveTableName)).stream()
+        .flatMap(Collection::stream).collect(Collectors.toList());
+  }
+
+  private static class MigrateMap implements MapFunction<PartitionAndLocation, List<DataFile>> {
+    private final PartitionSpec spec;
+    private final String nameMappingString;
+    private final FileFormat fileFormat;
+    private final MetricsConfig metricsConfig;
+
+    MigrateMap(PartitionSpec spec, String nameMapping, FileFormat fileFormat, MetricsConfig metricsConfig) {
+      this.spec = spec;
+      this.nameMappingString = nameMapping;
+      this.fileFormat = fileFormat;
+      this.metricsConfig = metricsConfig;
+    }
+
+    @Override
+    public List<DataFile> map(PartitionAndLocation map) {
+      Map<String, String> partitions = map.getMap();
+      String location = map.getLocation();
+      Configuration conf = HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration());
+      NameMapping nameMapping = nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null;
+      List<DataFile> files;
+      switch (fileFormat) {

Review comment:
       One can change the InputFormat/OutputFormat, but that will result in a "dead" table. So it is ok to expect only a single file format.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r579544509



##########
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);

Review comment:
       `ManifestFile` in-memory object is relatively small in bytes. Even for 1M objects might fit in memory. 
   
   One way to improve the memory profile is not to collect the complete list of manifest files. We can add another commit operator (with parallelism 1). it can collect a batch of manifest files (like 100 or 1,000) and commit them. This commit operator can also apply backpressure to upstream via the Flink network stack.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571645449



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.flink.api.common.functions.RichMapFunction;
+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.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;
+
+public class MigrateAction implements Action<List<ManifestFile>> {
+
+  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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  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, icebergTable, nameMapping, metricsConfig,
+                metadataLocation);
+      } else {
+        manifestFiles =
+            migratePartitionedTable(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name(),
+                metadataLocation);
+
+      }
+    } catch (Exception e) {
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Migrate", e);
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    manifestFiles.forEach(append::appendManifest);
+    append.commit();
+
+    flinkHiveCatalog.close();
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,
+                                                       Table icebergTable, String nameMapping,
+                                                       MetricsConfig metricsConfig, String metadataLocation)
+      throws Exception {
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<PartitionAndLocation> dataStream =
+        env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+    DataStream<List<ManifestFile>> ds = dataStream.map(migrateMapper);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+        .flatMap(Collection::stream).collect(Collectors.toList());
+  }
+
+  private Namespace toNamespace() {
+    return Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));
+  }
+
+  private List<ManifestFile> migratePartitionedTable(PartitionSpec spec, ObjectPath tableSource,
+                                                     String nameMapping, FileFormat fileFormat,
+                                                     MetricsConfig metricsConfig, String hiveTableName,
+                                                     String metadataLocation)
+      throws Exception {
+    List<CatalogPartitionSpec> partitionSpecs = flinkHiveCatalog.listPartitions(tableSource);
+    List<PartitionAndLocation> inputs = Lists.newArrayList();
+    for (CatalogPartitionSpec partitionSpec : partitionSpecs) {
+      Partition partition =
+          flinkHiveCatalog.getHivePartition(flinkHiveCatalog.getHiveTable(tableSource), partitionSpec);
+      inputs.add(
+          new PartitionAndLocation(partition.getSd().getLocation(), Maps.newHashMap(partitionSpec.getPartitionSpec())));
+    }
+
+    int size = partitionSpecs.size();
+    int parallelism = Math.min(size, maxParallelism);
+
+    DataStream<PartitionAndLocation> dataStream = env.fromCollection(inputs);
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<List<ManifestFile>> ds = dataStream.map(migrateMapper).setParallelism(parallelism);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveTableName)).stream()
+        .flatMap(Collection::stream).collect(Collectors.toList());
+  }
+
+  private static class MigrateMapper extends RichMapFunction<PartitionAndLocation, List<ManifestFile>> {

Review comment:
       this is really a flapMap, right?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r582251675



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -346,7 +346,7 @@ public void testPartitionTypes() throws Exception {
     assertRecords(run(), records, typesSchema);
   }
 
-  static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {
+  public static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {

Review comment:
       I am fine with moving this refactoring to another PR.
   Thanks, Peter 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-779213255


   I have just found this PR when I was wondering how to migrate existing Hive tables to Iceberg tables.
   My use-case is that I have an existing Hive table, and I would like to convert it to a Hive table backed by an Iceberg table in-place, and without moving the actual data. I would like to create the corresponding manifest files and the first snapshot using the existing files.
   
   When I sketched the code I found that I was listing the partitions / creating `DataFile` objects / creating a new Iceberg table and adding the data files to it. And then I have found that you have already did the same in the `Actions.migrateHive2Iceberg`. At first glance there is not too much Flink specific code in this change. Would it be hard to create a general tool for the migration instead of a Flink specific action?
   
   Thanks,
   Peter


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578959300



##########
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);

Review comment:
       Do you have better suggestions for this case? thanks




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r572194593



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -294,27 +292,26 @@ private Namespace toNamespace() {
       return buildManifest(conf, spec, metadataLocation, files);
     }
 
-    private List<ManifestFile> buildManifest(Configuration conf, PartitionSpec partitionSpec,
-                                             String basePath, List<DataFile> dataFiles) {
+    private ManifestFile buildManifest(Configuration conf, PartitionSpec partitionSpec,
+                                       String basePath, List<DataFile> dataFiles) {
       if (dataFiles.size() > 0) {
         FileIO io = new HadoopFileIO(conf);
         int subTaskId = getRuntimeContext().getIndexOfThisSubtask();
         int attemptId = getRuntimeContext().getAttemptNumber();
-        String suffix = String.format("%d-task-%d-manifest", subTaskId, attemptId);
+        String suffix = String.format("manifest-%d-%d", subTaskId, attemptId);
         Path location = new Path(basePath, suffix);
         String outputPath = FileFormat.AVRO.addExtension(location.toString());
         OutputFile outputFile = io.newOutputFile(outputPath);
         ManifestWriter<DataFile> writer = ManifestFiles.write(partitionSpec, outputFile);
-        try (ManifestWriter<DataFile> writerRef = writer) {
-          dataFiles.forEach(writerRef::add);
+        try (ManifestWriter<DataFile> manifestWriter = writer) {
+          dataFiles.forEach(manifestWriter::add);
         } catch (IOException e) {
           throw new UncheckedIOException("Unable to close the manifest writer", e);
         }
 
-        ManifestFile manifestFile = writer.toManifestFile();
-        return Lists.newArrayList(manifestFile);
+        return writer.toManifestFile();
       } else {
-        return Lists.newArrayList();
+        return null;

Review comment:
       now we got null value. `AppendFiles#appendManifest` doesn't handle null values. is `flatMap` better?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r577642328



##########
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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Failed to migrate hive table", e);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {

Review comment:
       nit: Could this be static?
   Tend to create static utility methods if it is possible. Less error prone, and the reader can be sure nothing interferes  outside of the method




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571736781



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    getTableEnv().registerCatalog(SOURCE_HIVE_CATALOG_NAME, flinkHiveCatalog);

Review comment:
       `BeforeClass` is a static method, if we move them to beforeClass,it will cause a compile error.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571061958



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();

Review comment:
       this method is getting very long. maybe extract this fileFormat logic to a util class/method?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571759749



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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);

Review comment:
       maybe add a comment to explain it?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-782132895


   Really exciting to see more functionality being added here!


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571646599



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    getTableEnv().registerCatalog(SOURCE_HIVE_CATALOG_NAME, flinkHiveCatalog);

Review comment:
       should line 104 and 106 also be moved to beforeClass?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571766041



##########
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() {

Review comment:
       I am aware of the DataSet deprecation in the future. Let me ask in a diff way. Why `FlinkTestBase#getTableEnv` doesn't work for this case? `TableEnvironment` interface is a higher abstraction than `DataSet/DataStream`.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571643437



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.flink.api.common.functions.RichMapFunction;
+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.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;
+
+public class MigrateAction implements Action<List<ManifestFile>> {
+
+  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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  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, icebergTable, nameMapping, metricsConfig,
+                metadataLocation);
+      } else {
+        manifestFiles =
+            migratePartitionedTable(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name(),
+                metadataLocation);
+
+      }
+    } catch (Exception e) {
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Migrate", e);

Review comment:
       nit: "Failed to migrate"?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571643703



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.flink.api.common.functions.RichMapFunction;
+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.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;
+
+public class MigrateAction implements Action<List<ManifestFile>> {
+
+  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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  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, icebergTable, nameMapping, metricsConfig,
+                metadataLocation);
+      } else {
+        manifestFiles =
+            migratePartitionedTable(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name(),
+                metadataLocation);
+
+      }
+    } catch (Exception e) {
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Migrate", e);
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    manifestFiles.forEach(append::appendManifest);
+    append.commit();
+
+    flinkHiveCatalog.close();
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()

Review comment:
       nit: add some logging `onFailure`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571647782



##########
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() {

Review comment:
       Can you elaborate a little why do we need `StreamExecutionEnvironment`?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571749060



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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) {
+          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"});
+      }
+    }
+
+    List<ManifestFile> manifestFiles =
+        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 manifestFiles count.", 1, manifestFiles.size());
+
+    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, TableNotExistException {
+    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());
+    getTableEnv().getConfig().setSqlDialect(SqlDialect.DEFAULT);
+    String hiveLocation = flinkHiveCatalog.getHiveTable(new ObjectPath(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME))
+        .getSd().getLocation();
+
+    List<Object[]> expected = Lists.newArrayList();
+    String[] partitions = new String[] {"iceberg", "flink"};
+    for (String partitionValue : partitions) {
+      String partitionPath = hiveLocation + "/p=" + partitionValue;
+
+      Partition hivePartition = createHivePartition(format, partitionPath, partitionValue);
+      metastoreClient.add_partition(hivePartition);
+
+      Partition partition =
+          metastoreClient.getPartition(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME, "p=" + partitionValue);
+      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());
+
+      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, partitionValue});
+        }
+      }
+    }
+
+    List<ManifestFile> manifestFiles =
+        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 manifestFiles count.", 2, manifestFiles.size());
+
+    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.", 20, list.size());
+    sortList(list);
+    sortList(expected);
+    Assert.assertArrayEquals("Should produce the expected records.", expected.toArray(), list.toArray());

Review comment:
       I use `TestFlinkScan.assertRecords` to do the  assertion, and add a method `List<Row> executeSql(String query, Object... args)` in the `FlinkTestBase` to get `List<Row>`, because there are a few test method call the `TestFlinkScan.assertRecords`,if we move it to `TestHelpers ` , we will  have to modify these test methods




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-774643313


   @stevenzwu thanks for you review,I updated all


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571732761



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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) {
+          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"});
+      }
+    }
+
+    List<ManifestFile> manifestFiles =
+        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 manifestFiles count.", 1, manifestFiles.size());
+
+    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, TableNotExistException {
+    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());
+    getTableEnv().getConfig().setSqlDialect(SqlDialect.DEFAULT);
+    String hiveLocation = flinkHiveCatalog.getHiveTable(new ObjectPath(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME))
+        .getSd().getLocation();
+
+    List<Object[]> expected = Lists.newArrayList();
+    String[] partitions = new String[] {"iceberg", "flink"};
+    for (String partitionValue : partitions) {
+      String partitionPath = hiveLocation + "/p=" + partitionValue;
+
+      Partition hivePartition = createHivePartition(format, partitionPath, partitionValue);
+      metastoreClient.add_partition(hivePartition);
+
+      Partition partition =
+          metastoreClient.getPartition(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME, "p=" + partitionValue);
+      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());
+
+      try (

Review comment:
       It required an exist iceberg table, now we need to generate data for hive table, not iceberg 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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571128269



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return files.size();
+  }
+
+  private Namespace toNamespace(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Namespace.of(icebergDbName);
+    } else {
+      String[] namespaces = new String[baseNamespace.levels().length + 1];
+      System.arraycopy(baseNamespace.levels(), 0, namespaces, 0, baseNamespace.levels().length);
+      namespaces[baseNamespace.levels().length] = icebergDbName;
+      return Namespace.of(namespaces);
+    }
+  }
+
+  private List<DataFile> importPartitions(PartitionSpec spec, ObjectPath tableSource,
+                                          String nameMapping, FileFormat fileFormat,
+                                          MetricsConfig metricsConfig, String hiveTableName)
+      throws Exception {
+    List<CatalogPartitionSpec> partitionSpecs = flinkHiveCatalog.listPartitions(tableSource);
+    List<PartitionAndLocation> inputs = Lists.newArrayList();
+    for (CatalogPartitionSpec partitionSpec : partitionSpecs) {
+      Partition partition =
+          flinkHiveCatalog.getHivePartition(flinkHiveCatalog.getHiveTable(tableSource), partitionSpec);
+      inputs.add(
+          new PartitionAndLocation(partition.getSd().getLocation(), Maps.newHashMap(partitionSpec.getPartitionSpec())));
+    }
+
+    int size = partitionSpecs.size();
+    int parallelism = Math.min(size, maxParallelism);
+
+    DataStream<PartitionAndLocation> dataStream = env.fromCollection(inputs);
+    MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+    DataStream<List<DataFile>> ds = dataStream.map(migrateMap).setParallelism(parallelism);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveTableName)).stream()
+        .flatMap(Collection::stream).collect(Collectors.toList());
+  }
+
+  private static class MigrateMap implements MapFunction<PartitionAndLocation, List<DataFile>> {

Review comment:
       nit: Map -> Mapper? `Map` gives the impression of Java Collection.
   
   Here is the Flink `DataStream` map API
   ```
   map(MapFunction<T, R> mapper)
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r580845480



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -346,7 +346,7 @@ public void testPartitionTypes() throws Exception {
     assertRecords(run(), records, typesSchema);
   }
 
-  static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {
+  public static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {

Review comment:
       @zhangjun0x01: We have 2 flavors for checking the contents of the table:
   - [HiveIcebergTestUtils.validateData(Table table, List<Record> expected, int sortBy)](https://github.com/apache/iceberg/blob/e4185cd18412711242d0233ff901fb185ce63c94/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java#L220)
   - [HiveIcebergTestUtils.validateData(List<Record> expected, List<Record> actual, int sortBy)](https://github.com/apache/iceberg/blob/e4185cd18412711242d0233ff901fb185ce63c94/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java#L238)
   
   Thanks,
   Peter




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r579544509



##########
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);

Review comment:
       `ManifestFile` in-memory object is relatively small in bytes. Even for 1M objects might fit in memory. 
   
   One way to improve the memory profile is not to collect the complete list of manifest files. We can add another commit operator (with parallelism 1). it can collect a batch of manifest files (like 100 or 1,000) and commit them. This commit operator can also apply backpressure to upstream via the Flink network stack. This optimization could be done in a separate follow-up task. 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571068535



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return files.size();
+  }
+
+  private Namespace toNamespace(Namespace namespace) {
+    if (namespace.isEmpty()) {

Review comment:
       Actually, I found a simpler one-liner.
   ```
   Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {DATABASE}))
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571501255



##########
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");

Review comment:
       should we create `HiveCatalog` once in `beforeClass`?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r579248286



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,406 @@
+/*
+ * 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.hive.common.FileUtils;
+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 PARQUET_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 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 = HiveSchemaUtil.spec(icebergSchema, partitionList);
+
+    FileFormat fileFormat = getHiveFileFormat(hiveTable);
+
+    Namespace namespace = Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));
+    TableIdentifier identifier = TableIdentifier.of(namespace, 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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private static void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private static FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PARQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,

Review comment:
       The issue is that, say the user is writing to hivetable and while a write is in progress you start the migrate operation. The write finishes and commits, but the migrate command will not pick up these new files since they were created after the migrate scanned the partitions for existing datafiles.
   
   So you can have a successful write to the table which doesn't get migrated, I'm not sure if this is ok or not.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] openinx commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r603259159



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java
##########
@@ -50,4 +53,34 @@ public RewriteDataFilesAction rewriteDataFiles() {
     return new RewriteDataFilesAction(env, table);
   }
 
+  /**
+   * Migrate an exist hive table into iceberg table
+   *
+   * @param flinkHiveCatalog    the HiveCatalog of flink, we get hive table info from this
+   * @param hiveSourceDatabaseName    source hive database name
+   * @param hiveSourceTableName source hive table name
+   * @param icebergCatalog      target iceberg catalog
+   * @param baseNamespace       target iceberg Namespace
+   * @param icebergDbName       target iceberg database name
+   * @param icebergTableName    target iceberg table name
+   * @return the MigrateAction
+   */
+  public static MigrateAction migrateHive2Iceberg(HiveCatalog flinkHiveCatalog, String hiveSourceDatabaseName,
+                                                  String hiveSourceTableName, Catalog icebergCatalog,
+                                                  Namespace baseNamespace,
+                                                  String icebergDbName, String icebergTableName) {
+    return migrateHive2Iceberg(StreamExecutionEnvironment.getExecutionEnvironment(), flinkHiveCatalog,
+        hiveSourceDatabaseName,
+        hiveSourceTableName, icebergCatalog, baseNamespace,
+        icebergDbName, icebergTableName);
+  }
+
+  public static MigrateAction migrateHive2Iceberg(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog,
+                                                  String hiveSourceDatabaseName, String hiveSourceTableName,
+                                                  Catalog icebergCatalog, Namespace baseNamespace,
+                                                  String icebergDbName, String icebergTableName) {

Review comment:
       Here we pass the `namespace`, `database`, `table` , then how about just use a `TableIdentifier` ?

##########
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) {

Review comment:
       @zhangjun0x01 Is possible to use the `FlinkCatalog` to create the table ?  Then we don't have to use different code paths to create the iceberg table,  I'm concerning that there will be more Catalog that will be introduced in future.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java
##########
@@ -50,4 +53,34 @@ public RewriteDataFilesAction rewriteDataFiles() {
     return new RewriteDataFilesAction(env, table);
   }
 
+  /**
+   * Migrate an exist hive table into iceberg table
+   *
+   * @param flinkHiveCatalog    the HiveCatalog of flink, we get hive table info from this
+   * @param hiveSourceDatabaseName    source hive database name
+   * @param hiveSourceTableName source hive table name
+   * @param icebergCatalog      target iceberg catalog
+   * @param baseNamespace       target iceberg Namespace
+   * @param icebergDbName       target iceberg database name
+   * @param icebergTableName    target iceberg table name
+   * @return the MigrateAction
+   */
+  public static MigrateAction migrateHive2Iceberg(HiveCatalog flinkHiveCatalog, String hiveSourceDatabaseName,
+                                                  String hiveSourceTableName, Catalog icebergCatalog,
+                                                  Namespace baseNamespace,
+                                                  String icebergDbName, String icebergTableName) {
+    return migrateHive2Iceberg(StreamExecutionEnvironment.getExecutionEnvironment(), flinkHiveCatalog,
+        hiveSourceDatabaseName,
+        hiveSourceTableName, icebergCatalog, baseNamespace,
+        icebergDbName, icebergTableName);
+  }
+
+  public static MigrateAction migrateHive2Iceberg(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog,
+                                                  String hiveSourceDatabaseName, String hiveSourceTableName,

Review comment:
       Those `database` & `table` could just be simplified by a `TableIdentifier`.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,443 @@
+/*
+ * 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.UUID;
+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.hive.common.FileUtils;
+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.MappingUtil;
+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;
+
+/**
+ * Migrate exist hive table to iceberg table.
+ * <p>
+ * It support migrate hive table with parquet and orc format now, we can migrate it to iceberg table with {@link
+ * HadoopCatalog} and {@link HiveCatalog}.
+ * <p>
+ * The migration method is to keep the data file in the original hive table unchanged, and then create a new iceberg
+ * table, the new iceberg table use the data file of hive, and generate new metadata for the iceberg table.
+ * <p>
+ * In order to prevent data from being written to hive during the migration process so that new data cannot be migrated.
+ * so when we do the migration, we need to stop the writing hive job first, then migrate, and finally modify the logic
+ * to reading and writing iceberg table,If migrate failed, we will clean the iceberg table and metadata.If unfortunately
+ * the clean failed, you may need to manually clean the iceberg table and manifests.
+ */
+public class MigrateAction implements Action<List<ManifestFile>> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(MigrateAction.class);
+
+  private static final String PARQUET_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 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 = HiveSchemaUtil.spec(icebergSchema, partitionList);
+
+    FileFormat fileFormat = getHiveFileFormat(hiveTable);
+
+    Namespace namespace = Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));

Review comment:
       The `ArrayUtils` is a `Internal` API for flink,  I will suggest not to use it because it may be changed in the next release.




-- 
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r577634735



##########
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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Failed to migrate hive table", e);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,
+                                                       String nameMapping, MetricsConfig metricsConfig,
+                                                       String metadataLocation)
+      throws Exception {
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<PartitionAndLocation> dataStream =
+        env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+    DataStream<ManifestFile> ds = dataStream.flatMap(migrateMapper);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveSourceTableName)).stream()

Review comment:
       Why do we wrap the list to a `newArrayList`?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571664807



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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) {
+          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"});
+      }
+    }
+
+    List<ManifestFile> manifestFiles =
+        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 manifestFiles count.", 1, manifestFiles.size());
+
+    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, TableNotExistException {
+    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());
+    getTableEnv().getConfig().setSqlDialect(SqlDialect.DEFAULT);
+    String hiveLocation = flinkHiveCatalog.getHiveTable(new ObjectPath(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME))
+        .getSd().getLocation();
+
+    List<Object[]> expected = Lists.newArrayList();
+    String[] partitions = new String[] {"iceberg", "flink"};
+    for (String partitionValue : partitions) {
+      String partitionPath = hiveLocation + "/p=" + partitionValue;
+
+      Partition hivePartition = createHivePartition(format, partitionPath, partitionValue);
+      metastoreClient.add_partition(hivePartition);
+
+      Partition partition =
+          metastoreClient.getPartition(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME, "p=" + partitionValue);
+      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());
+
+      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, partitionValue});
+        }
+      }
+    }
+
+    List<ManifestFile> manifestFiles =
+        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 manifestFiles count.", 2, manifestFiles.size());
+
+    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.", 20, list.size());
+    sortList(list);
+    sortList(expected);
+    Assert.assertArrayEquals("Should produce the expected records.", expected.toArray(), list.toArray());

Review comment:
       It is probably simplier to use this util method to assert. Then we can get rid of the `sortList` below. Don't have to convert back and forth btw Row/Record and Object[].
   ```
   TestHelpers#assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) 
   ```
   
   Maybe move this util method from `TestFlinkScanSql` to `TestHelpers`
   ```
     private List<Row> sql(String query, Object... args) {
       TableResult tableResult = getTableEnv().executeSql(String.format(query, args));
       try (CloseableIterator<Row> iter = tableResult.collect()) {
         List<Row> results = Lists.newArrayList(iter);
         return results;
       } catch (Exception e) {
         throw new RuntimeException("Failed to collect table result", e);
       }
     }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571646725



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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);

Review comment:
       move this line to afterClass?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578677007



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,406 @@
+/*
+ * 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.hive.common.FileUtils;
+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 PARQUET_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 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 = HiveSchemaUtil.spec(icebergSchema, partitionList);
+
+    FileFormat fileFormat = getHiveFileFormat(hiveTable);
+
+    Namespace namespace = Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));
+    TableIdentifier identifier = TableIdentifier.of(namespace, 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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private static void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private static FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PARQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,

Review comment:
       One thing that we worried about here, and I'm not sure you have the same issue here, is what happens if a Hive writer is attempting to write while these migrations are occurring. We don't want to end up with dead files that a writer thinks are committed. To avoid this in the Spark Action, we move the catalog entry to a backup identifier so that all modifications will fail until the migration is complete. Basically making the old entry unavailable until we are sure it is either a Iceberg Table or we have aborted and we move it back as it's original config.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-773890650


   @openinx @stevenzwu could you help me review it ? thanks 


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r577632876



##########
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) {

Review comment:
       Why are we using different location for non-HadoopCatalogs?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571734472



##########
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() {

Review comment:
       in the future, we will use `StreamExecutionEnvironment ` and `DataStream` to do streaming and batch jobs. the `DataSet` will be deprecated, ([here](https://flink.apache.org/news/2020/12/10/release-1.12.0.html#batch-execution-mode-in-the-datastream-api))
   
   `
   This means that, in the long run, the DataSet API will be deprecated and subsumed by the DataStream API and the Table API/SQL (FLIP-131).
   `




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571064676



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return files.size();
+  }
+
+  private Namespace toNamespace(Namespace namespace) {

Review comment:
       to be consistent, I would suggest we also pass in the `icebergDbName` as arg.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571058880



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();

Review comment:
       we need to close the catalog in the end




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r577623393



##########
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);

Review comment:
       Maybe we can use `HiveSchemaUtil.spec(schema, hmsTable.getPartitionKeys());` instead?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578077112



##########
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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Failed to migrate hive table", e);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,
+                                                       String nameMapping, MetricsConfig metricsConfig,
+                                                       String metadataLocation)
+      throws Exception {
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<PartitionAndLocation> dataStream =
+        env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+    DataStream<ManifestFile> ds = dataStream.flatMap(migrateMapper);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveSourceTableName)).stream()

Review comment:
       because we need to wait for all flink sub task finished,and then submit complete data to iceberg.`Lists.newArrayList` can block all subtasks of flink until they are completed.
   
   But it seems that `stream()` is redundant, I removed it




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578672667



##########
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:
       Mostly want to make sure we clean up after this https://github.com/apache/iceberg/pull/2217/files#diff-50f83db1271f286f324eedd65c06dcae46e0827c2428c0de712a544b53698353R151
   




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578080136



##########
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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Failed to migrate hive table", e);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,
+                                                       String nameMapping, MetricsConfig metricsConfig,
+                                                       String metadataLocation)
+      throws Exception {
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<PartitionAndLocation> dataStream =
+        env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+    DataStream<ManifestFile> ds = dataStream.flatMap(migrateMapper);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveSourceTableName)).stream()
+        .collect(Collectors.toList());
+  }
+
+  private Namespace toNamespace() {

Review comment:
       There were a lot of codes for get Namespace before, so I extracted a method, but later simplified the logic.
   
   I moved it to the call method.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571646679



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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);

Review comment:
       in line 113, we use SQL to drop table. here we use the metastoreClient. any particular reason?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-781258888


   Found #2210 for Spark which tries to archive the same. We might want to synchronize our efforts.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571733401



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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);

Review comment:
       in the `before` method,use `super.before();` to create a catalog,so I drop the catalog in the `after`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571067117



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return files.size();
+  }
+
+  private Namespace toNamespace(Namespace namespace) {
+    if (namespace.isEmpty()) {

Review comment:
       one probably slightly simpler approach is to use List to avoid the if-else. First add the levels from `baseNamespace`. Then add the `icebergDbName`. Then `toArray`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571528775



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);

Review comment:
       > * collecting all of them into a single list can cause memory pressure.
   
   I looked at the code again, the `listStatus` method of hadoop filesystem use `ArrayList` to construct the `FileStatus` array.
   ```
     public FileStatus[] listStatus(Path f, PathFilter filter) 
                                      throws FileNotFoundException, IOException {
       ArrayList<FileStatus> results = new ArrayList<FileStatus>();
       listStatus(results, f, filter);
       return results.toArray(new FileStatus[results.size()]);
     }
   ```
   
   although there is the `org.apache.hadoop.fs.FileSystem#listStatusIterator` method, the `listStatus` method is still called in the internal of `FileSystem#listStatusIterator` method, so when we get the file list of hive, even if we use Iterator, I think it is still useless.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r581565779



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -346,7 +346,7 @@ public void testPartitionTypes() throws Exception {
     assertRecords(run(), records, typesSchema);
   }
 
-  static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {
+  public static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {

Review comment:
       @pvary I search the spark code ,[ It does have a similar required](https://github.com/apache/iceberg/blob/master/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java#L512), but  I think we should extract the  function in a new PR later ,what do you think?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571759644



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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);

Review comment:
       got it. base class create catalog




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r618120897



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,443 @@
+/*
+ * 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.UUID;
+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.hive.common.FileUtils;
+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.MappingUtil;
+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;
+
+/**
+ * Migrate exist hive table to iceberg table.
+ * <p>
+ * It support migrate hive table with parquet and orc format now, we can migrate it to iceberg table with {@link
+ * HadoopCatalog} and {@link HiveCatalog}.
+ * <p>
+ * The migration method is to keep the data file in the original hive table unchanged, and then create a new iceberg
+ * table, the new iceberg table use the data file of hive, and generate new metadata for the iceberg table.
+ * <p>
+ * In order to prevent data from being written to hive during the migration process so that new data cannot be migrated.
+ * so when we do the migration, we need to stop the writing hive job first, then migrate, and finally modify the logic
+ * to reading and writing iceberg table,If migrate failed, we will clean the iceberg table and metadata.If unfortunately
+ * the clean failed, you may need to manually clean the iceberg table and manifests.
+ */
+public class MigrateAction implements Action<List<ManifestFile>> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(MigrateAction.class);
+
+  private static final String PARQUET_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 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 = HiveSchemaUtil.spec(icebergSchema, partitionList);
+
+    FileFormat fileFormat = getHiveFileFormat(hiveTable);
+
+    Namespace namespace = Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));

Review comment:
       yes ,I use the `ObjectArrays` in google guava to replace the `ArrayUtils `




-- 
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571645073



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.flink.api.common.functions.RichMapFunction;
+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.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;
+
+public class MigrateAction implements Action<List<ManifestFile>> {
+
+  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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  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, icebergTable, nameMapping, metricsConfig,
+                metadataLocation);
+      } else {
+        manifestFiles =
+            migratePartitionedTable(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name(),
+                metadataLocation);
+
+      }
+    } catch (Exception e) {
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Migrate", e);
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    manifestFiles.forEach(append::appendManifest);
+    append.commit();
+
+    flinkHiveCatalog.close();

Review comment:
       nit: move this into finally block?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571085366



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);

Review comment:
       I am concerned about how this works for huge tables with millions of files.
   
   - collecting all of them into a single list can cause memory pressure.
   - writing all of them into a single manifest file may also hurt read performance for partitioned tables, as we can't filter manifest files by partition value ranges in manifest list file. 
   
   Maybe we should commit the files by batches. Each Hive partition is a batch and all data files under one Hive partitions get written into a single manifest file. We can use `AppendFiles#appendManifest` API too.
   




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578665925



##########
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) {

Review comment:
       In the Spark version of this we just forbid migration to a HadoopCatalog if you attempt to set an alternative location, I think this is basically the same thing. :+1:
   




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r570812464



##########
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);

Review comment:
       iceberg orc use orc-core-nohive ,flink use orc-core. They will have some conflicts, so I can not  use sql to write data into hive tables




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571736437



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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);

Review comment:
       because the `org.apache.flink.table.catalog.hive.HiveCatalog` has be closed in the `MigrateAction`,if we use sql to do the drop,it will throw an `NullPointerException` exception




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-780517320


   > We can make a tool to do this migration work, just like the snapshot expired, iceberg provides Java api and a spark action, but for some large hive tables, if it is only done through the Java api, it may be very slow, using the engine (spark or flink) can increase the speed of migration. If we only migrate a small hive table, we can run the flink program on our own machine, just like a test case. What do you think?
   
   Sounds good. My only concern is the dependency on `StreamExecutionEnvironment` and other Flink constructs. That would cause Flink dependency, even if we do not use Flink for a minimal local migration. Is there a way to separate the logic and the parallel execution?
   
   What is the number of partitions/files when the parallelization becomes necessary? (Just out of curiosity, you might have some numbers)
   
   Thanks,
   Peter


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r577644366



##########
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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Failed to migrate hive table", e);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,
+                                                       String nameMapping, MetricsConfig metricsConfig,
+                                                       String metadataLocation)
+      throws Exception {
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<PartitionAndLocation> dataStream =
+        env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+    DataStream<ManifestFile> ds = dataStream.flatMap(migrateMapper);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveSourceTableName)).stream()
+        .collect(Collectors.toList());
+  }
+
+  private Namespace toNamespace() {

Review comment:
       nit: Not reused, and no parameters - hard to read. Usually it is better to keep this line where it is used




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578671093



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,406 @@
+/*
+ * 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.hive.common.FileUtils;
+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 PARQUET_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 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 = HiveSchemaUtil.spec(icebergSchema, partitionList);
+
+    FileFormat fileFormat = getHiveFileFormat(hiveTable);
+
+    Namespace namespace = Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));
+    TableIdentifier identifier = TableIdentifier.of(namespace, 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);

Review comment:
       I'm not sure if this is an issue in Hive, but in Spark there can be differences between file layout and schema of the table which lets files possibly not match the Iceberg schema. Adding in the name mapping protects against these cases




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578977060



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,406 @@
+/*
+ * 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.hive.common.FileUtils;
+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 PARQUET_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 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 = HiveSchemaUtil.spec(icebergSchema, partitionList);
+
+    FileFormat fileFormat = getHiveFileFormat(hiveTable);
+
+    Namespace namespace = Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));
+    TableIdentifier identifier = TableIdentifier.of(namespace, 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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private static void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private static FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PARQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,

Review comment:
       I read the code of `Spark3MigrateAction`, but I am not familiar with spark, I don’t know if I understand it correctly. In `Spark3MigrateAction`, source catalog and destination catalog, source table name and destination table name are the same, so after we had migrated a non-iceberg table  to the iceberg table, the read and write operations using spark may not require any changes.
   
   But for hive table and iceberg table, some changes will be needed. For example, we are using flink to read and write hive table, the sql like this: `SELECT * FROM hivecatalog.hivedb.hivetable;` After the migration is completed, the sql needs to be modified to `select * from iceberg_catalog.iceberg_db.iceberg_table`, so when users migrate the hive table to iceberg, they need to stop the writing hive job first, then migrate, and finally modify the logic to reading and writing iceberg table. So I think even if we renamed the source hive table, it does  not make much sence  to the user, because the user still needs to manually stop and modify the job of writing hive to write iceberg.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578082962



##########
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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Failed to migrate hive table", e);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {

Review comment:
       yes,I update it.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r577642648



##########
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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Failed to migrate hive table", e);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {

Review comment:
       nit: Could this be static?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578066603



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

Review comment:
       ok,I update to `org.apache.hadoop.hive.common.FileUtils.HIDDEN_FILES_PATH_FILTER`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r579252784



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -346,7 +346,7 @@ public void testPartitionTypes() throws Exception {
     assertRecords(run(), records, typesSchema);
   }
 
-  static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {
+  public static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {

Review comment:
       Maybe move this into a utility class now that it's public static?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r579564750



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,406 @@
+/*
+ * 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.hive.common.FileUtils;
+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 PARQUET_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 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 = HiveSchemaUtil.spec(icebergSchema, partitionList);
+
+    FileFormat fileFormat = getHiveFileFormat(hiveTable);
+
+    Namespace namespace = Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));
+    TableIdentifier identifier = TableIdentifier.of(namespace, 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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private static void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private static FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PARQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,

Review comment:
       So my idea is that in order to prevent this case, we recommend that users should  stop writing hive job first before migrating, then do the migration, and finally modify the writing and reading job of hive. 
   
   if migrate failed, we will do the clean work on catch block, If unfortunately the clean failed, the user may need to manually clean the iceberg table and manifests.
   
   I will add some comments on `MigrateAction`,what do you think ?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578667299



##########
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);

Review comment:
       This can also lead to memory pressure issues as it collects all manifest information for the table at the same time.
   




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578077112



##########
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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Failed to migrate hive table", e);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,
+                                                       String nameMapping, MetricsConfig metricsConfig,
+                                                       String metadataLocation)
+      throws Exception {
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<PartitionAndLocation> dataStream =
+        env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+    DataStream<ManifestFile> ds = dataStream.flatMap(migrateMapper);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveSourceTableName)).stream()

Review comment:
       because we need to wait for all flink sub task finished,and then submit complete data to iceberg.
   But it seems that `stream()` is redundant, I removed it




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-780975701


   > What is the number of partitions/files when the parallelization becomes necessary? (Just out of curiosity, you might have some numbers)
   
   Now, the parallelism setting of flink is divided into two situations. For unpartitioned tables, I think it may be a small table. The current parallelism is the default parallelism of flink. For partitioned tables, the parallelism of flink job is  the number of partitions, but  In order to prevent too many partitions, causing migration job to request a lot of resources, users can set a max parallelism. When the number of partitions is greater than this max parallelism, the actual parallelism of the flink migrate job uses the configured max parallelism.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r579575076



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -346,7 +346,7 @@ public void testPartitionTypes() throws Exception {
     assertRecords(run(), records, typesSchema);
   }
 
-  static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {
+  public static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {

Review comment:
       yes, I move it to `SimpleDataUtil`,this method can be used for the assertion of unsorted list,I have encountered similar needs in other PR




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571515416



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return files.size();
+  }
+
+  private Namespace toNamespace(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Namespace.of(icebergDbName);
+    } else {
+      String[] namespaces = new String[baseNamespace.levels().length + 1];
+      System.arraycopy(baseNamespace.levels(), 0, namespaces, 0, baseNamespace.levels().length);
+      namespaces[baseNamespace.levels().length] = icebergDbName;
+      return Namespace.of(namespaces);
+    }
+  }
+
+  private List<DataFile> importPartitions(PartitionSpec spec, ObjectPath tableSource,
+                                          String nameMapping, FileFormat fileFormat,
+                                          MetricsConfig metricsConfig, String hiveTableName)
+      throws Exception {
+    List<CatalogPartitionSpec> partitionSpecs = flinkHiveCatalog.listPartitions(tableSource);
+    List<PartitionAndLocation> inputs = Lists.newArrayList();
+    for (CatalogPartitionSpec partitionSpec : partitionSpecs) {
+      Partition partition =
+          flinkHiveCatalog.getHivePartition(flinkHiveCatalog.getHiveTable(tableSource), partitionSpec);
+      inputs.add(
+          new PartitionAndLocation(partition.getSd().getLocation(), Maps.newHashMap(partitionSpec.getPartitionSpec())));
+    }
+
+    int size = partitionSpecs.size();
+    int parallelism = Math.min(size, maxParallelism);
+
+    DataStream<PartitionAndLocation> dataStream = env.fromCollection(inputs);
+    MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+    DataStream<List<DataFile>> ds = dataStream.map(migrateMap).setParallelism(parallelism);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveTableName)).stream()
+        .flatMap(Collection::stream).collect(Collectors.toList());
+  }
+
+  private static class MigrateMap implements MapFunction<PartitionAndLocation, List<DataFile>> {
+    private final PartitionSpec spec;
+    private final String nameMappingString;
+    private final FileFormat fileFormat;
+    private final MetricsConfig metricsConfig;
+
+    MigrateMap(PartitionSpec spec, String nameMapping, FileFormat fileFormat, MetricsConfig metricsConfig) {
+      this.spec = spec;
+      this.nameMappingString = nameMapping;
+      this.fileFormat = fileFormat;
+      this.metricsConfig = metricsConfig;
+    }
+
+    @Override
+    public List<DataFile> map(PartitionAndLocation map) {
+      Map<String, String> partitions = map.getMap();
+      String location = map.getLocation();
+      Configuration conf = HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration());
+      NameMapping nameMapping = nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null;
+      List<DataFile> files;
+      switch (fileFormat) {

Review comment:
       I lookup the [hive doc](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-CreateTableCreate/Drop/TruncateTable), when we  create  a hive table , specify the table format by `STORED AS file_format`, and the table format is table level properties. I think we cannot create different formats for the same 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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578070339



##########
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) {

Review comment:
       because the `HadoopCatalog`[ has a check](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java#L440), if we set a location for `HadoopCatalog`, it is different from the default,it will throw an exception . so I create the `HadoopCatalog` without  pass the location.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r579567736



##########
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);

Review comment:
       I think of another optimization solution. Instead of collecting all the manifests, we can collect the path of manifests, and finally construct the manifests according to the manifests path, and then submit them in batches.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578667299



##########
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);

Review comment:
       This can also lead to memory pressure issues as it collects all manifest information for the table at the same time.
   
   https://github.com/apache/iceberg/pull/2217/files#diff-50f83db1271f286f324eedd65c06dcae46e0827c2428c0de712a544b53698353R151




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r577636855



##########
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:
       Generally it is a good practice to add every information to the new exception (cause, message, etc), and if we are throwing one the we do not log the original one. Otherwise we will see the exception multiple times in the logs and it will be confusing.
   
   Is there any particular reason for logging and rethrowing here?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r579570784



##########
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);

Review comment:
       Isn't this already the case? `ManifestFile` contains the `manifest_path` field (for location URI) + other metadata fields. It is not very big. maybe a few hundred bytes.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r580718707



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -346,7 +346,7 @@ public void testPartitionTypes() throws Exception {
     assertRecords(run(), records, typesSchema);
   }
 
-  static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {
+  public static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {

Review comment:
       @pvary  , Could you give me some cases in the Hive module? Let me see if we can move them to data or core module




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571758219



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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) {
+          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"});
+      }
+    }
+
+    List<ManifestFile> manifestFiles =
+        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 manifestFiles count.", 1, manifestFiles.size());
+
+    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, TableNotExistException {
+    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());
+    getTableEnv().getConfig().setSqlDialect(SqlDialect.DEFAULT);
+    String hiveLocation = flinkHiveCatalog.getHiveTable(new ObjectPath(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME))
+        .getSd().getLocation();
+
+    List<Object[]> expected = Lists.newArrayList();
+    String[] partitions = new String[] {"iceberg", "flink"};
+    for (String partitionValue : partitions) {
+      String partitionPath = hiveLocation + "/p=" + partitionValue;
+
+      Partition hivePartition = createHivePartition(format, partitionPath, partitionValue);
+      metastoreClient.add_partition(hivePartition);
+
+      Partition partition =
+          metastoreClient.getPartition(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME, "p=" + partitionValue);
+      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());
+
+      try (

Review comment:
       That is right. I forgot we are inserting into Hive tables




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571646351



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.flink.api.common.functions.RichMapFunction;
+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.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;
+
+public class MigrateAction implements Action<List<ManifestFile>> {
+
+  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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  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, icebergTable, nameMapping, metricsConfig,
+                metadataLocation);
+      } else {
+        manifestFiles =
+            migratePartitionedTable(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name(),
+                metadataLocation);
+
+      }
+    } catch (Exception e) {
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Migrate", e);
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    manifestFiles.forEach(append::appendManifest);
+    append.commit();
+
+    flinkHiveCatalog.close();
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,
+                                                       Table icebergTable, String nameMapping,
+                                                       MetricsConfig metricsConfig, String metadataLocation)
+      throws Exception {
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<PartitionAndLocation> dataStream =
+        env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+    DataStream<List<ManifestFile>> ds = dataStream.map(migrateMapper);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+        .flatMap(Collection::stream).collect(Collectors.toList());
+  }
+
+  private Namespace toNamespace() {
+    return Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));
+  }
+
+  private List<ManifestFile> migratePartitionedTable(PartitionSpec spec, ObjectPath tableSource,
+                                                     String nameMapping, FileFormat fileFormat,
+                                                     MetricsConfig metricsConfig, String hiveTableName,
+                                                     String metadataLocation)
+      throws Exception {
+    List<CatalogPartitionSpec> partitionSpecs = flinkHiveCatalog.listPartitions(tableSource);
+    List<PartitionAndLocation> inputs = Lists.newArrayList();
+    for (CatalogPartitionSpec partitionSpec : partitionSpecs) {
+      Partition partition =
+          flinkHiveCatalog.getHivePartition(flinkHiveCatalog.getHiveTable(tableSource), partitionSpec);
+      inputs.add(
+          new PartitionAndLocation(partition.getSd().getLocation(), Maps.newHashMap(partitionSpec.getPartitionSpec())));
+    }
+
+    int size = partitionSpecs.size();
+    int parallelism = Math.min(size, maxParallelism);
+
+    DataStream<PartitionAndLocation> dataStream = env.fromCollection(inputs);
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<List<ManifestFile>> ds = dataStream.map(migrateMapper).setParallelism(parallelism);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveTableName)).stream()
+        .flatMap(Collection::stream).collect(Collectors.toList());
+  }
+
+  private static class MigrateMapper extends RichMapFunction<PartitionAndLocation, List<ManifestFile>> {
+    private final PartitionSpec spec;
+    private final String nameMappingString;
+    private final FileFormat fileFormat;
+    private final MetricsConfig metricsConfig;
+    private final String metadataLocation;
+
+    MigrateMapper(PartitionSpec spec, String nameMapping, FileFormat fileFormat, MetricsConfig metricsConfig,
+                  String metadataLocation) {
+      this.spec = spec;
+      this.nameMappingString = nameMapping;
+      this.fileFormat = fileFormat;
+      this.metricsConfig = metricsConfig;
+      this.metadataLocation = metadataLocation;
+    }
+
+    @Override
+    public List<ManifestFile> map(PartitionAndLocation partitionAndLocation) {
+      Map<String, String> partitions = partitionAndLocation.getPartitionSpec();
+      String location = partitionAndLocation.getLocation();
+      Configuration conf = HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration());
+      NameMapping nameMapping = nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null;
+      List<DataFile> files;
+      switch (fileFormat) {
+        case PARQUET:
+          files = listParquetPartition(partitions, location, spec, conf, metricsConfig);
+          break;
+
+        case ORC:
+          files = listOrcPartition(partitions, location, spec, conf, metricsConfig, nameMapping);
+          break;
+
+        default:
+          throw new UnsupportedOperationException("Unsupported file format");
+      }
+
+      return buildManifest(conf, spec, metadataLocation, files);
+    }
+
+    private List<ManifestFile> buildManifest(Configuration conf, PartitionSpec partitionSpec,
+                                             String basePath, List<DataFile> dataFiles) {
+      if (dataFiles.size() > 0) {
+        FileIO io = new HadoopFileIO(conf);
+        int subTaskId = getRuntimeContext().getIndexOfThisSubtask();
+        int attemptId = getRuntimeContext().getAttemptNumber();
+        String suffix = String.format("%d-task-%d-manifest", subTaskId, attemptId);

Review comment:
       nit: move `manifest` to the beginning? maybe sth like `manifest-task%d-attempt%d` or `manifest-%d-%d` just to be consistent?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-775639083


   LGTM


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571519281



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);

Review comment:
       thanks for  your suggesion,I will refactor this logic




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571196963



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return files.size();
+  }
+
+  private Namespace toNamespace(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Namespace.of(icebergDbName);
+    } else {
+      String[] namespaces = new String[baseNamespace.levels().length + 1];
+      System.arraycopy(baseNamespace.levels(), 0, namespaces, 0, baseNamespace.levels().length);
+      namespaces[baseNamespace.levels().length] = icebergDbName;
+      return Namespace.of(namespaces);
+    }
+  }
+
+  private List<DataFile> importPartitions(PartitionSpec spec, ObjectPath tableSource,
+                                          String nameMapping, FileFormat fileFormat,
+                                          MetricsConfig metricsConfig, String hiveTableName)
+      throws Exception {
+    List<CatalogPartitionSpec> partitionSpecs = flinkHiveCatalog.listPartitions(tableSource);
+    List<PartitionAndLocation> inputs = Lists.newArrayList();
+    for (CatalogPartitionSpec partitionSpec : partitionSpecs) {
+      Partition partition =
+          flinkHiveCatalog.getHivePartition(flinkHiveCatalog.getHiveTable(tableSource), partitionSpec);
+      inputs.add(
+          new PartitionAndLocation(partition.getSd().getLocation(), Maps.newHashMap(partitionSpec.getPartitionSpec())));
+    }
+
+    int size = partitionSpecs.size();
+    int parallelism = Math.min(size, maxParallelism);
+
+    DataStream<PartitionAndLocation> dataStream = env.fromCollection(inputs);
+    MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+    DataStream<List<DataFile>> ds = dataStream.map(migrateMap).setParallelism(parallelism);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveTableName)).stream()
+        .flatMap(Collection::stream).collect(Collectors.toList());
+  }
+
+  private static class MigrateMap implements MapFunction<PartitionAndLocation, List<DataFile>> {
+    private final PartitionSpec spec;
+    private final String nameMappingString;
+    private final FileFormat fileFormat;
+    private final MetricsConfig metricsConfig;
+
+    MigrateMap(PartitionSpec spec, String nameMapping, FileFormat fileFormat, MetricsConfig metricsConfig) {
+      this.spec = spec;
+      this.nameMappingString = nameMapping;
+      this.fileFormat = fileFormat;
+      this.metricsConfig = metricsConfig;
+    }
+
+    @Override
+    public List<DataFile> map(PartitionAndLocation map) {
+      Map<String, String> partitions = map.getMap();
+      String location = map.getLocation();
+      Configuration conf = HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration());
+      NameMapping nameMapping = nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null;
+      List<DataFile> files;
+      switch (fileFormat) {
+        case PARQUET:
+          files = listParquetPartition(partitions, location, spec, conf, metricsConfig, nameMapping);
+          break;
+
+        case ORC:
+          files = listOrcPartition(partitions, location, spec, conf, metricsConfig, nameMapping);
+          break;
+
+        default:
+          throw new UnsupportedOperationException("Unsupported file format");
+      }
+
+      return files;
+    }
+
+    private List<DataFile> listOrcPartition(Map<String, String> partitionPath, String partitionUri,
+                                            PartitionSpec partitionSpec, Configuration conf,
+                                            MetricsConfig metricsSpec, NameMapping mapping) {
+      try {
+        Path partition = new Path(partitionUri);
+        FileSystem fs = partition.getFileSystem(conf);
+
+        return Arrays.stream(fs.listStatus(partition, HIDDEN_PATH_FILTER))
+            .filter(FileStatus::isFile)
+            .map(stat -> {
+              Metrics metrics = OrcMetrics.fromInputFile(HadoopInputFile.fromPath(stat.getPath(), conf),
+                  metricsSpec, mapping);
+              String partitionKey = partitionSpec.fields().stream()
+                  .map(PartitionField::name)
+                  .map(name -> String.format("%s=%s", name, partitionPath.get(name)))
+                  .collect(Collectors.joining("/"));
+
+              return DataFiles.builder(partitionSpec)
+                  .withPath(stat.getPath().toString())
+                  .withFormat(FileFormat.ORC.name())

Review comment:
       nit: there is a `withFormat` overload that takes the enum directly. we don't have to call `.name()`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r580448516



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -346,7 +346,7 @@ public void testPartitionTypes() throws Exception {
     assertRecords(run(), records, typesSchema);
   }
 
-  static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {
+  public static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) {

Review comment:
       We have something like it in the Hive tests as well, and I would guess Spark has one too 😄 
   Do we have some other place even more accessible?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571192409



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return files.size();
+  }
+
+  private Namespace toNamespace(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Namespace.of(icebergDbName);
+    } else {
+      String[] namespaces = new String[baseNamespace.levels().length + 1];
+      System.arraycopy(baseNamespace.levels(), 0, namespaces, 0, baseNamespace.levels().length);
+      namespaces[baseNamespace.levels().length] = icebergDbName;
+      return Namespace.of(namespaces);
+    }
+  }
+
+  private List<DataFile> importPartitions(PartitionSpec spec, ObjectPath tableSource,
+                                          String nameMapping, FileFormat fileFormat,
+                                          MetricsConfig metricsConfig, String hiveTableName)
+      throws Exception {
+    List<CatalogPartitionSpec> partitionSpecs = flinkHiveCatalog.listPartitions(tableSource);
+    List<PartitionAndLocation> inputs = Lists.newArrayList();
+    for (CatalogPartitionSpec partitionSpec : partitionSpecs) {
+      Partition partition =
+          flinkHiveCatalog.getHivePartition(flinkHiveCatalog.getHiveTable(tableSource), partitionSpec);
+      inputs.add(
+          new PartitionAndLocation(partition.getSd().getLocation(), Maps.newHashMap(partitionSpec.getPartitionSpec())));
+    }
+
+    int size = partitionSpecs.size();
+    int parallelism = Math.min(size, maxParallelism);
+
+    DataStream<PartitionAndLocation> dataStream = env.fromCollection(inputs);
+    MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+    DataStream<List<DataFile>> ds = dataStream.map(migrateMap).setParallelism(parallelism);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveTableName)).stream()
+        .flatMap(Collection::stream).collect(Collectors.toList());
+  }
+
+  private static class MigrateMap implements MapFunction<PartitionAndLocation, List<DataFile>> {
+    private final PartitionSpec spec;
+    private final String nameMappingString;
+    private final FileFormat fileFormat;
+    private final MetricsConfig metricsConfig;
+
+    MigrateMap(PartitionSpec spec, String nameMapping, FileFormat fileFormat, MetricsConfig metricsConfig) {
+      this.spec = spec;
+      this.nameMappingString = nameMapping;
+      this.fileFormat = fileFormat;
+      this.metricsConfig = metricsConfig;
+    }
+
+    @Override
+    public List<DataFile> map(PartitionAndLocation map) {
+      Map<String, String> partitions = map.getMap();
+      String location = map.getLocation();
+      Configuration conf = HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration());
+      NameMapping nameMapping = nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null;
+      List<DataFile> files;
+      switch (fileFormat) {

Review comment:
       could a table have mixed file formats?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-781538336


   Also adding @RussellSpitzer who did the snapshot and migrate actions.
   
   Russell, could you take a look to make sure the migration path is similar for both query engines?


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571065568



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;

Review comment:
       nit: it seems that Iceberg convention is to have more full name. like `Db` -> `Database`. sam thing for the `hiveSourceDbName` above




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571793969



##########
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() {

Review comment:
       because the `FlinkTestBase#getTableEnv` only provide a `TableEnvironment`, but in the `MigrateAction`,we need a `StreamExecutionEnvironment` to get `DataStream` by `StreamExecutionEnvironment#fromElements`.  
   
   Simply , we need to do the migrate by flink api instead of flink sql




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r577613319



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

Review comment:
       Could we use `org.apache.hadoop.hive.common.FileUtils.HIDDEN_FILES_PATH_FILTER` instead?
   Or we do want depend on Hive stuff?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571501642



##########
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() {

Review comment:
       why override it? what is the reason to `enableCheckpointing` and set parallelism to 1?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571663472



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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) {
+          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"});
+      }
+    }
+
+    List<ManifestFile> manifestFiles =
+        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 manifestFiles count.", 1, manifestFiles.size());
+
+    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, TableNotExistException {
+    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());
+    getTableEnv().getConfig().setSqlDialect(SqlDialect.DEFAULT);
+    String hiveLocation = flinkHiveCatalog.getHiveTable(new ObjectPath(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME))
+        .getSd().getLocation();
+
+    List<Object[]> expected = Lists.newArrayList();
+    String[] partitions = new String[] {"iceberg", "flink"};
+    for (String partitionValue : partitions) {
+      String partitionPath = hiveLocation + "/p=" + partitionValue;
+
+      Partition hivePartition = createHivePartition(format, partitionPath, partitionValue);
+      metastoreClient.add_partition(hivePartition);
+
+      Partition partition =
+          metastoreClient.getPartition(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME, "p=" + partitionValue);
+      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());
+
+      try (

Review comment:
       nit: slightly simpler version.
   ```
       GenericAppenderHelper dataAppender = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER);
       final List<Record> batch1 = RandomGenericData.generate(table.schema(), 2, 0L);
       dataAppender.appendToTable(batch1);
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r577621222



##########
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);

Review comment:
       Why do not we use a list of the `DataFile` objects instead of the manifests?
   ```
         AppendFiles append = icebergTable.newAppend();
         dataFiles.forEach(append::appendFile);
         append.commit();
   ```
   
   In this way we would not have to generate the manifests by hand.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r572198700



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -294,27 +292,26 @@ private Namespace toNamespace() {
       return buildManifest(conf, spec, metadataLocation, files);
     }
 
-    private List<ManifestFile> buildManifest(Configuration conf, PartitionSpec partitionSpec,
-                                             String basePath, List<DataFile> dataFiles) {
+    private ManifestFile buildManifest(Configuration conf, PartitionSpec partitionSpec,
+                                       String basePath, List<DataFile> dataFiles) {
       if (dataFiles.size() > 0) {
         FileIO io = new HadoopFileIO(conf);
         int subTaskId = getRuntimeContext().getIndexOfThisSubtask();
         int attemptId = getRuntimeContext().getAttemptNumber();
-        String suffix = String.format("%d-task-%d-manifest", subTaskId, attemptId);
+        String suffix = String.format("manifest-%d-%d", subTaskId, attemptId);
         Path location = new Path(basePath, suffix);
         String outputPath = FileFormat.AVRO.addExtension(location.toString());
         OutputFile outputFile = io.newOutputFile(outputPath);
         ManifestWriter<DataFile> writer = ManifestFiles.write(partitionSpec, outputFile);
-        try (ManifestWriter<DataFile> writerRef = writer) {
-          dataFiles.forEach(writerRef::add);
+        try (ManifestWriter<DataFile> manifestWriter = writer) {
+          dataFiles.forEach(manifestWriter::add);
         } catch (IOException e) {
           throw new UncheckedIOException("Unable to close the manifest writer", e);
         }
 
-        ManifestFile manifestFile = writer.toManifestFile();
-        return Lists.newArrayList(manifestFile);
+        return writer.toManifestFile();
       } else {
-        return Lists.newArrayList();
+        return null;

Review comment:
       maybe also add the case of no files in a Hive partition in the unit test




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571086916



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);

Review comment:
       extract this to a separate method `migrateUnpartitionedTable`?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhaoyuxiaoxiao commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhaoyuxiaoxiao commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-917782631


   @zhangjun0x01 hello,How does this example work  FlinkSQL? Can you give an example?thanks


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571265021



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,374 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.flink.api.common.functions.MapFunction;
+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.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.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.HadoopInputFile;
+import org.apache.iceberg.hive.HiveSchemaUtil;
+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.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MigrateAction implements Action<Integer> {
+  private static final Logger LOG = 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 String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat";
+  private static final PathFilter HIDDEN_PATH_FILTER =
+      p -> !p.getName().startsWith("_") && !p.getName().startsWith(".");
+
+  private final StreamExecutionEnvironment env;
+  private final HiveCatalog flinkHiveCatalog; // the HiveCatalog of flink
+  private final String hiveSourceDbName;
+  private final String hiveSourceTableName;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final String icebergDbName;
+  private final String icebergTableName;
+  private int maxParallelism;
+
+  public MigrateAction(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, String hiveSourceDbName,
+                       String hiveSourceTableName, Catalog icebergCatalog, Namespace baseNamespace,
+                       String icebergDbName,
+                       String icebergTableName) {
+    this.env = env;
+    this.flinkHiveCatalog = flinkHiveCatalog;
+    this.hiveSourceDbName = hiveSourceDbName;
+    this.hiveSourceTableName = hiveSourceTableName;
+    this.icebergCatalog = icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    this.icebergDbName = icebergDbName;
+    this.icebergTableName = icebergTableName;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  public Integer execute() {
+    // hive source table
+    ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName);
+    org.apache.hadoop.hive.metastore.api.Table hiveTable;
+    flinkHiveCatalog.open();
+    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);
+
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      case AVRO_INPUT_FORMAT:
+        fileFormat = FileFormat.AVRO;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    TableIdentifier identifier = TableIdentifier.of(toNamespace(baseNamespace), 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 nameMapping =
+        PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null);
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(icebergTable.properties());
+
+    List<DataFile> files = null;
+    if (spec.isUnpartitioned()) {
+      MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+      DataStream<PartitionAndLocation> dataStream =
+          env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+      DataStream<List<DataFile>> ds = dataStream.map(migrateMap);
+      try {
+        files = Lists.newArrayList(ds.executeAndCollect("migrate table :" + icebergTable.name())).stream()
+            .flatMap(Collection::stream).collect(Collectors.toList());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      try {
+        files = importPartitions(spec, tableSource, nameMapping, fileFormat, metricsConfig, icebergTable.name());
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    AppendFiles append = icebergTable.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return files.size();
+  }
+
+  private Namespace toNamespace(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return Namespace.of(icebergDbName);
+    } else {
+      String[] namespaces = new String[baseNamespace.levels().length + 1];
+      System.arraycopy(baseNamespace.levels(), 0, namespaces, 0, baseNamespace.levels().length);
+      namespaces[baseNamespace.levels().length] = icebergDbName;
+      return Namespace.of(namespaces);
+    }
+  }
+
+  private List<DataFile> importPartitions(PartitionSpec spec, ObjectPath tableSource,
+                                          String nameMapping, FileFormat fileFormat,
+                                          MetricsConfig metricsConfig, String hiveTableName)
+      throws Exception {
+    List<CatalogPartitionSpec> partitionSpecs = flinkHiveCatalog.listPartitions(tableSource);
+    List<PartitionAndLocation> inputs = Lists.newArrayList();
+    for (CatalogPartitionSpec partitionSpec : partitionSpecs) {
+      Partition partition =
+          flinkHiveCatalog.getHivePartition(flinkHiveCatalog.getHiveTable(tableSource), partitionSpec);
+      inputs.add(
+          new PartitionAndLocation(partition.getSd().getLocation(), Maps.newHashMap(partitionSpec.getPartitionSpec())));
+    }
+
+    int size = partitionSpecs.size();
+    int parallelism = Math.min(size, maxParallelism);
+
+    DataStream<PartitionAndLocation> dataStream = env.fromCollection(inputs);
+    MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig);
+    DataStream<List<DataFile>> ds = dataStream.map(migrateMap).setParallelism(parallelism);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveTableName)).stream()
+        .flatMap(Collection::stream).collect(Collectors.toList());
+  }
+
+  private static class MigrateMap implements MapFunction<PartitionAndLocation, List<DataFile>> {
+    private final PartitionSpec spec;
+    private final String nameMappingString;
+    private final FileFormat fileFormat;
+    private final MetricsConfig metricsConfig;
+
+    MigrateMap(PartitionSpec spec, String nameMapping, FileFormat fileFormat, MetricsConfig metricsConfig) {
+      this.spec = spec;
+      this.nameMappingString = nameMapping;
+      this.fileFormat = fileFormat;
+      this.metricsConfig = metricsConfig;
+    }
+
+    @Override
+    public List<DataFile> map(PartitionAndLocation map) {
+      Map<String, String> partitions = map.getMap();
+      String location = map.getLocation();
+      Configuration conf = HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration());
+      NameMapping nameMapping = nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null;
+      List<DataFile> files;
+      switch (fileFormat) {
+        case PARQUET:
+          files = listParquetPartition(partitions, location, spec, conf, metricsConfig, nameMapping);
+          break;
+
+        case ORC:
+          files = listOrcPartition(partitions, location, spec, conf, metricsConfig, nameMapping);
+          break;
+
+        default:
+          throw new UnsupportedOperationException("Unsupported file format");
+      }
+
+      return files;
+    }
+
+    private List<DataFile> listOrcPartition(Map<String, String> partitionPath, String partitionUri,
+                                            PartitionSpec partitionSpec, Configuration conf,
+                                            MetricsConfig metricsSpec, NameMapping mapping) {
+      try {
+        Path partition = new Path(partitionUri);
+        FileSystem fs = partition.getFileSystem(conf);
+
+        return Arrays.stream(fs.listStatus(partition, HIDDEN_PATH_FILTER))
+            .filter(FileStatus::isFile)
+            .map(stat -> {
+              Metrics metrics = OrcMetrics.fromInputFile(HadoopInputFile.fromPath(stat.getPath(), conf),
+                  metricsSpec, mapping);
+              String partitionKey = partitionSpec.fields().stream()
+                  .map(PartitionField::name)
+                  .map(name -> String.format("%s=%s", name, partitionPath.get(name)))
+                  .collect(Collectors.joining("/"));
+
+              return DataFiles.builder(partitionSpec)
+                  .withPath(stat.getPath().toString())
+                  .withFormat(FileFormat.ORC.name())
+                  .withFileSizeInBytes(stat.getLen())
+                  .withMetrics(metrics)
+                  .withPartitionPath(partitionKey)
+                  .build();
+
+            }).collect(Collectors.toList());
+      } catch (IOException e) {
+        throw new UncheckedIOException(String.format("Unable to list files in partition: %s", partitionUri), e);
+      }
+    }
+
+    private static List<DataFile> listParquetPartition(Map<String, String> partitionPath, String partitionUri,
+                                                       PartitionSpec spec, Configuration conf,
+                                                       MetricsConfig metricsSpec, NameMapping mapping) {
+      try {
+        Path partition = new Path(partitionUri);
+        FileSystem fs = partition.getFileSystem(conf);
+
+        return Arrays.stream(fs.listStatus(partition, HIDDEN_PATH_FILTER))
+            .filter(FileStatus::isFile)
+            .map(stat -> {
+              Metrics metrics;
+              try {
+                ParquetMetadata metadata = ParquetFileReader.readFooter(conf, stat);
+                metrics = ParquetUtil.footerMetrics(metadata, Stream.empty(), metricsSpec, mapping);
+              } catch (IOException e) {
+                throw new UncheckedIOException(
+                    String.format("Unable to read the footer of the parquet file: %s", stat.getPath()), e);
+              }
+
+              String partitionKey = spec.fields().stream()
+                  .map(PartitionField::name)
+                  .map(name -> String.format("%s=%s", name, partitionPath.get(name)))
+                  .collect(Collectors.joining("/"));
+
+              return DataFiles.builder(spec)
+                  .withPath(stat.getPath().toString())
+                  .withFormat(FileFormat.PARQUET.name())
+                  .withFileSizeInBytes(stat.getLen())
+                  .withMetrics(metrics)
+                  .withPartitionPath(partitionKey)
+                  .build();
+
+            }).collect(Collectors.toList());
+      } catch (IOException e) {
+        throw new UncheckedIOException(String.format("Unable to list files in partition: %s", partitionUri), e);
+      }
+    }
+  }
+
+  public static class PartitionAndLocation implements java.io.Serializable {
+    private String location;
+    private Map<String, String> map;
+
+    public PartitionAndLocation(String location, Map<String, String> map) {

Review comment:
       nit: for better readability, should we rename `map` to `partitionSpec`?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] pvary commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-977799683


   > I'm using a Trino environment where it's easy to write Hive parquet files from pandas, but I haven't found the fast path from pandas to Iceberg. (Row-by-row insertion is too slow.) I'm here because I thought table migration might be an answer...
   
   We implemented Hive(external) to Iceberg migration in the Hive repo: https://issues.apache.org/jira/browse/HIVE-25008
   
   Not sure it helps you, but just for the record is someone stumbles to this issue again
   


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571502279



##########
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);

Review comment:
       can we test the case with multiple partitions?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571500713



##########
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);

Review comment:
       nit: drop the `TestMigrateAction.`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578961320



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -0,0 +1,406 @@
+/*
+ * 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.hive.common.FileUtils;
+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 PARQUET_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 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 = HiveSchemaUtil.spec(icebergSchema, partitionList);
+
+    FileFormat fileFormat = getHiveFileFormat(hiveTable);
+
+    Namespace namespace = Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName}));
+    TableIdentifier identifier = TableIdentifier.of(namespace, 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);

Review comment:
       yes, I add the name mapping.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#issuecomment-780382563


   > I have just found this PR when I was wondering how to migrate existing Hive tables to Iceberg tables.
   > My use-case is that I have an existing Hive table, and I would like to convert it to a Hive table backed by an Iceberg table in-place, and without moving the actual data. I would like to create the corresponding manifest files and the first snapshot using the existing files.
   > 
   > When I sketched the code I found that I was listing the partitions / creating `DataFile` objects / creating a new Iceberg table and adding the data files to it. And then I have found that you have already did the same in the `Actions.migrateHive2Iceberg`. At first glance there is not too much Flink specific code in this change. Would it be hard to create a general tool for the migration instead of a Flink specific action?
   > 
   > Thanks,
   > Peter
   
   We can make a tool to do this migration work, just like the snapshot expired, iceberg provides Java api and a spark action, but for some large hive tables, if it is only done through the Java api, it may be very slow, using  the engine (spark or flink) can increase the speed of migration. If we only migrate a small hive table, we can run the flink program on our own machine, just like a test case.  What do you think?


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r572502284



##########
File path: flink/src/main/java/org/apache/iceberg/flink/actions/MigrateAction.java
##########
@@ -294,27 +292,26 @@ private Namespace toNamespace() {
       return buildManifest(conf, spec, metadataLocation, files);
     }
 
-    private List<ManifestFile> buildManifest(Configuration conf, PartitionSpec partitionSpec,
-                                             String basePath, List<DataFile> dataFiles) {
+    private ManifestFile buildManifest(Configuration conf, PartitionSpec partitionSpec,
+                                       String basePath, List<DataFile> dataFiles) {
       if (dataFiles.size() > 0) {
         FileIO io = new HadoopFileIO(conf);
         int subTaskId = getRuntimeContext().getIndexOfThisSubtask();
         int attemptId = getRuntimeContext().getAttemptNumber();
-        String suffix = String.format("%d-task-%d-manifest", subTaskId, attemptId);
+        String suffix = String.format("manifest-%d-%d", subTaskId, attemptId);
         Path location = new Path(basePath, suffix);
         String outputPath = FileFormat.AVRO.addExtension(location.toString());
         OutputFile outputFile = io.newOutputFile(outputPath);
         ManifestWriter<DataFile> writer = ManifestFiles.write(partitionSpec, outputFile);
-        try (ManifestWriter<DataFile> writerRef = writer) {
-          dataFiles.forEach(writerRef::add);
+        try (ManifestWriter<DataFile> manifestWriter = writer) {
+          dataFiles.forEach(manifestWriter::add);
         } catch (IOException e) {
           throw new UncheckedIOException("Unable to close the manifest writer", e);
         }
 
-        ManifestFile manifestFile = writer.toManifestFile();
-        return Lists.newArrayList(manifestFile);
+        return writer.toManifestFile();
       } else {
-        return Lists.newArrayList();
+        return null;

Review comment:
       thanks for your suggestion,I add the test with empty table and empty partition




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r570812464



##########
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);

Review comment:
       iceberg orc use orc-core-nohive ,flink use orc-core. They will have some conflicts, so I can not  use sql to write data into hive tables




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571662600



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    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) {
+          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"});
+      }
+    }
+
+    List<ManifestFile> manifestFiles =
+        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 manifestFiles count.", 1, manifestFiles.size());
+
+    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, TableNotExistException {
+    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());
+    getTableEnv().getConfig().setSqlDialect(SqlDialect.DEFAULT);
+    String hiveLocation = flinkHiveCatalog.getHiveTable(new ObjectPath(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME))
+        .getSd().getLocation();
+
+    List<Object[]> expected = Lists.newArrayList();
+    String[] partitions = new String[] {"iceberg", "flink"};
+    for (String partitionValue : partitions) {
+      String partitionPath = hiveLocation + "/p=" + partitionValue;
+
+      Partition hivePartition = createHivePartition(format, partitionPath, partitionValue);
+      metastoreClient.add_partition(hivePartition);
+
+      Partition partition =
+          metastoreClient.getPartition(SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME, "p=" + partitionValue);
+      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());
+
+      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, partitionValue});
+        }
+      }
+    }
+
+    List<ManifestFile> manifestFiles =
+        Actions.migrateHive2Iceberg(env, flinkHiveCatalog, SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME,
+            validationCatalog, baseNamespace, DATABASE, TABLE_NAME_UNPARTITIONED).execute();

Review comment:
       is `TABLE_NAME_UNPARTITIONED ` a typo here? 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571766435



##########
File path: flink/src/test/java/org/apache/iceberg/flink/actions/TestMigrateAction.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.ManifestFile;
+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.Test;
+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 static HiveCatalog flinkHiveCatalog;
+  private StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+  private StreamTableEnvironment tEnv;
+
+  @BeforeClass
+  public static void createHiveDB() {
+    try {
+      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);
+
+      flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6");
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @AfterClass
+  public static void dropHiveDB() {
+    try {
+      metastoreClient.dropDatabase(SOURCE_HIVE_DB_NAME);
+      metastoreClient.close();
+    } catch (TException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    exec("USE CATALOG %s", catalogName);
+    exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE);
+
+    getTableEnv().registerCatalog(SOURCE_HIVE_CATALOG_NAME, flinkHiveCatalog);

Review comment:
       ok. forgot `getTableEnv()` is non static. Just conceptually, we don't need to create/register/delete catalog per test method.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r578080136



##########
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);
+      deleteManifests(icebergTable.io(), manifestFiles);
+      throw new RuntimeException("Failed to migrate hive table", e);
+    } finally {
+      flinkHiveCatalog.close();
+    }
+
+    return manifestFiles;
+  }
+
+  private void deleteManifests(FileIO io, List<ManifestFile> manifests) {
+    Tasks.foreach(manifests)
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .run(item -> io.deleteFile(item.path()));
+  }
+
+  private FileFormat getHiveFileFormat(org.apache.hadoop.hive.metastore.api.Table hiveTable) {
+    String hiveFormat = hiveTable.getSd().getInputFormat();
+    FileFormat fileFormat;
+    switch (hiveFormat) {
+      case PATQUET_INPUT_FORMAT:
+        fileFormat = FileFormat.PARQUET;
+        break;
+
+      case ORC_INPUT_FORMAT:
+        fileFormat = FileFormat.ORC;
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unsupported file format");
+    }
+
+    return fileFormat;
+  }
+
+  private String getMetadataLocation(Table table) {
+    return table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER);
+  }
+
+  private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation,
+                                                       String nameMapping, MetricsConfig metricsConfig,
+                                                       String metadataLocation)
+      throws Exception {
+    MigrateMapper migrateMapper = new MigrateMapper(spec, nameMapping, fileFormat, metricsConfig, metadataLocation);
+    DataStream<PartitionAndLocation> dataStream =
+        env.fromElements(new PartitionAndLocation(hiveLocation, Maps.newHashMap()));
+    DataStream<ManifestFile> ds = dataStream.flatMap(migrateMapper);
+    return Lists.newArrayList(ds.executeAndCollect("migrate table :" + hiveSourceTableName)).stream()
+        .collect(Collectors.toList());
+  }
+
+  private Namespace toNamespace() {

Review comment:
       There were a lot of codes for get Namespace before, so I extracted a method, but later simplified the logic,




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2217: Flink : migrate hive table to iceberg table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2217:
URL: https://github.com/apache/iceberg/pull/2217#discussion_r571500785



##########
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() {

Review comment:
       we need to close `HiveMetaStoreClient`, right?




----------------------------------------------------------------
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:
users@infra.apache.org



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