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/04 04:13:51 UTC

[GitHub] [iceberg] RussellSpitzer opened a new pull request #2210: Adds AddFiles Procedure

RussellSpitzer opened a new pull request #2210:
URL: https://github.com/apache/iceberg/pull/2210


   This procedure mimics our old support for "IMPORT DATA" but does not allow for dynamic
   overwriting of files in partitions. This will now require a seperate DELETE command to
   remove a partition. Other than that, the capabilites are identical to the previous functionality
   except now the functionality is a procedure rather than an SQL command.


----------------------------------------------------------------
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] rdsr commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       @aokolnychyi is the order among the partition columns assumed? I slightly prefer @rdblue's where order seems explicit




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {

Review comment:
       The changes in filter partition should let this import all values for which p1 = x and p2 = y and p3 is anything




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {

Review comment:
       We could probably add a method like `parseIdentifier` that would return `Identifier` without trying to resolve the catalog. Then we can check if it is a path identifier and then call the appropriate method to do the import. That way, we will parse the identifier only once and using the Spark parser. For path-based tables, `namespace` will be file format and `name` will contain 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add data files to target table %s because that table is partitioned, " +
+            "but the number of columns in the provided partition spec (%d) " +
+            "does not match the number of partitioned columns in table (%d)",
+              table.name(), partition.size(), partitionFields.size()));
+      }
+      partitionFields.forEach(field -> {
+        if (!partition.containsKey(field.name())) {
+          throw new IllegalArgumentException(
+            String.format(
+              "Cannot add files to target table %s. %s is partitioned but the specified partition spec " +
+              "refers to a column that is not partitioned: '%s'",
+                table.name(), table.name(), field.name())
+            );
+        }
+      });
+    } else {
+      if (partitionSpecPassed) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add files to target table %s which is not partitioned, but a partition spec was provided",

Review comment:
       nit: I think the error message should be now adapted a bit if we use the map as a filter with 0 to n partition columns.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);

Review comment:
       If we use the identifier approach, we no longer have to parse it 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long filesAdded = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].toLowerCase(Locale.ROOT).equals("orc") ||

Review comment:
       nit: `equalsIgnoreCase`?




----------------------------------------------------------------
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] talgos1 commented on pull request #2210: Adds AddFiles Procedure

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


   Hi, Im wondering if it's possible to add a specific file (or an unpartitioned parquet table) to a partitioned iceberg table (specific 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.

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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");

Review comment:
       I feel like this should be checked outside of `filterPartitions`. I think this utility method should focus only on filtering passed partitions. Also, I think the error message can be a bit more precise. In this case, we did not find partitions for the partitioned table rather than did not find files. 




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   @RussellSpitzer, could you please create issues for ORC and Avro tests? We already have an issue to not depend on Spark in-memory file index.


-- 
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataUnpartitionedOrc() {
+    createUnpartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedHive() {
+    createUnpartitionedHiveTable();
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '%s')",
+        catalogName, tableName, sourceTableName);
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedExtraCol() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String, foo string) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedMissingCol() {

Review comment:
       Do we need an `addDataPartitionedMissingCol`?
   
   What if one of the partition column is not in the target 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {

Review comment:
       Does it have to be public?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {

Review comment:
       partition -> partitionFilter?




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   Thanks for working on this, @RussellSpitzer! Let me take a look today.


----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add data files to target table %s because that table is partitioned, " +
+            "but the number of columns in the provided partition spec (%d) " +
+            "does not match the number of partitioned columns in table (%d)",
+              table.name(), partition.size(), partitionFields.size()));
+      }
+      partitionFields.forEach(field -> {
+        if (!partition.containsKey(field.name())) {
+          throw new IllegalArgumentException(
+            String.format(
+              "Cannot add files to target table %s. %s is partitioned but the specified partition spec " +
+              "refers to a column that is not partitioned: '%s'",
+                table.name(), table.name(), field.name())
+            );
+        }
+      });

Review comment:
       I think we need to add validation that all columns from Iceberg partition spec are present in the partition map provided by the user to avoid cases mentioned by @rdblue.
   
   I think the following should be illegal.
   
   ```
   add_files(
     table => 'db.iceberg_table', -- partitioned Iceberg table
     source_table => '`parquet`.`path/to/table/part_col_1=x`'
     partition => map('part_col_2', 'y')
   )
   ```




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,

Review comment:
       nit: Shall we use capital letters?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I had offline discussions with @RussellSpitzer and @rdblue and there is no ideal solution. Probably, we won't get around inferring partitions and since we will need that in other places, so we can probably support it now.
   
   It is fine to start with Spark's `InMemoryFileIndex` for now. However, we should migrate to our own solution and leverage it in other query engines during snapshot and migrate commands. I've created #2300 for this. 




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       We can't use the version in SparkUtils because unfortunately the Scala api for this method has added additional default args in Spark3 vs Spark2. In Scala this wouldn't be a problem, but in java we need to explicitly call the function with all args.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add data files to target table %s because that table is partitioned, " +
+            "but the number of columns in the provided partition spec (%d) " +
+            "does not match the number of partitioned columns in table (%d)",
+              table.name(), partition.size(), partitionFields.size()));
+      }
+      partitionFields.forEach(field -> {
+        if (!partition.containsKey(field.name())) {
+          throw new IllegalArgumentException(
+            String.format(
+              "Cannot add files to target table %s. %s is partitioned but the specified partition spec " +
+              "refers to a column that is not partitioned: '%s'",
+                table.name(), table.name(), field.name())
+            );
+        }
+      });

Review comment:
       I'd say the `partition` map should be required for partitioned 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       Alright, I think we all agree that `add_files` procedure should be limited to identity partitioning and, ideally, it should not infer partitions (i.e. we should not try to build a list of partitions in a given location). The primary use case for this procedure is partial migration and people can use SNAPSHOT or MIGRATE if they need to migrate all partitions at the same time. Instead of inferring a list of partitions ourselves, we better ask the user which partitions to import.
   
   That means we don't need to use `InMemoryFileIndex` right now. We will come back to it while adding support for migrating path-based tables. We should also take other query engines into account.
   
   The open question right now is whether we should support adding multiple partitions in the same call? That seems beneficial but how do we do this and what would it mean for the procedure API? In earlier discussions, @rdblue mentioned supporting something like `part_col1=*/part_col2=x` would be great. While it would be easier for us to support only a single partition, I'd vote for a better user experience.
   
   We could represent partition path as a string with wildcards but I'd suggest using a map.
   
   ```
   iceberg.system.add_files(
     table => 'db.tbl', -- required
     path => '/root/table/location', -- required
     format => 'parquet', -- required
     partition => map('p1', '*', 'p2', 'v2') -- required for partitioned tables
   )
   ```
   
   We may add support for `*` later but using a map instead of a string does not require us parsing the path to get individual 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] pvary commented on pull request #2210: Adds AddFiles Procedure

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


   > @pvary + @aokolnychyi Should be ready for another look if ya'll have time, Thanks in advance!
   
   LGTM on Hive related stuff. Only minors and mostly questions.
   When we start working on Hive migration definitely will move some of the code from the `SparkTableUtil`, `Spark3Util` to some more accessible place, but that's another story.
   Too much Spark code to be comfortable giving a +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] rdblue commented on pull request #2210: Adds AddFiles Procedure

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


   @talgos1, have you tried using a full file path as the source 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.

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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I am open to other alternatives but this idea seems to be the easiest among the options we discussed and we can actually avoid using the in-memory file index for now.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add data files to target table %s because that table is partitioned, " +
+            "but the number of columns in the provided partition spec (%d) " +
+            "does not match the number of partitioned columns in table (%d)",
+              table.name(), partition.size(), partitionFields.size()));
+      }
+      partitionFields.forEach(field -> {
+        if (!partition.containsKey(field.name())) {
+          throw new IllegalArgumentException(
+            String.format(
+              "Cannot add files to target table %s. %s is partitioned but the specified partition spec " +
+              "refers to a column that is not partitioned: '%s'",
+                table.name(), table.name(), field.name())
+            );
+        }
+      });
+    } else {
+      if (partitionSpecPassed) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add files to target table %s which is not partitioned, but a partition spec was provided",

Review comment:
       Something like `Cannot use a partition filter as the target table is not partitioned?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+import static org.apache.iceberg.spark.SparkTableUtil.MAP_JOINER;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private void importFileTable(Table table, Path tableLocation, String format, Map<String, String> partitionFilter) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
+    boolean sourceUnpartitioned = partitions.size() == 1 && partitions.get(0).getValues().isEmpty();
+    Preconditions.checkArgument(
+        (table.spec().isUnpartitioned() && sourceUnpartitioned) || !table.spec().isUnpartitioned(),
+        "Cannot add partitioned files to an unpartitioned table");
+    Preconditions.checkArgument(!partitions.isEmpty(),
+        "Cannot find any partitions in table %s", partitions);
+    List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);
+    Preconditions.checkArgument(!filteredPartitions.isEmpty(),
+        "Cannot find any partitions which match the given filter. Partition filter is %s",
+        MAP_JOINER.join(partitionFilter));
+    importPartitions(table, filteredPartitions);
+  }
+
+  private void importCatalogTable(Table table, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    String stagingLocation = getMetadataLocation(table);
+    TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent);
+    SparkTableUtil.importSparkTable(spark(), sourceTableIdentifier, table, stagingLocation, partitionFilter);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = getMetadataLocation(table);
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private String getMetadataLocation(Table table) {
+    String defaultValue = table.location() + "/metadata";
+    return table.properties().getOrDefault(TableProperties.WRITE_METADATA_LOCATION, defaultValue);
+  }
+
+  @Override
+  public String description() {
+    return "AddFiles";
+  }
+
+  private static void validatePartitionSpec(Table table, Map<String, String> partitionFilter) {
+    List<PartitionField> partitionFields = table.spec().fields();
+    Set<String> partitionNames = table.spec().fields().stream().map(PartitionField::name).collect(Collectors.toSet());
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partitionFilter.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Not enough partition columns
+      Preconditions.checkArgument(partitionFields.size() >= partitionFilter.size(),
+          "Cannot add data files to target table %s because that table is partitioned, " +
+              "but the number of columns in the provided partition filter (%d) " +
+              "is greater than the number of partitioned columns in table (%d)",
+          table.name(), partitionFilter.size(), partitionFields.size());
+
+      // Check for any non-identity partition columns
+      List<PartitionField> nonIdentityFields =
+          partitionFields.stream().filter(x -> !x.transform().isIdentity()).collect(Collectors.toList());
+      Preconditions.checkArgument(nonIdentityFields.isEmpty(),
+          "Cannot add data files to target table %s because that table is partitioned and contains non-identity" +
+              "partition transforms which will not be compatible. Found non-identity fields %s",
+          table.name(), nonIdentityFields);
+
+      // Check for any filters of non existent columns
+      List<String> unMatchedFilters =
+          partitionFilter.keySet().stream().filter(filterName -> !partitionNames.contains(filterName))
+              .collect(Collectors.toList());
+      Preconditions.checkArgument(unMatchedFilters.isEmpty(),
+          "Cannot add files to target table %s. %s is partitioned but the specified partition filter " +

Review comment:
       We would also get if somebody uses backtick quotes in the map.




-- 
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =

Review comment:
       Do we use this variable?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{

Review comment:
       I think these arg names are reasonable. I'd consider `partition_key`, `partition_values` or simply `partition` instead of `partition_value`. I tend to like `partition`, probably, as it is short.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long filesAdded = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&

Review comment:
       If we allow import of Avro, we enable converting Avro to Parquet partition at a time. There could be valid Avro cases as well. 




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I am open to other alternatives but this idea seems to be the easiest among the options we discussed and we can actually avoid using the in-memory file index for now. We will need it to determine all partitions (e.g. full migration).




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       Could we use reflection for this? Seems we had a similar issue in `HiveClientPool`.
   
   I understand `InMemoryFileIndex` is a low-level API and it may not be the best idea to rely on it. I'm ok with this as the first step. We may build our own logic if we need it in other query engines.
   
   I'd be also alright to always require a specific partition to import and not support importing all partitions in a given path, which should avoid the need to use Spark's `InMemoryFileIndex` to infer the underlying partitioning. As I wrote before, I think the primary use case for this is CREATE TABLE LIKE and then calling `add_files` for a couple of partitions to try reads/writes or migrating partition by partition if a table has too many partitions to migrate them at once. In both cases, there is a specific partition we have to import.
   
   If there is a need to import files in a given location, one can create an external Spark table pointing to the location, call `MSCK REPAIR TABLE` to infer partitions and add them to the metastore and then call either snapshot or migrate procedure.
   
   What are your thoughts, @RussellSpitzer @rdblue? Shall we bother ourselves to infer partitioning?
   
   




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+        // Forces Name based resolution instead of position based resolution
+        NameMapping mapping = MappingUtil.create(table.schema());
+        String mappingJson = NameMappingParser.toJson(mapping);
+        table.updateProperties()
+            .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+            .commit();
+      }
+
+      if (isFile || !partition.isEmpty()) {
+        // we do a list operation on the driver to import 1 file or 1 partition
+        PartitionSpec spec = table.spec();
+        MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+        String partitionURI;
+
+        if (isFile) {
+          partitionURI = dataPath.toString();
+        } else {
+          partitionURI = dataPath.toString() + toPartitionPath(partition);
+        }
+
+        List<DataFile> files = SparkTableUtil.listPartition(partition, partitionURI, format, spec, conf, metricsConfig);
+        if (files.size() == 0) {
+          throw new IllegalArgumentException(String.format("No files found for add_file command. Looking in URI %s",
+              partitionURI));
+        }
+
+        AppendFiles append = table.newAppend();
+        files.forEach(append::appendFile);
+        append.commit();
+      } else {
+        // Importing multiple partitions
+        List<SparkTableUtil.SparkPartition> partitions = Spark3Util.getPartitions(spark(), dataPath, format);
+        String stagingLocation = table.properties()
+            .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+        SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      Long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private String toPartitionPath(Map<String, String> partition) {

Review comment:
       Hm, this is a good point. 




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long filesAdded = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&

Review comment:
       What about Avro?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long filesAdded = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].toLowerCase(Locale.ROOT).equals("orc") ||
+            namespace[0].toLowerCase(Locale.ROOT).equals("parquet"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));

Review comment:
       nit: just return without an extra var?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +803,66 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),
+        Option.empty(),
+        fileStatusCache,
+        Option.empty(),
+        Option.empty());
+
+    org.apache.spark.sql.execution.datasources.PartitionSpec spec = fileIndex.partitionSpec();
+    StructType schema = spec.partitionColumns();
+    if (spec.partitions().isEmpty()) {
+      return ImmutableList.of(new SparkPartition(Collections.emptyMap(), rootPath.toString(), format));
+    }
+
+    return JavaConverters
+        .seqAsJavaListConverter(spec.partitions())
+        .asJava()
+        .stream()
+        .map(partition -> {
+          Map<String, String> values = new HashMap<>();
+          JavaConverters.asJavaIterableConverter(schema).asJava().forEach(field -> {
+            int fieldIndex = schema.fieldIndex(field.name());
+            Object catalystValue = partition.values().get(fieldIndex, field.dataType());
+            Object value = CatalystTypeConverters.convertToScala(catalystValue, field.dataType());
+            values.put(field.name(), value.toString());
+          });
+          return new SparkPartition(values, partition.path().toString(), format);
+        }).collect(Collectors.toList());
+  }
+
+  public static org.apache.spark.sql.catalyst.TableIdentifier toV1TableIdentifier(Identifier identifier) {
+    Preconditions.checkArgument(identifier.namespace().length <= 1,
+        "Cannot load a session catalog namespace with more than 1 part. Given %s", identifier);
+
+    Option<String> namespace =
+        identifier.namespace().length == 1 ? Option.apply(identifier.namespace()[0]) : Option.empty();
+
+    org.apache.spark.sql.catalyst.TableIdentifier tableIdent =

Review comment:
       What about this?
   
   ```
   String[] namespace = identifier.namespace();
   
   Preconditions.checkArgument(namespace.length <= 1, ...);
   
   String table = identifier.name();
   Option<String> database = namespace.length == 1 ? Option.apply(namespace[0]) : Option.empty();
   return org.apache.spark.sql.catalyst.TableIdentifier.apply(table, 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +802,64 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),
+        Option.empty(),
+        fileStatusCache,
+        Option.empty(),
+        Option.empty());
+
+    org.apache.spark.sql.execution.datasources.PartitionSpec spec = fileIndex.partitionSpec();
+    StructType schema = spec.partitionColumns();
+    if (spec.partitions().isEmpty()) {
+      return ImmutableList.of(new SparkPartition(Collections.emptyMap(), rootPath.toString(), format));

Review comment:
       Question: how expected is it to get back one partition in a non-partitioned table?
   
   This is used in `importFileTable`. Will the validation be easier if we return an empty list and don't call this method for unpartitioned tables at all?
   
   ```
   if (table.spec().isUnpartitioned()) {
     SparkPartition partition = new SparkPartition(Collections.emptyMap(), tableLocation.toString(), format);
     importPartitions(table, ImmutableList.of(partition));
   } else {
     List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
     Preconditions.checkArgument(...);
   
     List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);
     Preconditions.checkArgument(...);
   
     importPartitions(table, filteredPartitions);
   }
   ```
   
   Just thinking out loud.

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+import static org.apache.iceberg.spark.SparkTableUtil.MAP_JOINER;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private void importFileTable(Table table, Path tableLocation, String format, Map<String, String> partitionFilter) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
+    boolean sourceUnpartitioned = partitions.size() == 1 && partitions.get(0).getValues().isEmpty();
+    Preconditions.checkArgument(
+        (table.spec().isUnpartitioned() && sourceUnpartitioned) || !table.spec().isUnpartitioned(),
+        "Cannot add partitioned files to an unpartitioned table");
+    Preconditions.checkArgument(!partitions.isEmpty(),
+        "Cannot find any partitions in table %s", partitions);
+    List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);
+    Preconditions.checkArgument(!filteredPartitions.isEmpty(),
+        "Cannot find any partitions which match the given filter. Partition filter is %s",
+        MAP_JOINER.join(partitionFilter));
+    importPartitions(table, filteredPartitions);
+  }
+
+  private void importCatalogTable(Table table, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    String stagingLocation = getMetadataLocation(table);
+    TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent);
+    SparkTableUtil.importSparkTable(spark(), sourceTableIdentifier, table, stagingLocation, partitionFilter);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = getMetadataLocation(table);
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private String getMetadataLocation(Table table) {
+    String defaultValue = table.location() + "/metadata";
+    return table.properties().getOrDefault(TableProperties.WRITE_METADATA_LOCATION, defaultValue);
+  }
+
+  @Override
+  public String description() {
+    return "AddFiles";
+  }
+
+  private static void validatePartitionSpec(Table table, Map<String, String> partitionFilter) {
+    List<PartitionField> partitionFields = table.spec().fields();
+    Set<String> partitionNames = table.spec().fields().stream().map(PartitionField::name).collect(Collectors.toSet());
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partitionFilter.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Not enough partition columns
+      Preconditions.checkArgument(partitionFields.size() >= partitionFilter.size(),
+          "Cannot add data files to target table %s because that table is partitioned, " +
+              "but the number of columns in the provided partition filter (%d) " +
+              "is greater than the number of partitioned columns in table (%d)",
+          table.name(), partitionFilter.size(), partitionFields.size());
+
+      // Check for any non-identity partition columns
+      List<PartitionField> nonIdentityFields =
+          partitionFields.stream().filter(x -> !x.transform().isIdentity()).collect(Collectors.toList());
+      Preconditions.checkArgument(nonIdentityFields.isEmpty(),
+          "Cannot add data files to target table %s because that table is partitioned and contains non-identity" +
+              "partition transforms which will not be compatible. Found non-identity fields %s",
+          table.name(), nonIdentityFields);
+
+      // Check for any filters of non existent columns
+      List<String> unMatchedFilters =
+          partitionFilter.keySet().stream().filter(filterName -> !partitionNames.contains(filterName))
+              .collect(Collectors.toList());
+      Preconditions.checkArgument(unMatchedFilters.isEmpty(),
+          "Cannot add files to target table %s. %s is partitioned but the specified partition filter " +

Review comment:
       nit: I think this comment assumes the provided column is a valid non-partition column. However, we would get this exception even in case of case mismatches. I wonder whether it makes sense to add the list of partition columns to the error message too.

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {

Review comment:
       I guess it is just a personal preference. I usually don't make methods static even if they are pure unless they need to as I don't want to reason about grouping static methods together or potential inheritance problems. 
   
   I have no issue keeping it and `validatePartitionSpec` static. Was just wondering whether there was a particular reason.

##########
File path: .baseline/checkstyle/checkstyle.xml
##########
@@ -92,6 +92,7 @@
                 org.apache.iceberg.IsolationLevel.*,
                 org.apache.iceberg.NullOrder.*,
                 org.apache.iceberg.MetadataTableType.*,
+                org.apache.iceberg.spark.SparkTableUtil.*,

Review comment:
       Is it worth changing this just to reuse `MapJoiner`? I'd say it is easier to simply define one more `MapJoiner` as we do in a couple of other places.

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +802,64 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),

Review comment:
       nit: redundant <Tuple2<String, String>>




-- 
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] rdblue commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I'd prefer not to infer partitioning. I left a comment on #2068. The Hive import works because we know that the partitions match. Inferring too much seems like it would lead to bad data to me.




----------------------------------------------------------------
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] talgos1 edited a comment on pull request #2210: Adds AddFiles Procedure

Posted by GitBox <gi...@apache.org>.
talgos1 edited a comment on pull request #2210:
URL: https://github.com/apache/iceberg/pull/2210#issuecomment-975352083


   Hi, I'm wondering if it's possible to add a specific file (or even an unpartitioned parquet table) to a partitioned iceberg table (specific 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.

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] rdblue commented on pull request #2210: Adds AddFiles Procedure

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


   I'm glad you were able to find a way to get it working! We may want to update this so that we can detect when there's only a single file and handle that case. But in the meantime, it looks like this is a good way to get it working using the API directly.


-- 
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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       One scenario where we may need to infer partitioning is `migrate('path/to/hadoop/table')` (i.e. if we want to support migrating a given location, which I think we do).




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -522,14 +523,32 @@ public static void importSparkTable(
         importUnpartitionedSparkTable(spark, sourceTableIdentWithDB, targetTable);
       } else {
         List<SparkPartition> sourceTablePartitions = getPartitions(spark, sourceTableIdent);
-        importSparkPartitions(spark, sourceTablePartitions, targetTable, spec, stagingDir);
+        List<SparkPartition> filteredPartitions = filterPartitions(sourceTablePartitions, partitionFilter);
+        importSparkPartitions(spark, filteredPartitions, targetTable, spec, stagingDir);
       }
     } catch (AnalysisException e) {
       throw SparkExceptionUtil.toUncheckedException(
           e, "Unable to get partition spec for table: %s", sourceTableIdentWithDB);
     }
   }
 
+  /**
+   * Import files from an existing Spark table to an Iceberg table.
+   *
+   * The import uses the Spark session to get table metadata. It assumes no
+   * operation is going on the original and target table and thus is not
+   * thread-safe.
+   *
+   * @param spark a Spark session
+   * @param sourceTableIdent an identifier of the source Spark table
+   * @param targetTable an Iceberg table where to import the data
+   * @param stagingDir a staging directory to store temporary manifest files
+   */
+  public static void importSparkTable(
+      SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, String stagingDir) {

Review comment:
       nit: I like your formatting for the method above a bit more but I understand this file is not consistent.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -522,14 +523,32 @@ public static void importSparkTable(
         importUnpartitionedSparkTable(spark, sourceTableIdentWithDB, targetTable);
       } else {
         List<SparkPartition> sourceTablePartitions = getPartitions(spark, sourceTableIdent);
-        importSparkPartitions(spark, sourceTablePartitions, targetTable, spec, stagingDir);
+        List<SparkPartition> filteredPartitions = filterPartitions(sourceTablePartitions, partitionFilter);
+        importSparkPartitions(spark, filteredPartitions, targetTable, spec, stagingDir);
       }
     } catch (AnalysisException e) {
       throw SparkExceptionUtil.toUncheckedException(
           e, "Unable to get partition spec for table: %s", sourceTableIdentWithDB);
     }
   }
 
+  /**
+   * Import files from an existing Spark table to an Iceberg table.
+   *
+   * The import uses the Spark session to get table metadata. It assumes no
+   * operation is going on the original and target table and thus is not
+   * thread-safe.
+   *
+   * @param spark a Spark session
+   * @param sourceTableIdent an identifier of the source Spark table
+   * @param targetTable an Iceberg table where to import the data
+   * @param stagingDir a staging directory to store temporary manifest files
+   */
+  public static void importSparkTable(
+      SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, String stagingDir) {

Review comment:
       nit: I like your arg formatting for the method above a bit more but I understand this file is not 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {

Review comment:
       What will happen in this case now?
   
   ```
   // table partitioned by p1, p2, p3
   add_files(
     table => 'db.iceberg_table', -- partitioned Iceberg table
     source_table => '`parquet`.`path/to/table/p1=x`'
     partition_filter => map('p2, ‘y’)
   )
   ```




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {

Review comment:
       We could probably add a method like `parseIdentifier` that would return `Identifier` without trying to resolve the catalog. Then we can check if it is a path identifier and then call the appropriate method to do the import. That way, we will parse the identifier only once and using the Spark parser. both the file format and location. For path-based tables, `namespace` will be file format and `name` 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(

Review comment:
       Shall we use `Preconditions`?




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataUnpartitionedOrc() {
+    createUnpartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedHive() {
+    createUnpartitionedHiveTable();
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '%s')",
+        catalogName, tableName, sourceTableName);
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedExtraCol() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String, foo string) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedMissingCol() {
+    createUnpartitionedFileTable("parquet");
+    String createIceberg =

Review comment:
       :) 




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add data files to target table %s because that table is partitioned, " +
+            "but the number of columns in the provided partition spec (%d) " +
+            "does not match the number of partitioned columns in table (%d)",
+              table.name(), partition.size(), partitionFields.size()));
+      }
+      partitionFields.forEach(field -> {
+        if (!partition.containsKey(field.name())) {
+          throw new IllegalArgumentException(
+            String.format(
+              "Cannot add files to target table %s. %s is partitioned but the specified partition spec " +
+              "refers to a column that is not partitioned: '%s'",
+                table.name(), table.name(), field.name())
+            );
+        }
+      });
+    } else {
+      if (partitionSpecPassed) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add files to target table %s which is not partitioned, but a partition spec was provided",

Review comment:
       Something like `Cannot use a partition filter as the target table is not partitioned`?




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: .baseline/checkstyle/checkstyle.xml
##########
@@ -92,6 +92,7 @@
                 org.apache.iceberg.IsolationLevel.*,
                 org.apache.iceberg.NullOrder.*,
                 org.apache.iceberg.MetadataTableType.*,
+                org.apache.iceberg.spark.SparkTableUtil.*,

Review comment:
       So far Majoiner is only defined in one other place, I was actually wondering whether I should move it to a global util class. I think duplicating here is fine




-- 
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);

Review comment:
       I need to go back through the PR and switch these to unchecked IO's, this is a good suggestion but in the midst of refactoring I forgot to do 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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();

Review comment:
       changed to partition along with the arg name of the procedure, hopefully everything is now 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] rdblue commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I think that Spark allows pointing to a location that is inside a path-based table. So if you pointed to p1=x, then you'd get p2 and p3 columns, but not p1. We can test that, but that's what I'm concerned about. If we need a value for p1, but don't get it from Spark then there is no way to add it besides including it in the partition map.
   
   Maybe using the map as a filter like you suggest would work, but that seems a bit strange to me because it is sometimes supplying partition values and sometimes getting those values from the path.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       @rdblue, I think `path/to/table` would need to point to the root table location all the time, just like in SQL on file in Spark. The `partition` map would tell us which partitions to import.
   
   Let's consider the following call:
   
   ```
   iceberg.system.add_files(
     source_table => `parquet`.`path/to/table`
     table => 'iceberg.db.tbl',
     partition => map('p1', 'v1', 'p2', 'v2')
   )
   ```
   
   I think the procedure would detect the identifier points to a path-based Parquet table and would take the root table location as a basis and add a path for the partition. In other words, the call above would be translated into importing a single Parquet partition.
   
   ```
   path/to/table/p1=v1/p2=v2
   ```
   
   Does that make sense?
   




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");

Review comment:
       I feel like this should be checked outside of `filterPartitions`. I think this utility method should focus only on filtering passed partitions. Also, I think the error message can be a bit more precise. In this case, we did not find partitions for a partitioned table rather than did not find files. 




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -522,14 +523,32 @@ public static void importSparkTable(
         importUnpartitionedSparkTable(spark, sourceTableIdentWithDB, targetTable);
       } else {
         List<SparkPartition> sourceTablePartitions = getPartitions(spark, sourceTableIdent);
-        importSparkPartitions(spark, sourceTablePartitions, targetTable, spec, stagingDir);
+        List<SparkPartition> filteredPartitions = filterPartitions(sourceTablePartitions, partitionFilter);
+        importSparkPartitions(spark, filteredPartitions, targetTable, spec, stagingDir);
       }
     } catch (AnalysisException e) {
       throw SparkExceptionUtil.toUncheckedException(
           e, "Unable to get partition spec for table: %s", sourceTableIdentWithDB);
     }
   }
 
+  /**
+   * Import files from an existing Spark table to an Iceberg table.
+   *
+   * The import uses the Spark session to get table metadata. It assumes no
+   * operation is going on the original and target table and thus is not
+   * thread-safe.
+   *
+   * @param spark a Spark session
+   * @param sourceTableIdent an identifier of the source Spark table
+   * @param targetTable an Iceberg table where to import the data
+   * @param stagingDir a staging directory to store temporary manifest files
+   */
+  public static void importSparkTable(
+      SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, String stagingDir) {

Review comment:
       Let's format the way we dream and not be constrained by past mistakes




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +802,64 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),

Review comment:
       IntelliJ doesn't like this but the compiler seems to have no problem with it, so I'll 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {

Review comment:
       We could probably add a method like `parseIdentifier` that would return `Identifier` without trying to resolve the catalog. Then we can check if it is a path identifier and then call the appropriate method to do the import. That way, we will parse the identifier only once and using the Spark parser. For path-based tables, `namespace` will be file format and `name` 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] pvary commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+        // Forces Name based resolution instead of position based resolution
+        NameMapping mapping = MappingUtil.create(table.schema());
+        String mappingJson = NameMappingParser.toJson(mapping);
+        table.updateProperties()
+            .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+            .commit();
+      }
+
+      if (isFile || !partition.isEmpty()) {
+        // we do a list operation on the driver to import 1 file or 1 partition
+        PartitionSpec spec = table.spec();
+        MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+        String partitionURI;
+
+        if (isFile) {
+          partitionURI = dataPath.toString();
+        } else {
+          partitionURI = dataPath.toString() + toPartitionPath(partition);
+        }
+
+        List<DataFile> files = SparkTableUtil.listPartition(partition, partitionURI, format, spec, conf, metricsConfig);
+        if (files.size() == 0) {
+          throw new IllegalArgumentException(String.format("No files found for add_file command. Looking in URI %s",
+              partitionURI));
+        }
+
+        AppendFiles append = table.newAppend();
+        files.forEach(append::appendFile);
+        append.commit();
+      } else {
+        // Importing multiple partitions
+        List<SparkTableUtil.SparkPartition> partitions = Spark3Util.getPartitions(spark(), dataPath, format);
+        String stagingLocation = table.properties()
+            .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+        SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      Long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private String toPartitionPath(Map<String, String> partition) {

Review comment:
       In Hive it is theoretically possible to assign individual path to a partition.
   See the [spec](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-AddPartitions):
   ```
   ALTER TABLE page_view ADD PARTITION (dt='2008-08-08', country='us') location '/path/to/us/part080808'
                             PARTITION (dt='2008-08-09', country='us') location '/path/to/us/part080809';
   ```
   Maybe it would be better to get the partition location as an input for `importDataToIcebergTable` instead of (or addition to) the table 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] talgos1 edited a comment on pull request #2210: Adds AddFiles Procedure

Posted by GitBox <gi...@apache.org>.
talgos1 edited a comment on pull request #2210:
URL: https://github.com/apache/iceberg/pull/2210#issuecomment-975352083


   Hi, I'm wondering if it's possible to add a specific file (or even an unpartitioned parquet/orc table) to a partitioned iceberg table (specific 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.

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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {

Review comment:
       nit: `partition` -> `partitionFilter`




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet()))
+          .collect(Collectors.toList());
+
+      Preconditions.checkArgument(!filteredPartitions.isEmpty(),
+          "No partitions found in table for add_file command. Looking for partitions with the values %s",
+          partitionFilter.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())

Review comment:
       nit: MapJoiner?
   
   ```
   public static final MapJoiner MAP_JOINER = Joiner.on(",").withKeyValueSeparator("=");
   ```




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I thought we would not need to use Spark's in-memory file index for path-based tables to determine which partitions are inside if we use this approach. The main reason is that path-based tables cannot have custom table locations and we can always construct a partition path if we have a partition map and the root table location. Also, using the in-memory file index is expensive as it will build us a map of all partitions even thought we neew a couple of them and we can construct partition paths ourselves.
   
   @rdblue, I think your concerns can be addressed by extra validation. We will have access to the Iceberg partition spec. We can validate the map of partition values contains all columns.
   
   




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +801,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {

Review comment:
       nit: shall we import `SparkPartition` and use it directly? Like `List<SparkPartition>`?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {

Review comment:
       Does it mean we will allow pointing to the root table location like this?
   
   ```
   add_files(
     table => 'db.iceberg_table', -- partitioned Iceberg table
     source_table => '`parquet`.`path/to/table`'
   )
   ```
   
   I am still not sure it is a good idea. I think `add_files` should be more about incremental migration/import where you have a single or a couple of partitions to import. Supporting bulk imports is useful but inferring which partitions are inside seems too much for `add_files`, in my view. Other use cases should be covered by snapshot/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] RussellSpitzer commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +802,64 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),
+        Option.empty(),
+        fileStatusCache,
+        Option.empty(),
+        Option.empty());
+
+    org.apache.spark.sql.execution.datasources.PartitionSpec spec = fileIndex.partitionSpec();
+    StructType schema = spec.partitionColumns();
+    if (spec.partitions().isEmpty()) {
+      return ImmutableList.of(new SparkPartition(Collections.emptyMap(), rootPath.toString(), format));

Review comment:
       hmm this does mean we don't check whether we are importing from a partitioned source .... let me think about this a bit more




-- 
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+        // Forces Name based resolution instead of position based resolution
+        NameMapping mapping = MappingUtil.create(table.schema());
+        String mappingJson = NameMappingParser.toJson(mapping);
+        table.updateProperties()
+            .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+            .commit();
+      }
+
+      if (isFile || !partition.isEmpty()) {
+        // we do a list operation on the driver to import 1 file or 1 partition
+        PartitionSpec spec = table.spec();
+        MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+        String partitionURI;
+
+        if (isFile) {
+          partitionURI = dataPath.toString();
+        } else {
+          partitionURI = dataPath.toString() + toPartitionPath(partition);
+        }
+
+        List<DataFile> files = SparkTableUtil.listPartition(partition, partitionURI, format, spec, conf, metricsConfig);
+        if (files.size() == 0) {
+          throw new IllegalArgumentException(String.format("No files found for add_file command. Looking in URI %s",
+              partitionURI));
+        }
+
+        AppendFiles append = table.newAppend();
+        files.forEach(append::appendFile);
+        append.commit();
+      } else {
+        // Importing multiple partitions
+        List<SparkTableUtil.SparkPartition> partitions = Spark3Util.getPartitions(spark(), dataPath, format);
+        String stagingLocation = table.properties()
+            .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+        SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      Long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));

Review comment:
       I think `parseLong` returns a primitive, no need to box it. I'd make it:
   
   ```
       Snapshot snapshot = table.currentSnapshot();
       long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
       LOG.info("Successfully loaded Iceberg metadata for {} files", numAddedFiles);
       return numAddedFiles;
   ```
   
   And make `importDataToIcebergTable` to return a primitive.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: .baseline/checkstyle/checkstyle.xml
##########
@@ -92,6 +92,7 @@
                 org.apache.iceberg.IsolationLevel.*,
                 org.apache.iceberg.NullOrder.*,
                 org.apache.iceberg.MetadataTableType.*,
+                org.apache.iceberg.spark.SparkTableUtil.*,

Review comment:
       I think duplicating it for now is OK. That way, we don't have to maintain one more utility class.




-- 
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I thought we would not need to use Spark's in-memory file index for path-based tables to determine which partitions are inside if we use this approach. The main reason is that path-based tables cannot have custom table locations and we can always construct a partition path if we have a partition map and the root table location. Also, using the in-memory file index is expensive as it will build us a map of all partitions even though we need a couple of them and we can construct partition paths ourselves.
   
   @rdblue, I think your concerns can be addressed by extra validation. We will have access to the Iceberg partition spec. We can validate the map of partition values contains all columns.
   
   




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add data files to target table %s because that table is partitioned, " +
+            "but the number of columns in the provided partition spec (%d) " +
+            "does not match the number of partitioned columns in table (%d)",
+              table.name(), partition.size(), partitionFields.size()));
+      }
+      partitionFields.forEach(field -> {
+        if (!partition.containsKey(field.name())) {
+          throw new IllegalArgumentException(
+            String.format(
+              "Cannot add files to target table %s. %s is partitioned but the specified partition spec " +
+              "refers to a column that is not partitioned: '%s'",
+                table.name(), table.name(), field.name())
+            );
+        }
+      });

Review comment:
       I think we need to add validation that all columns from Iceberg partition spec are present in the partition map provided by the user to avoid cases mentioned by @rdblue.
   
   I think the following should be illegal.
   
   ```
   add_files(
     table => 'db.iceberg_table', -- partitioned by part_col_1 and part_col_2
     source_table => '`parquet`.`path/to/table/part_col_1=x`'
     partition => map('part_col_2', 'y')
   )
   ```




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       For HMS tables, we will need to go to the metastore and find the location of `p1=v1, p2=v2` 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       @rdsr, I think the procedure can load the Iceberg table and use its spec to determine the correct order of partition columns. That seems like the most reliable way and does not require parsing partition strings.
   
   @pvary brought up a good point about custom partition locations in [this](https://github.com/apache/iceberg/pull/2210#discussion_r578317436) comment. 
   
   The API I proposed in the comment above won't work for that. @RussellSpitzer and I had a quick chat and it seems we can support the following:
   
   ```
   iceberg.system.add_files(
     source_table => 'source_table' -- required
     table => 'db.tbl', -- required
     partition => map('p1', '*', 'p2', 'v2') -- optional
   )
   ```
   
   The procedure can be called like this:
   
   ```
   iceberg.system.add_files(
     source_table => `parquet`.`path/to/table`
     table => 'iceberg.db.tbl',
     partition => map('p1', '*', 'p2', 'v2')
   )
   ```
   
   ```
   iceberg.system.add_files(
     source_table => `db.hive_tbl`
     table => 'db.iceberg_tbl',
     partition => map('p1', '*', 'p2', 'v2')
   )
   ```
   
   This way, we should support custom partition locations as we will go to HMS for 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] RussellSpitzer commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+        // Forces Name based resolution instead of position based resolution
+        NameMapping mapping = MappingUtil.create(table.schema());
+        String mappingJson = NameMappingParser.toJson(mapping);
+        table.updateProperties()
+            .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+            .commit();
+      }
+
+      if (isFile || !partition.isEmpty()) {
+        // we do a list operation on the driver to import 1 file or 1 partition
+        PartitionSpec spec = table.spec();
+        MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+        String partitionURI;
+
+        if (isFile) {
+          partitionURI = dataPath.toString();
+        } else {
+          partitionURI = dataPath.toString() + toPartitionPath(partition);
+        }
+
+        List<DataFile> files = SparkTableUtil.listPartition(partition, partitionURI, format, spec, conf, metricsConfig);
+        if (files.size() == 0) {
+          throw new IllegalArgumentException(String.format("No files found for add_file command. Looking in URI %s",
+              partitionURI));
+        }
+
+        AppendFiles append = table.newAppend();
+        files.forEach(append::appendFile);
+        append.commit();
+      } else {
+        // Importing multiple partitions
+        List<SparkTableUtil.SparkPartition> partitions = Spark3Util.getPartitions(spark(), dataPath, format);
+        String stagingLocation = table.properties()
+            .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+        SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      Long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private String toPartitionPath(Map<String, String> partition) {
+    return partition.entrySet().stream()
+        .map(entry -> entry.getKey() + "=" + entry.getValue())
+        .collect(Collectors.joining("/", "/", ""));
+  }
+
+  private void validatePartitionSpec(Table table, Path path, FileSystem fs, Map<String, String> partition) {
+
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add data files to target table %s because that table is partitioned, " +
+            "but the number of columns in the provided partition spec (%d) " +
+            "does not match the number of partitioned columns in table (%d)",
+              table.name(), partition.size(), partitionFields.size()));
+      }
+      partitionFields.forEach(field -> {
+        if (!partition.containsKey(field.name())) {

Review comment:
       Yeah this could be a problem if the entries that Spark returns are different than those we expect based on column names, i'll need to add some hive tests for that




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       Alright, I think we all agree that `add_files` procedure should be limited to identity partitioning and, ideally, it should not infer partitions (i.e. we should not try to build a list of partitions in a given location). The primary use case for this procedure is partial migration and people can use SNAPSHOT or MIGRATE if they need to migrate all partitions at the same time. Instead of inferring a list of partitions ourselves, we better ask the user which partitions to import.
   
   That means we don't need to use `InMemoryFileIndex` right now. We will come back to it while adding support for migrating path-based tables. We should also take other query engines into account.
   
   The open question right now is whether we should support adding multiple partitions in the same call? That seems beneficial but how do we do this and what would it mean for the procedure API? In earlier discussions, @rdblue mentioned supporting something like `part_col1=*/part_col2=x` would be great. While it would be easier for us to support only a single partition, I'd vote for a better user experience.
   
   We could represent partition path as a string with wildcards but I'd suggest using a map.
   
   ```
   iceberg.system.add_files(
     table => 'db.tbl', -- required
     path => '/root/table/location', -- required
     format => 'parquet', -- required
     partition => map('p1', '*', 'p2', 'v2') -- optional but required for partitioned tables
   )
   ```
   
   We may add support for `*` later but using a map instead of a string does not require us parsing the path to get individual 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards

Review comment:
       I think we need to support partial maps instead of wildcards.




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   Implementation of #2068
   


----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   Thanks everybody! I think the review period really helped make this code better, I hope it's of use to folks in the future.


-- 
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I am going to resolve this thread to simplify the review. Feel free to reopen as needed.




----------------------------------------------------------------
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] rdsr commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       Sounds good @aokolnychyi !




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +802,64 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),
+        Option.empty(),
+        fileStatusCache,
+        Option.empty(),
+        Option.empty());
+
+    org.apache.spark.sql.execution.datasources.PartitionSpec spec = fileIndex.partitionSpec();
+    StructType schema = spec.partitionColumns();
+    if (spec.partitions().isEmpty()) {
+      return ImmutableList.of(new SparkPartition(Collections.emptyMap(), rootPath.toString(), format));

Review comment:
       The issue here is that getPartitions throws an exception here
   ```
         Seq<CatalogTablePartition> partitions = catalog.listPartitions(tableIdent, Option.empty());
   ```
   If there are no partitions, but we could change the code so that it returns an empy seq if the table in unpartitioned




-- 
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet()))

Review comment:
       nit: I'd put `partitions.stream()` on one line and `filter` on a new line.




----------------------------------------------------------------
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] rdblue commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       For the path-based option, would that load the path-based table using Spark and then get the files from it? If so, I like delegating the partition inference like that. I guess my question is: what if `path/to/table` isn't a partition directory and is further up the file tree?




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   @pvary we also have the migrate and snapshot actions


----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {

Review comment:
       If we do so, we probably don't need `Format` enum anymore.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add data files to target table %s because that table is partitioned, " +
+            "but the number of columns in the provided partition spec (%d) " +
+            "does not match the number of partitioned columns in table (%d)",
+              table.name(), partition.size(), partitionFields.size()));
+      }
+      partitionFields.forEach(field -> {
+        if (!partition.containsKey(field.name())) {
+          throw new IllegalArgumentException(
+            String.format(
+              "Cannot add files to target table %s. %s is partitioned but the specified partition spec " +
+              "refers to a column that is not partitioned: '%s'",
+                table.name(), table.name(), field.name())
+            );
+        }
+      });

Review comment:
       Per discussion, this should act as a filter and can be optional.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       @rdsr, I think the procedure can load the Iceberg table and use its spec to determine the correct order of partition columns. That seems like the most reliable way and does not require parsing partition strings.
   
   @pvary brought up a good point about custom partition locations in [this](https://github.com/apache/iceberg/pull/2210#discussion_r578317436) comment. 
   
   The API I proposed in the comment above won't work for that. @RussellSpitzer and I had a quick chat and it seems we can support the following:
   
   ```
   iceberg.system.add_files(
     source_table => 'source_table' -- required
     table => 'db.tbl', -- required
     partition => map('p1', '*', 'p2', 'v2') -- optional
   )
   ```
   
   The procedure can be called like this:
   
   ```
   // path-based
   iceberg.system.add_files(
     source_table => `parquet`.`path/to/table`
     table => 'iceberg.db.tbl',
     partition => map('p1', '*', 'p2', 'v2')
   )
   ```
   
   ```
   // metastore-based
   iceberg.system.add_files(
     source_table => `db.hive_tbl`
     table => 'db.iceberg_tbl',
     partition => map('p1', '*', 'p2', 'v2')
   )
   ```
   
   This way, we should support custom partition locations as we will go to HMS for 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       Thoughts, @rdsr @RussellSpitzer @rdblue?




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),

Review comment:
       Sometimes I'm just lucky, i'll add in the order by to make sure




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   I'm having issues with writing to ORC tables from spark in our tests, probably a version conflict?
   
   ```
   Caused by: java.lang.NoSuchMethodError: org.apache.orc.TypeDescription.createRowBatch()Lorg/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch;
   ```


----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       @rdsr, I think the procedure can load the Iceberg table and use its spec to determine the correct order of partition columns. That seems like the most reliable way and does not require parsing partition strings.
   
   @pvary brought up a good point about custom partition locations in [this](https://github.com/apache/iceberg/pull/2210#discussion_r578317436) comment. 
   
   The API I proposed in the comment above won't work for that. @RussellSpitzer and I had a quick chat and it seems we can support the following:
   
   ```
   iceberg.system.add_files(
     source_table => 'source_table' -- required
     table => 'db.tbl', -- required
     partition => map('p1', '*', 'p2', 'v2') -- optional
   )
   ```
   
   The procedure can be called like this:
   
   ```
   -- path-based
   iceberg.system.add_files(
     source_table => `parquet`.`path/to/table`
     table => 'iceberg.db.tbl',
     partition => map('p1', '*', 'p2', 'v2')
   )
   ```
   
   ```
   -- metastore-based
   iceberg.system.add_files(
     source_table => `db.hive_tbl`
     table => 'db.iceberg_tbl',
     partition => map('p1', '*', 'p2', 'v2')
   )
   ```
   
   This way, we should support custom partition locations as we will go to HMS for 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: .baseline/checkstyle/checkstyle.xml
##########
@@ -92,6 +92,7 @@
                 org.apache.iceberg.IsolationLevel.*,
                 org.apache.iceberg.NullOrder.*,
                 org.apache.iceberg.MetadataTableType.*,
+                org.apache.iceberg.spark.SparkTableUtil.*,

Review comment:
       Looks like this is no longer needed?

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final Joiner.MapJoiner MAP_JOINER = Joiner.on(",").withKeyValueSeparator("=");
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private void importFileTable(Table table, Path tableLocation, String format, Map<String, String> partitionFilter) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
+
+    if (table.spec().isUnpartitioned()) {
+      Preconditions.checkArgument(partitions.isEmpty(), "Cannot add partitioned files to an unpartitioned table");
+      // Build a Global Partition for the source
+      SparkPartition partition = new SparkPartition(Collections.emptyMap(), tableLocation.toString(), format);
+      Preconditions.checkArgument(partitionFilter.isEmpty(), "Cannot use a partition filter when importing" +
+          "to an unpartitioned table");
+      importPartitions(table, ImmutableList.of(partition));
+    } else {
+      Preconditions.checkArgument(!partitions.isEmpty(),
+          "Cannot find any partitions in table %s", partitions);
+      List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);
+      Preconditions.checkArgument(!filteredPartitions.isEmpty(),
+          "Cannot find any partitions which match the given filter. Partition filter is %s",
+          MAP_JOINER.join(partitionFilter));
+      importPartitions(table, filteredPartitions);
+    }
+  }
+
+  private void importCatalogTable(Table table, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    String stagingLocation = getMetadataLocation(table);
+    TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent);
+    SparkTableUtil.importSparkTable(spark(), sourceTableIdentifier, table, stagingLocation, partitionFilter);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = getMetadataLocation(table);
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private String getMetadataLocation(Table table) {
+    String defaultValue = table.location() + "/metadata";
+    return table.properties().getOrDefault(TableProperties.WRITE_METADATA_LOCATION, defaultValue);
+  }
+
+  @Override
+  public String description() {
+    return "AddFiles";
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partitionFilter) {
+    List<PartitionField> partitionFields = table.spec().fields();
+    Set<String> partitionNames = table.spec().fields().stream().map(PartitionField::name).collect(Collectors.toSet());
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partitionFilter.isEmpty();
+
+    // Check for any non-identity partition columns
+    List<PartitionField> nonIdentityFields =
+        partitionFields.stream()

Review comment:
       nit: I think it would be more natural to have `partitionFields.stream()` on the same line and then use 4 spaces for filter and collect lines.

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final Joiner.MapJoiner MAP_JOINER = Joiner.on(",").withKeyValueSeparator("=");
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private void importFileTable(Table table, Path tableLocation, String format, Map<String, String> partitionFilter) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
+
+    if (table.spec().isUnpartitioned()) {
+      Preconditions.checkArgument(partitions.isEmpty(), "Cannot add partitioned files to an unpartitioned table");
+      // Build a Global Partition for the source
+      SparkPartition partition = new SparkPartition(Collections.emptyMap(), tableLocation.toString(), format);
+      Preconditions.checkArgument(partitionFilter.isEmpty(), "Cannot use a partition filter when importing" +
+          "to an unpartitioned table");
+      importPartitions(table, ImmutableList.of(partition));
+    } else {
+      Preconditions.checkArgument(!partitions.isEmpty(),
+          "Cannot find any partitions in table %s", partitions);
+      List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);
+      Preconditions.checkArgument(!filteredPartitions.isEmpty(),
+          "Cannot find any partitions which match the given filter. Partition filter is %s",
+          MAP_JOINER.join(partitionFilter));
+      importPartitions(table, filteredPartitions);
+    }
+  }
+
+  private void importCatalogTable(Table table, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    String stagingLocation = getMetadataLocation(table);
+    TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent);
+    SparkTableUtil.importSparkTable(spark(), sourceTableIdentifier, table, stagingLocation, partitionFilter);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = getMetadataLocation(table);
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private String getMetadataLocation(Table table) {
+    String defaultValue = table.location() + "/metadata";
+    return table.properties().getOrDefault(TableProperties.WRITE_METADATA_LOCATION, defaultValue);
+  }
+
+  @Override
+  public String description() {
+    return "AddFiles";
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partitionFilter) {
+    List<PartitionField> partitionFields = table.spec().fields();
+    Set<String> partitionNames = table.spec().fields().stream().map(PartitionField::name).collect(Collectors.toSet());
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partitionFilter.isEmpty();
+
+    // Check for any non-identity partition columns
+    List<PartitionField> nonIdentityFields =
+        partitionFields.stream()
+            .filter(x -> !x.transform().isIdentity())
+            .collect(Collectors.toList());
+    Preconditions.checkArgument(nonIdentityFields.isEmpty(),
+        "Cannot add data files to target table %s because that table is partitioned and contains non-identity" +
+            "partition transforms which will not be compatible. Found non-identity fields %s",
+        table.name(), nonIdentityFields);
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Check to see there are sufficient partition columns to satisfy the filter
+      Preconditions.checkArgument(partitionFields.size() >= partitionFilter.size(),
+          "Cannot add data files to target table %s because that table is partitioned, " +
+              "but the number of columns in the provided partition filter (%d) " +
+              "is greater than the number of partitioned columns in table (%d)",
+          table.name(), partitionFilter.size(), partitionFields.size());
+
+      // Check for any filters of non existent columns
+      List<String> unMatchedFilters =
+          partitionFilter.keySet().stream()

Review comment:
       nit: same as the one above

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final Joiner.MapJoiner MAP_JOINER = Joiner.on(",").withKeyValueSeparator("=");
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private void importFileTable(Table table, Path tableLocation, String format, Map<String, String> partitionFilter) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
+
+    if (table.spec().isUnpartitioned()) {
+      Preconditions.checkArgument(partitions.isEmpty(), "Cannot add partitioned files to an unpartitioned table");
+      // Build a Global Partition for the source
+      SparkPartition partition = new SparkPartition(Collections.emptyMap(), tableLocation.toString(), format);
+      Preconditions.checkArgument(partitionFilter.isEmpty(), "Cannot use a partition filter when importing" +

Review comment:
       nit: Shall we group Preconditions together before constructing a global partition?

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.
+ */
+

Review comment:
       nit: extra line




-- 
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {

Review comment:
       nope




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+        // Forces Name based resolution instead of position based resolution
+        NameMapping mapping = MappingUtil.create(table.schema());
+        String mappingJson = NameMappingParser.toJson(mapping);
+        table.updateProperties()
+            .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+            .commit();
+      }
+
+      if (isFile || !partition.isEmpty()) {
+        // we do a list operation on the driver to import 1 file or 1 partition
+        PartitionSpec spec = table.spec();
+        MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+        String partitionURI;
+
+        if (isFile) {
+          partitionURI = dataPath.toString();
+        } else {
+          partitionURI = dataPath.toString() + toPartitionPath(partition);
+        }
+
+        List<DataFile> files = SparkTableUtil.listPartition(partition, partitionURI, format, spec, conf, metricsConfig);
+        if (files.size() == 0) {
+          throw new IllegalArgumentException(String.format("No files found for add_file command. Looking in URI %s",
+              partitionURI));
+        }
+
+        AppendFiles append = table.newAppend();
+        files.forEach(append::appendFile);
+        append.commit();
+      } else {
+        // Importing multiple partitions
+        List<SparkTableUtil.SparkPartition> partitions = Spark3Util.getPartitions(spark(), dataPath, format);
+        String stagingLocation = table.properties()
+            .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+        SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      Long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private String toPartitionPath(Map<String, String> partition) {

Review comment:
       Fixed this with our alternate "hive" based import for non path tables, tests too be added




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+        // Forces Name based resolution instead of position based resolution
+        NameMapping mapping = MappingUtil.create(table.schema());
+        String mappingJson = NameMappingParser.toJson(mapping);
+        table.updateProperties()
+            .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+            .commit();
+      }
+
+      if (isFile || !partition.isEmpty()) {
+        // we do a list operation on the driver to import 1 file or 1 partition
+        PartitionSpec spec = table.spec();
+        MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+        String partitionURI;
+
+        if (isFile) {
+          partitionURI = dataPath.toString();
+        } else {
+          partitionURI = dataPath.toString() + toPartitionPath(partition);
+        }
+
+        List<DataFile> files = SparkTableUtil.listPartition(partition, partitionURI, format, spec, conf, metricsConfig);
+        if (files.size() == 0) {
+          throw new IllegalArgumentException(String.format("No files found for add_file command. Looking in URI %s",
+              partitionURI));
+        }
+
+        AppendFiles append = table.newAppend();
+        files.forEach(append::appendFile);
+        append.commit();
+      } else {
+        // Importing multiple partitions
+        List<SparkTableUtil.SparkPartition> partitions = Spark3Util.getPartitions(spark(), dataPath, format);
+        String stagingLocation = table.properties()
+            .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+        SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      Long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private String toPartitionPath(Map<String, String> partition) {

Review comment:
       One way to handle this is described in [this](https://github.com/apache/iceberg/pull/2210#discussion_r580599502) comment.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {

Review comment:
       And then `importToIceberg` can also accept just `Identifier sourceIdent`.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);

Review comment:
       nit: `ensureNameMappingPresent` for consistency with other places?




----------------------------------------------------------------
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] talgos1 edited a comment on pull request #2210: Adds AddFiles Procedure

Posted by GitBox <gi...@apache.org>.
talgos1 edited a comment on pull request #2210:
URL: https://github.com/apache/iceberg/pull/2210#issuecomment-975352083


   Hi, I'm wondering if it's possible to add a specific file (or an unpartitioned parquet table) to a partitioned iceberg table (specific 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.

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] aokolnychyi merged pull request #2210: Adds AddFiles Procedure

Posted by GitBox <gi...@apache.org>.
aokolnychyi merged pull request #2210:
URL: https://github.com/apache/iceberg/pull/2210


   


-- 
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long filesAdded = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&

Review comment:
       Added, I wasn't sure how ofter we have this since I feel like we have everyone converting from Avro to Parquet




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       We can't use the version in SparkUtils because unfortunately the Scala api for this method has added additional default args in Spark3 vs Spark2. In Scala this wouldn't be a problem, but in java we need to explicitly call the function with all args.




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   Also, I am not sure about the import of individual files. I think it will set a bad example and people will misuse it. Importing file by file will generate a snapshot on each commit, putting too much pressure on the metadata. I'd probably drop that case.


----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   I did one thorough pass. I think there is a temp consensus on inferring partitions in this PR and we can use Spark's in-memory file index for now until we have our own solution.
   
   I think we need to rename `partition` into `partition_filter` and don't require all columns. Also, it would be great to test special chars and spaces in partition names. 


----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {

Review comment:
       I think we should validate the table spec contains only identity partitioning. No complex partition transforms.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +802,64 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),
+        Option.empty(),
+        fileStatusCache,
+        Option.empty(),
+        Option.empty());
+
+    org.apache.spark.sql.execution.datasources.PartitionSpec spec = fileIndex.partitionSpec();
+    StructType schema = spec.partitionColumns();
+    if (spec.partitions().isEmpty()) {
+      return ImmutableList.of(new SparkPartition(Collections.emptyMap(), rootPath.toString(), format));

Review comment:
       Well, if we want to validate that explicitly, we can do something like this.
   
   ```
   if (table.spec().isUnpartitioned()) {
     List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
     Preconditions.checkArgument(partitions.size() == 0, ...);
     
     SparkPartition partition = new SparkPartition(Collections.emptyMap(), tableLocation.toString(), format);
     importPartitions(table, ImmutableList.of(partition));
   } else {
     List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
     Preconditions.checkArgument(partitions.size() > 0, ...);
   
     List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);
     Preconditions.checkArgument(...);
   
     importPartitions(table, filteredPartitions);
   }
   ```
   
   I am just thinking out loud here. The validation logic in `importFileTable` was a bit hard to follow.




-- 
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet()))
+          .collect(Collectors.toList());
+
+      Preconditions.checkArgument(!filteredPartitions.isEmpty(),

Review comment:
       Well, this one is a bit different. So we could probably validate it here. Do we need to actually fail this with an exception? Or should this be noop if no partitions match our 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);

Review comment:
       nit: `Long filesAdded` -> `long addedFilesCount`




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);

Review comment:
       Do we have a good use case for importing a single file?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +802,64 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),
+        Option.empty(),
+        fileStatusCache,
+        Option.empty(),
+        Option.empty());
+
+    org.apache.spark.sql.execution.datasources.PartitionSpec spec = fileIndex.partitionSpec();
+    StructType schema = spec.partitionColumns();
+    if (spec.partitions().isEmpty()) {
+      return ImmutableList.of(new SparkPartition(Collections.emptyMap(), rootPath.toString(), format));

Review comment:
       Up to you, @RussellSpitzer.




-- 
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       Thanks everybody for the discussion!




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +802,64 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),
+        Option.empty(),
+        fileStatusCache,
+        Option.empty(),
+        Option.empty());
+
+    org.apache.spark.sql.execution.datasources.PartitionSpec spec = fileIndex.partitionSpec();
+    StructType schema = spec.partitionColumns();
+    if (spec.partitions().isEmpty()) {
+      return ImmutableList.of(new SparkPartition(Collections.emptyMap(), rootPath.toString(), format));

Review comment:
       sure we can do that, I think that's also reasonable
   




-- 
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 #2210: Adds AddFiles Procedure

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


   We have some strange classpath issues with ORC in our test classpath, to make things worse, when I attempt to debug the issue the debug console does not have the same issue and can call the missing method without issue.


----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   @RussellSpitzer hello,How to associate an existing ICEBERG table with a hive table。What is the syntax of sparkSQL? 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {

Review comment:
       I am +1 on this too. I think it will be safer.




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {

Review comment:
       I like static functions 🤷‍♀️ , is the a reason to make it not static if it doesn't use class state?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();

Review comment:
       We could consider offering `toIdentifier` that accepts `CatalogPlugin` as the third arg. Then this will be a bit shorter but the current approach is fine with me 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);

Review comment:
       I am not sure that's going to work all the time. Specifically, this won't handle backtick quotes. It seems we need to convert `sourceTable` to `Identifier` and then check if the namespace is a file format and the name is a location. We could have a dedicated method that would check if a given `Identifier` is a path-based identifier.




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   I am looking for a way to migrate a native Hive table to a Hive table backed by an Iceberg table, and during my search I found this PR (along with the #2068 - Procedure for adding files to a Table), but I have also found the same thing for Flink: #2217 (Flink : migrate hive table to iceberg table).
   
   Maybe we should factor-out the common parts to a place which is accessible for both Spark and Flink (and for Hive as well). Like a common java API or something like it. Would this be possible?


----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +803,66 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),
+        Option.empty(),
+        fileStatusCache,
+        Option.empty(),
+        Option.empty());
+
+    org.apache.spark.sql.execution.datasources.PartitionSpec spec = fileIndex.partitionSpec();
+    StructType schema = spec.partitionColumns();
+    if (spec.partitions().isEmpty()) {
+      return ImmutableList.of(new SparkPartition(Collections.emptyMap(), rootPath.toString(), format));
+    }
+
+    return JavaConverters
+        .seqAsJavaListConverter(spec.partitions())
+        .asJava()
+        .stream()
+        .map(partition -> {
+          Map<String, String> values = new HashMap<>();
+          JavaConverters.asJavaIterableConverter(schema).asJava().forEach(field -> {
+            int fieldIndex = schema.fieldIndex(field.name());
+            Object catalystValue = partition.values().get(fieldIndex, field.dataType());
+            Object value = CatalystTypeConverters.convertToScala(catalystValue, field.dataType());
+            values.put(field.name(), value.toString());
+          });
+          return new SparkPartition(values, partition.path().toString(), format);
+        }).collect(Collectors.toList());
+  }
+
+  public static org.apache.spark.sql.catalyst.TableIdentifier toV1TableIdentifier(Identifier identifier) {
+    Preconditions.checkArgument(identifier.namespace().length <= 1,
+        "Cannot load a session catalog namespace with more than 1 part. Given %s", identifier);

Review comment:
       nit: I feel like the error message here can be improved as we don't have any loading in this method. 
   
   ```
   Cannot convert %s to a v1 identifier; namespace contains more than 1 part
   ```




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {

Review comment:
       fair enough, I considered doing this but I wasn't sure if we were ready to go all out with recognizing path based tables this way.




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataUnpartitionedOrc() {
+    createUnpartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedHive() {
+    createUnpartitionedHiveTable();
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '%s')",
+        catalogName, tableName, sourceTableName);
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedExtraCol() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String, foo string) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedMissingCol() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataPartitionedMissingCol() {
+    createPartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(8L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataPartitioned() {
+    createPartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(8L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataPartitionedOrc() {
+    createPartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(8L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataPartitionedHive() {
+    createPartitionedHiveTable();
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '%s')",
+        catalogName, tableName, sourceTableName);
+
+    Assert.assertEquals(8L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addPartitionToPartitioned() {
+    createPartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addFilteredPartitionsToPartitioned() {
+    createCompositePartitionedTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg " +
+            "PARTITIONED BY (id, dept)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addPartitionToPartitionedHive() {
+    createPartitionedHiveTable();
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '%s', map('id', 1))",
+        catalogName, tableName, sourceTableName);
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void invalidDataImport() {
+    createPartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    AssertHelpers.assertThrows("Should forbid adding of partitioned data to unpartitioned table",
+        IllegalArgumentException.class,
+        "Cannot use partition filter with an unpartitioned table",
+        () -> scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))",
+            catalogName, tableName, fileTableDir.getAbsolutePath())
+    );
+  }
+
+  @Test
+  public void invalidDataImportPartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    File fileToAdd = fileTableDir.listFiles((dir, name) -> name.endsWith("parquet"))[0];
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    AssertHelpers.assertThrows("Should forbid adding with a mismatching partition spec",
+        IllegalArgumentException.class,
+        "is greater than the number of partitioned columns",
+        () -> scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`', map('x', '1', 'y', '2'))",
+            catalogName, tableName, fileTableDir.getAbsolutePath()));
+
+    AssertHelpers.assertThrows("Should forbid adding with partition spec with incorrect columns",
+        IllegalArgumentException.class,
+        "specified partition filter refers to columns that are not partitioned",
+        () -> scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', '2'))",
+            catalogName, tableName, fileTableDir.getAbsolutePath()));
+
+  }
+
+  private static final StructField[] unpartitionedStruct = {

Review comment:
       I've already smashed all these together, so ignore the replication 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] RussellSpitzer commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet()))
+          .collect(Collectors.toList());
+
+      Preconditions.checkArgument(!filteredPartitions.isEmpty(),

Review comment:
       I can move them both to the import, I do think it should be an exception because I dislike silent noops




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -706,6 +718,7 @@ public static CatalogAndIdentifier catalogAndIdentifier(SparkSession spark, Stri
     ParserInterface parser = spark.sessionState().sqlParser();
     Seq<String> multiPartIdentifier = parser.parseMultipartIdentifier(name);
     List<String> javaMultiPartIdentifier = JavaConverters.seqAsJavaList(multiPartIdentifier);
+

Review comment:
       Seems unnecessary?




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I've switched this around and now we always determine partitions via filtering, we never explicitly add the path but do rely on the Spark methodology to determine which partitions exist. This means for Hive tables we ask hive, and for file based tables we use the in memory file index.
   
   if you specify a sub directory as your location i'll have to test what our current code does, i'm not sure what the outcome would be




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long filesAdded = importToIceberg(tableIdent, sourceIdent, partitionFilter);

Review comment:
       nit: addedFilesCount




----------------------------------------------------------------
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] rdsr commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+        // Forces Name based resolution instead of position based resolution
+        NameMapping mapping = MappingUtil.create(table.schema());
+        String mappingJson = NameMappingParser.toJson(mapping);
+        table.updateProperties()
+            .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+            .commit();
+      }
+
+      if (isFile || !partition.isEmpty()) {
+        // we do a list operation on the driver to import 1 file or 1 partition
+        PartitionSpec spec = table.spec();
+        MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+        String partitionURI;
+
+        if (isFile) {
+          partitionURI = dataPath.toString();
+        } else {
+          partitionURI = dataPath.toString() + toPartitionPath(partition);
+        }
+
+        List<DataFile> files = SparkTableUtil.listPartition(partition, partitionURI, format, spec, conf, metricsConfig);
+        if (files.size() == 0) {
+          throw new IllegalArgumentException(String.format("No files found for add_file command. Looking in URI %s",
+              partitionURI));
+        }
+
+        AppendFiles append = table.newAppend();
+        files.forEach(append::appendFile);
+        append.commit();
+      } else {
+        // Importing multiple partitions
+        List<SparkTableUtil.SparkPartition> partitions = Spark3Util.getPartitions(spark(), dataPath, format);
+        String stagingLocation = table.properties()
+            .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+        SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      Long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private String toPartitionPath(Map<String, String> partition) {
+    return partition.entrySet().stream()
+        .map(entry -> entry.getKey() + "=" + entry.getValue())
+        .collect(Collectors.joining("/", "/", ""));
+  }
+
+  private void validatePartitionSpec(Table table, Path path, FileSystem fs, Map<String, String> partition) {
+
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add data files to target table %s because that table is partitioned, " +
+            "but the number of columns in the provided partition spec (%d) " +
+            "does not match the number of partitioned columns in table (%d)",
+              table.name(), partition.size(), partitionFields.size()));
+      }
+      partitionFields.forEach(field -> {
+        if (!partition.containsKey(field.name())) {
+          throw new IllegalArgumentException(
+            String.format(
+              "Cannot add files to target table %s. %s is partitioned but the specified partition spec " +
+              "refers to a column that is not partitioned: '%s'",
+                table.name(), table.name(), field.name())
+            );
+        }
+      });
+    } else if (tablePartitioned && !partitionSpecPassed) {
+      try {
+        if (!fs.getFileStatus(path).isDirectory()) {
+          throw new IllegalArgumentException(
+            String.format(
+              "Cannot add files to target table %s which is partitioned, but no partition spec was provided " +
+              "and path '%s' is not a directory",
+                table.name(), path));
+        }
+      } catch (IOException e) {
+        throw new RuntimeException("Could not access path during add_files", e);
+      }
+    } else {
+      if (partitionSpecPassed) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add files to target table %s which is not partitioned, but a partition spec was provided",
+              table.name()));
+      }
+    }
+  }
+

Review comment:
       nit: empty line..

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);

Review comment:
       Maybe `UncheckedIOException` makes more sense. Or maybe just replace with Util.getFs ?

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+        // Forces Name based resolution instead of position based resolution
+        NameMapping mapping = MappingUtil.create(table.schema());
+        String mappingJson = NameMappingParser.toJson(mapping);
+        table.updateProperties()
+            .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+            .commit();
+      }
+
+      if (isFile || !partition.isEmpty()) {
+        // we do a list operation on the driver to import 1 file or 1 partition
+        PartitionSpec spec = table.spec();
+        MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+        String partitionURI;
+
+        if (isFile) {
+          partitionURI = dataPath.toString();
+        } else {
+          partitionURI = dataPath.toString() + toPartitionPath(partition);
+        }
+
+        List<DataFile> files = SparkTableUtil.listPartition(partition, partitionURI, format, spec, conf, metricsConfig);
+        if (files.size() == 0) {
+          throw new IllegalArgumentException(String.format("No files found for add_file command. Looking in URI %s",
+              partitionURI));
+        }
+
+        AppendFiles append = table.newAppend();
+        files.forEach(append::appendFile);
+        append.commit();
+      } else {
+        // Importing multiple partitions
+        List<SparkTableUtil.SparkPartition> partitions = Spark3Util.getPartitions(spark(), dataPath, format);
+        String stagingLocation = table.properties()
+            .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+        SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      Long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private String toPartitionPath(Map<String, String> partition) {
+    return partition.entrySet().stream()
+        .map(entry -> entry.getKey() + "=" + entry.getValue())
+        .collect(Collectors.joining("/", "/", ""));
+  }
+
+  private void validatePartitionSpec(Table table, Path path, FileSystem fs, Map<String, String> partition) {
+
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add data files to target table %s because that table is partitioned, " +
+            "but the number of columns in the provided partition spec (%d) " +
+            "does not match the number of partitioned columns in table (%d)",
+              table.name(), partition.size(), partitionFields.size()));
+      }
+      partitionFields.forEach(field -> {
+        if (!partition.containsKey(field.name())) {

Review comment:
       From the above discussion, I gather that Hive partitions can also be imported? Do we see issues here around Hive's lowercasing of columns? If yes, then something to consider for future enhancements...

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+        // Forces Name based resolution instead of position based resolution
+        NameMapping mapping = MappingUtil.create(table.schema());
+        String mappingJson = NameMappingParser.toJson(mapping);
+        table.updateProperties()
+            .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+            .commit();
+      }
+
+      if (isFile || !partition.isEmpty()) {
+        // we do a list operation on the driver to import 1 file or 1 partition
+        PartitionSpec spec = table.spec();
+        MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+        String partitionURI;
+
+        if (isFile) {
+          partitionURI = dataPath.toString();
+        } else {
+          partitionURI = dataPath.toString() + toPartitionPath(partition);
+        }
+
+        List<DataFile> files = SparkTableUtil.listPartition(partition, partitionURI, format, spec, conf, metricsConfig);
+        if (files.size() == 0) {
+          throw new IllegalArgumentException(String.format("No files found for add_file command. Looking in URI %s",
+              partitionURI));
+        }
+
+        AppendFiles append = table.newAppend();
+        files.forEach(append::appendFile);
+        append.commit();
+      } else {
+        // Importing multiple partitions
+        List<SparkTableUtil.SparkPartition> partitions = Spark3Util.getPartitions(spark(), dataPath, format);
+        String stagingLocation = table.properties()
+            .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+        SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      Long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private String toPartitionPath(Map<String, String> partition) {
+    return partition.entrySet().stream()
+        .map(entry -> entry.getKey() + "=" + entry.getValue())
+        .collect(Collectors.joining("/", "/", ""));
+  }
+
+  private void validatePartitionSpec(Table table, Path path, FileSystem fs, Map<String, String> partition) {

Review comment:
       probably can be static

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();

Review comment:
       I first interpreted this as Iceberg's `PartitionSpec`. This is more of a `partition` or maybe Hive's concept of a `partiitonSpec`. Should we rename it to `partiiton`?




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+        // Forces Name based resolution instead of position based resolution
+        NameMapping mapping = MappingUtil.create(table.schema());
+        String mappingJson = NameMappingParser.toJson(mapping);
+        table.updateProperties()
+            .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+            .commit();
+      }
+
+      if (isFile || !partition.isEmpty()) {
+        // we do a list operation on the driver to import 1 file or 1 partition
+        PartitionSpec spec = table.spec();
+        MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+        String partitionURI;
+
+        if (isFile) {
+          partitionURI = dataPath.toString();
+        } else {
+          partitionURI = dataPath.toString() + toPartitionPath(partition);
+        }
+
+        List<DataFile> files = SparkTableUtil.listPartition(partition, partitionURI, format, spec, conf, metricsConfig);
+        if (files.size() == 0) {
+          throw new IllegalArgumentException(String.format("No files found for add_file command. Looking in URI %s",
+              partitionURI));
+        }
+
+        AppendFiles append = table.newAppend();
+        files.forEach(append::appendFile);
+        append.commit();
+      } else {
+        // Importing multiple partitions
+        List<SparkTableUtil.SparkPartition> partitions = Spark3Util.getPartitions(spark(), dataPath, format);
+        String stagingLocation = table.properties()
+            .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+        SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      Long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));

Review comment:
       Will do, i changed the signature but forgot to unbox 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] RussellSpitzer commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I think in-memroy file index is fine for the moment, but we can always change it later, now that it's an internal part of just the file based approach it should be easy to swap out.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I thought we would not need to use Spark's in-memory file index for path-based tables to determine which partitions are inside if we use this approach. The main reason is that path-based tables cannot have custom partition locations and we can always construct a partition path if we have a partition map and the root table location. Also, using the in-memory file index is expensive as it will build us a map of all partitions even though we need a couple of them and we can construct partition paths ourselves.
   
   @rdblue, I think your concerns can be addressed by extra validation. We will have access to the Iceberg partition spec. We can validate the map of partition values contains all columns.
   
   




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {

Review comment:
       Per discussion, we are going to support this. I am resolving this thread.




----------------------------------------------------------------
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] talgos1 edited a comment on pull request #2210: Adds AddFiles Procedure

Posted by GitBox <gi...@apache.org>.
talgos1 edited a comment on pull request #2210:
URL: https://github.com/apache/iceberg/pull/2210#issuecomment-976203057


   > @talgos1, have you tried using a full file path as the source table?
   
   @rdblue 
   Using the `CALL` command, it expects the source to have same partition spec as destination (got an error the source has no partitions)
   
   Since my last comment, I succeeded doing that using the spark and java APIs and explicitly defining a synthetic SparkPartition for the file/path
   ```scala
   // Define the source file
   val uri = "/some/path/to/orc/file.orc"
   val format = "orc"
   val partitionSpec: util.Map[String, String] = Map("some_partition_key" -> "some_partition_value").asJava
   // Define a synthetic spark partition
   val sparkPartition = new SparkPartition(partitionSpec, uri,  format)
   
   // Do the add call for importing partitioned source table
   SparkTableUtil.importSparkPartitions(spark, Seq(sparkPartition).asJava, table, spec, stagingDir)
   ```
   
   WDYT?


-- 
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] talgos1 commented on pull request #2210: Adds AddFiles Procedure

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


   
   > @talgos1, have you tried using a full file path as the source table?
   
   @rdblue 
   Using the `CALL` command, it expects the source to have same partition spec as destination (got an error the source has no partitions)
   
   Since my last comment, I succeeded doing that using the spark and java APIs and explicitly defining a synthetic SparkPartition for the file/path
   ```
   // Define the source file
   val uri = "/some/path/to/orc/file.orc"
   val format = "orc"
   val partitionSpec: util.Map[String, String] = Map("some_partition_key" -> "some_partition_value").asJava
   // Define a synthetic spark partition
   val sparkPartition = new SparkPartition(partitionSpec, uri,  format)
   
   // Do the add call for importing partitioned source table
   SparkTableUtil.importSparkPartitions(spark, Seq(sparkPartition).asJava, table, spec, stagingDir)
   ```
   
   WDYT?


-- 
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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       One scenario where we may need to infer partitioning is `migrate('path/to/hadoop/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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {

Review comment:
       Since we use the map as a filter, we no longer should require the map to contain all arguments. 
   
   This should be valid, I guess:
   
   ```
   // table partitioned by p1, p2
   add_files(
     table => 'db.iceberg_table', -- partitioned Iceberg table
     source_table => '`parquet`.`path/to/table/`'
     partition_filter => map('p1', ‘x’)
   )
   ```
   
   That is instead of using *, which is more flexible. 




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       @rdblue, I think `parquet`.`path/to/table` would need to point to the root table location all the time, just like in SQL on file in Spark. The `partition` map would tell us which partitions to import.
   
   Let's consider the following call:
   
   ```
   iceberg.system.add_files(
     source_table => `parquet`.`path/to/table`
     table => 'iceberg.db.tbl',
     partition => map('p1', 'v1', 'p2', 'v2')
   )
   ```
   
   I think the procedure would detect the identifier points to a path-based Parquet table and would take the root table location as a basis and add a path for the partition. In other words, the call above would be translated into importing a single Parquet partition.
   
   ```
   path/to/table/p1=v1/p2=v2
   ```
   
   Does that make sense?
   




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),

Review comment:
       Is it ok not to use `ORDER BY` 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I am open to other alternatives but this idea seems to be the easiest among the options we discussed and we can actually avoid using the in-memory file index for now. We will need it when we need to know all partitions (e.g. full migration).




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {

Review comment:
       nit: why static?

##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataUnpartitionedOrc() {
+    createUnpartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",

Review comment:
       Typo? Should be `orc`?

##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataUnpartitionedOrc() {
+    createUnpartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedHive() {
+    createUnpartitionedHiveTable();
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '%s')",
+        catalogName, tableName, sourceTableName);
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedExtraCol() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String, foo string) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedMissingCol() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataPartitionedMissingCol() {
+    createPartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(8L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataPartitioned() {
+    createPartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(8L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataPartitionedOrc() {
+    createPartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",

Review comment:
       Typo? parquet -> orc.

##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";

Review comment:
       nit: could be final

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private void importFileTable(Table table, Path tableLocation, String format, Map<String, String> partitionFilter) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
+    List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);
+    importPartitions(table, filteredPartitions);
+  }
+
+  private void importCatalogTable(Table table, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent);
+    SparkTableUtil.importSparkTable(spark(), sourceTableIdentifier, table, stagingLocation, partitionFilter);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partitionFilter) {
+    List<PartitionField> partitionFields = table.spec().fields();
+    Set<String> partitionNames = table.spec().fields().stream().map(PartitionField::name).collect(Collectors.toSet());
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partitionFilter.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Not enough partition columns
+      Preconditions.checkArgument(partitionFields.size() >= partitionFilter.size(),
+          "Cannot add data files to target table %s because that table is partitioned, " +
+              "but the number of columns in the provided partition filter (%d) " +
+              "is greater than the number of partitioned columns in table (%d)",
+          table.name(), partitionFilter.size(), partitionFields.size());
+
+      // Check for any non-identity partition columns
+      List<PartitionField> nonIdentityFields =
+          partitionFields.stream().filter(x -> !x.transform().isIdentity()).collect(Collectors.toList());
+      Preconditions.checkArgument(nonIdentityFields.isEmpty(),
+          "Cannot add data files to target table %s because that table is partitioned and contains non-identity" +
+              "partition transforms which will not be compatible. Found non-identity fields %s",
+          table.name(), nonIdentityFields);
+
+      // Check for any filters of non existent columns
+      List<String> unMatchedFilters =

Review comment:
       nit: formatting
   
   ```
   List<String> invalidFilters = partitionFilter.keySet().stream()
       .filter(filterName -> !partitionNames.contains(filterName))
       .collect(Collectors.toList());
   ```

##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);

Review comment:
       Do we need this twice?

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private void importFileTable(Table table, Path tableLocation, String format, Map<String, String> partitionFilter) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
+    List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);
+    importPartitions(table, filteredPartitions);
+  }
+
+  private void importCatalogTable(Table table, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent);
+    SparkTableUtil.importSparkTable(spark(), sourceTableIdentifier, table, stagingLocation, partitionFilter);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partitionFilter) {
+    List<PartitionField> partitionFields = table.spec().fields();
+    Set<String> partitionNames = table.spec().fields().stream().map(PartitionField::name).collect(Collectors.toSet());
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partitionFilter.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Not enough partition columns

Review comment:
       nit: I think this comment can be a bit more precise

##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +648,20 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      return partitions;
+
+    } else {
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions.stream()

Review comment:
       nit: since we removed the code in between, we no longer need the intermediate var.
   
   ```
   return partitions.stream()
       .filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet()))
       .collect(Collectors.toList());
   ```

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private void importFileTable(Table table, Path tableLocation, String format, Map<String, String> partitionFilter) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
+    List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);
+    importPartitions(table, filteredPartitions);
+  }
+
+  private void importCatalogTable(Table table, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent);
+    SparkTableUtil.importSparkTable(spark(), sourceTableIdentifier, table, stagingLocation, partitionFilter);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  @Override
+  public String description() {
+    return null;

Review comment:
       Shall it be "AddFiles" to match other procedures?

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private void importFileTable(Table table, Path tableLocation, String format, Map<String, String> partitionFilter) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
+    List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);
+    importPartitions(table, filteredPartitions);
+  }
+
+  private void importCatalogTable(Table table, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent);
+    SparkTableUtil.importSparkTable(spark(), sourceTableIdentifier, table, stagingLocation, partitionFilter);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partitionFilter) {
+    List<PartitionField> partitionFields = table.spec().fields();
+    Set<String> partitionNames = table.spec().fields().stream().map(PartitionField::name).collect(Collectors.toSet());
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partitionFilter.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Not enough partition columns
+      Preconditions.checkArgument(partitionFields.size() >= partitionFilter.size(),
+          "Cannot add data files to target table %s because that table is partitioned, " +
+              "but the number of columns in the provided partition filter (%d) " +
+              "is greater than the number of partitioned columns in table (%d)",
+          table.name(), partitionFilter.size(), partitionFields.size());
+
+      // Check for any non-identity partition columns
+      List<PartitionField> nonIdentityFields =
+          partitionFields.stream().filter(x -> !x.transform().isIdentity()).collect(Collectors.toList());
+      Preconditions.checkArgument(nonIdentityFields.isEmpty(),

Review comment:
       Sounds like we should check this irrespectively whether we got a filter or not. Also, I'd format it a bit.
   
   ```
   List<PartitionField> nonIdentityFields = partitionFields.stream()
       .filter(x -> !x.transform().isIdentity())
       .collect(Collectors.toList());
   ```

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private void importFileTable(Table table, Path tableLocation, String format, Map<String, String> partitionFilter) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
+    List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);
+    importPartitions(table, filteredPartitions);
+  }
+
+  private void importCatalogTable(Table table, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    String stagingLocation = table.properties()

Review comment:
       nit: should we expose a method for getting the metadata location since we do this in 2 places?
   
   ```
     private String getMetadataLocation(Table table) {
       String defaultValue = table.location() + "/metadata";
       return table.properties().getOrDefault(TableProperties.WRITE_METADATA_LOCATION, defaultValue);
     }
   ```

##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table

Review comment:
       Can we add a couple of tests for Avro?

##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+

Review comment:
       nit: extra line

##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataUnpartitionedOrc() {
+    createUnpartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedHive() {
+    createUnpartitionedHiveTable();
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '%s')",
+        catalogName, tableName, sourceTableName);
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedExtraCol() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String, foo string) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedMissingCol() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataPartitionedMissingCol() {
+    createPartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(8L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataPartitioned() {
+    createPartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(8L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataPartitionedOrc() {
+    createPartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(8L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addDataPartitionedHive() {
+    createPartitionedHiveTable();
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '%s')",
+        catalogName, tableName, sourceTableName);
+
+    Assert.assertEquals(8L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addPartitionToPartitioned() {
+    createPartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addFilteredPartitionsToPartitioned() {
+    createCompositePartitionedTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg " +
+            "PARTITIONED BY (id, dept)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void addPartitionToPartitionedHive() {
+    createPartitionedHiveTable();
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '%s', map('id', 1))",
+        catalogName, tableName, sourceTableName);
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void invalidDataImport() {
+    createPartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    AssertHelpers.assertThrows("Should forbid adding of partitioned data to unpartitioned table",

Review comment:
       Should we test the scenario without the partition filter too?

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();
+    Identifier sourceIdent = toCatalogAndIdentifier(args.getString(1), PARAMETERS[1].name(), sessionCat).identifier();
+
+    Map<String, String> partitionFilter = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionFilter.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    long addedFilesCount = importToIceberg(tableIdent, sourceIdent, partitionFilter);
+    return new InternalRow[]{newInternalRow(addedFilesCount)};
+  }
+
+  private boolean isFileIdentifier(Identifier ident) {
+    String[] namespace = ident.namespace();
+    return namespace.length == 1 &&
+        (namespace[0].equalsIgnoreCase("orc") ||
+            namespace[0].equalsIgnoreCase("parquet") ||
+            namespace[0].equalsIgnoreCase("avro"));
+  }
+
+  private long importToIceberg(Identifier destIdent, Identifier sourceIdent, Map<String, String> partitionFilter) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partitionFilter);
+      ensureNameMappingPresent(table);
+
+      if (isFileIdentifier(sourceIdent)) {
+        Path sourcePath = new Path(sourceIdent.name());
+        String format = sourceIdent.namespace()[0];
+        importFileTable(table, sourcePath, format, partitionFilter);
+      } else {
+        importCatalogTable(table, sourceIdent, partitionFilter);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      return Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+    });
+  }
+
+  private static void ensureNameMappingPresent(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private void importFileTable(Table table, Path tableLocation, String format, Map<String, String> partitionFilter) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), tableLocation, format);
+    List<SparkPartition> filteredPartitions = SparkTableUtil.filterPartitions(partitions, partitionFilter);

Review comment:
       Do we need to validate the list is non-empty?

##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",

Review comment:
       It seems like `result` would be a more appropriate name for such vars.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {

Review comment:
       Does it mean we will allow pointing to the root table location like this?
   
   ```
   add_files(
     table => 'db.iceberg_table', -- partitioned Iceberg table
     source_table => '`parquet`.`path/to/table`'
   )
   ```
   
   I am still not sure it is a good idea. I think `add_files` should be more about incremental migration/import where you have a single or a couple of partitions to import. Supporting bulk imports is useful but inferring which partitions are inside seems too much in `add_files`. Other use cases should be covered by snapshot/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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)

Review comment:
       Let's call it `partition_filter` to better reflect what it does.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       Supporting bulk imports can be done by expanding `*` in the partition values.
   
   For example, `map('p1', '*', 'p2', 'v2')` would import all partitions where `p2=v2`.




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet()))

Review comment:
       I'm not sure, but if we don't match here we don't match against the iceberg column names which means we'll have some issues later. I'll add a test to check




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   I like the new structure, I had only minor points so far. I'll do another look with fresh eyes tomorrow but I think this is close.


----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())

Review comment:
       +1, I think this matches snapshot and migrate procedures.




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);

Review comment:
       I forgot I refactored this down from 2 different source 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] RussellSpitzer commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       We can't use the version in SparkUtils because unfortunately the Scala api for this method has added additional default args. In Scala this wouldn't be a problem, but in java we need to explicitly call the function with all args.




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet()))
+          .collect(Collectors.toList());
+
+      Preconditions.checkArgument(!filteredPartitions.isEmpty(),

Review comment:
       Well, this one is a bit different. So we could probably validate it here. Do we need to actually fail this with an exception? Or should this be noop?




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   @pvary + @aokolnychyi Should be ready for another look if ya'll have time, Thanks in advance!


----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();

Review comment:
       We currently don't support any catalogplugin but the session catalog for this (for the source v1 table lookups)




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   Thank you, @RussellSpitzer! Thanks everyone who participated in the review!


-- 
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet()))

Review comment:
       Also, do we handle spaces and special chars?




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataUnpartitionedOrc() {
+    createUnpartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedHive() {
+    createUnpartitionedHiveTable();
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '%s')",
+        catalogName, tableName, sourceTableName);
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedExtraCol() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String, foo string) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedMissingCol() {

Review comment:
       Yeah let me add a test to make sure that throws an 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] RussellSpitzer commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table

Review comment:
       Sure




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       Could we use reflection for this? Seems we had a similar issue in `HiveClientPool`.
   
   I understand `InMemoryFileIndex` is a low-level API and it may not be the best idea to rely on it. I'm ok with this as the first step. We may build our own logic if we need it in other query engines.
   
   I'd be also alright to always require a specific partition to import and not support importing all partitions in a given path, which should avoid the need to use Spark's `InMemoryFileIndex` to infer the underlying partitioning. As I wrote before, I think the primary use case for this is CREATE TABLE LIKE and then calling `add_files` for a couple of partitions to try reads/writes or migrating partition by partition if a table has too many partitions to migrate them at once. In both cases, there is a specific partition we have to import.
   
   If there is a need to import files in a given location, one can create an external Spark table pointing to the location, call `MSCK REPAIR TABLE` to infer partitions and add them to the metastore and then call either snapshot or migrate procedure.
   
   What are your thoughts, @RussellSpitzer @rdblue?
   
   




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +802,64 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),
+        Option.empty(),
+        fileStatusCache,
+        Option.empty(),
+        Option.empty());
+
+    org.apache.spark.sql.execution.datasources.PartitionSpec spec = fileIndex.partitionSpec();
+    StructType schema = spec.partitionColumns();
+    if (spec.partitions().isEmpty()) {
+      return ImmutableList.of(new SparkPartition(Collections.emptyMap(), rootPath.toString(), format));

Review comment:
       That ... may have been a different problem ... I'll figure this out




-- 
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet()))

Review comment:
       Let me think about this 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] pvary commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataUnpartitionedOrc() {
+    createUnpartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedHive() {
+    createUnpartitionedHiveTable();
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '%s')",
+        catalogName, tableName, sourceTableName);
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT * FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedExtraCol() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String, foo string) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s", sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s", tableName));
+  }
+
+  @Test
+  public void addDataUnpartitionedMissingCol() {
+    createUnpartitionedFileTable("parquet");
+    String createIceberg =

Review comment:
       nit of the nit: In this file you usually leave an empty line after table creation 😄 




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I think adding Iceberg files through this API is off the table. The only case we are talking about is the import of existing non-Iceberg tables/partitions. In this PR, Russell uses `InMemoryFileIndex` to get a list of partitions for a given root table location.
   
   The question I was debating in my previous comment is whether we need to support importing root table locations. As I said earlier, the main use case for `add_files` will probably be to add a few partitions after CREATE TABLE LIKE if the original non-Iceberg table contains a lot of partitions and calling SNAPSHOT may be expensive. Another use case is gradual migration. In that case, we don't need to infer what partitions are present. We can have the following procedure:
   
   ```
   iceberg.system.add_files(
     table => 'db.tbl', -- required
     path => '/root/table/location', -- required
     format => 'parquet', -- required
     partition => map('p1', 'v1', 'p2', 'v2') -- required for partitioned tables
   )
   ```
   
   In this case, we just combine the partition tuple with the root location and import a single partition at a time.
   
   The question is whether we should make `partition` optional for partitioned tables so that you can just point to the root table location. If yes, we will have to use something like `InMemoryFileIndex` or our own solution to find out which partitions are present in that location. I would like to avoid that but we need to come up with a solution for importing non-HMS tables.
   
   For example, I think this should be valid:
   
   ```
   MIGRATE `parquet`.`/path/to/table/on/hdfs`
   USING iceberg
   ```
   
   It seems we will need to know what partitions are present in that case.
   
    




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +803,66 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(
+        spark,
+        JavaConverters
+            .collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
+            .asScala()
+            .toSeq(),
+        JavaConverters
+            .mapAsScalaMapConverter(emptyMap)
+            .asScala()
+            .toMap(Predef.<Tuple2<String, String>>conforms()),
+        Option.empty(),
+        fileStatusCache,
+        Option.empty(),
+        Option.empty());
+
+    org.apache.spark.sql.execution.datasources.PartitionSpec spec = fileIndex.partitionSpec();
+    StructType schema = spec.partitionColumns();
+    if (spec.partitions().isEmpty()) {
+      return ImmutableList.of(new SparkPartition(Collections.emptyMap(), rootPath.toString(), format));
+    }
+
+    return JavaConverters
+        .seqAsJavaListConverter(spec.partitions())
+        .asJava()
+        .stream()
+        .map(partition -> {
+          Map<String, String> values = new HashMap<>();
+          JavaConverters.asJavaIterableConverter(schema).asJava().forEach(field -> {
+            int fieldIndex = schema.fieldIndex(field.name());
+            Object catalystValue = partition.values().get(fieldIndex, field.dataType());
+            Object value = CatalystTypeConverters.convertToScala(catalystValue, field.dataType());
+            values.put(field.name(), value.toString());
+          });
+          return new SparkPartition(values, partition.path().toString(), format);
+        }).collect(Collectors.toList());
+  }
+
+  public static org.apache.spark.sql.catalyst.TableIdentifier toV1TableIdentifier(Identifier identifier) {
+    Preconditions.checkArgument(identifier.namespace().length <= 1,
+        "Cannot load a session catalog namespace with more than 1 part. Given %s", identifier);
+
+    Option<String> namespace =
+        identifier.namespace().length == 1 ? Option.apply(identifier.namespace()[0]) : Option.empty();
+
+    org.apache.spark.sql.catalyst.TableIdentifier tableIdent =

Review comment:
       sgtm




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("path", DataTypes.StringType),
+      ProcedureParameter.required("format", DataTypes.StringType),
+      ProcedureParameter.optional("partition_value", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+    String path = args.getString(1);
+    String format =  args.getString(2);
+
+    Map<String, String> partitionSpec = Maps.newHashMap();
+    if (!args.isNullAt(3)) {
+      args.getMap(3).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partitionSpec.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importDataToIcebergTable(tableIdent, path, format, partitionSpec);
+
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Long importDataToIcebergTable(Identifier tableIdent, String path, String format,
+                                        Map<String, String> partition) {
+    return modifyIcebergTable(tableIdent, table -> {
+      Configuration conf = spark().sessionState().newHadoopConf();
+      Path dataPath = new Path(path);
+      FileSystem fs;
+      Boolean isFile;
+
+      try {
+        fs = dataPath.getFileSystem(conf);
+        isFile = fs.getFileStatus(dataPath).isFile();
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to access add_file path", e);
+      }
+      validatePartitionSpec(table, dataPath, fs, partition);
+
+      if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+        // Forces Name based resolution instead of position based resolution
+        NameMapping mapping = MappingUtil.create(table.schema());
+        String mappingJson = NameMappingParser.toJson(mapping);
+        table.updateProperties()
+            .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+            .commit();
+      }
+
+      if (isFile || !partition.isEmpty()) {
+        // we do a list operation on the driver to import 1 file or 1 partition
+        PartitionSpec spec = table.spec();
+        MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+        String partitionURI;
+
+        if (isFile) {
+          partitionURI = dataPath.toString();
+        } else {
+          partitionURI = dataPath.toString() + toPartitionPath(partition);
+        }
+
+        List<DataFile> files = SparkTableUtil.listPartition(partition, partitionURI, format, spec, conf, metricsConfig);
+        if (files.size() == 0) {
+          throw new IllegalArgumentException(String.format("No files found for add_file command. Looking in URI %s",
+              partitionURI));
+        }
+
+        AppendFiles append = table.newAppend();
+        files.forEach(append::appendFile);
+        append.commit();
+      } else {
+        // Importing multiple partitions
+        List<SparkTableUtil.SparkPartition> partitions = Spark3Util.getPartitions(spark(), dataPath, format);
+        String stagingLocation = table.properties()
+            .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+        SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      Long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private String toPartitionPath(Map<String, String> partition) {

Review comment:
       This is for specifying a particular partition to import, by default we import all of them. But you are correct this would skip partitions with the right value but alternate 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] aokolnychyi commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet()))
+          .collect(Collectors.toList());
+
+      Preconditions.checkArgument(!filteredPartitions.isEmpty(),

Review comment:
       I feel like this validation should be outside too. It is specific to `add_files` while the method is used in multiple places.




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java
##########
@@ -0,0 +1,437 @@
+/*
+ * 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.spark.extensions;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAddFilesProcedure extends SparkExtensionsTestBase {
+
+  private String sourceTableName = "source_table";
+  private File fileTableDir;
+
+  public TestAddFilesProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void setupTempDirs() {
+    try {
+      fileTableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void dropTables() {
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", sourceTableName);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void addDataUnpartitioned() {
+    createUnpartitionedFileTable("parquet");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",
+        catalogName, tableName, fileTableDir.getAbsolutePath());
+
+    Assert.assertEquals(2L, importOperation);
+
+    assertEquals("Iceberg table contains correct data",
+        sql("SELECT * FROM %s ORDER BY id", sourceTableName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+
+  @Ignore  // Classpath issues prevent us from actually writing to a Spark ORC table
+  public void addDataUnpartitionedOrc() {
+    createUnpartitionedFileTable("orc");
+
+    String createIceberg =
+        "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg";
+
+    sql(createIceberg, tableName);
+
+    Object importOperation = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`')",

Review comment:
       ah yeah, I coudln't even run these tests so I didn't notice. 




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format("No partitions found in table for add_file command. " +
+                    "Looking for a partition with the value %s",
+                partition.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue())
+                    .collect(Collectors.joining(",")))
+        );
+      }
+
+      return filteredPartitions;
+    }
+  }
+
+  @Override
+  public String description() {
+    return null;
+  }
+
+  private void validatePartitionSpec(Table table, Map<String, String> partition) {
+    List<PartitionField> partitionFields = table.spec().fields();
+
+    boolean tablePartitioned = !partitionFields.isEmpty();
+    boolean partitionSpecPassed = !partition.isEmpty();
+
+    if (tablePartitioned && partitionSpecPassed) {
+      // Has Partitions, Check that they are valid
+      if (partitionFields.size() != partition.size()) {
+        throw new IllegalArgumentException(
+          String.format(
+            "Cannot add data files to target table %s because that table is partitioned, " +
+            "but the number of columns in the provided partition spec (%d) " +
+            "does not match the number of partitioned columns in table (%d)",
+              table.name(), partition.size(), partitionFields.size()));
+      }
+      partitionFields.forEach(field -> {
+        if (!partition.containsKey(field.name())) {
+          throw new IllegalArgumentException(
+            String.format(
+              "Cannot add files to target table %s. %s is partitioned but the specified partition spec " +
+              "refers to a column that is not partitioned: '%s'",
+                table.name(), table.name(), field.name())
+            );
+        }
+      });

Review comment:
       I think we need to add validation that all columns from Iceberg partition spec are present in the partition map provided by the user to avoid cases mentioned by @rdblue.
   
   ```
   add_files(
     table => 'db.iceberg_table', -- partitioned Iceberg table
     source_table => '`parquet`.`path/to/table/part_col_1=x`'
     partition => map('part_col_2', 'y')
   )
   ```




----------------------------------------------------------------
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] rdblue commented on a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java
##########
@@ -790,4 +804,53 @@ public Identifier identifier() {
   public static TableIdentifier identifierToTableIdentifier(Identifier identifier) {
     return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
   }
+
+  /**
+   * Use Spark to list all partitions in the table.
+   *
+   * @param spark a Spark session
+   * @param rootPath a table identifier
+   * @param format format of the file
+   * @return all table's partitions
+   */
+  public static List<SparkTableUtil.SparkPartition> getPartitions(SparkSession spark, Path rootPath, String format) {
+    FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark);
+    Map<String, String> emptyMap = Collections.emptyMap();
+
+    InMemoryFileIndex fileIndex = new InMemoryFileIndex(

Review comment:
       I agree that migrating a location-based table is a valid use case. To me, that's slightly different because like Hive it is a "defined" table format. We would also convert the detected partition structure to a partition spec that uses identity, so it is predictable and limited.
   
   For importing files, I don't want to get into a situation where we have to support non-identity partition transforms or guess how to match up the partitioning. I think the safest thing is to import a single partition at a time, but that would be a pain. Maybe if we can come up with reasonable rules for this we can make it work:
   1. All partitions must be identity partitions
   2. No transforming data except to parse numbers
   
   Others?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
##########
@@ -621,6 +640,32 @@ public static void importSparkPartitions(
     }
   }
 
+  public static List<SparkPartition> filterPartitions(List<SparkPartition> partitions,
+                                                      Map<String, String> partitionFilter) {
+    if (partitionFilter.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet()))

Review comment:
       Do we do any escaping/normalization for column names and values?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =

Review comment:
       Shall we split it into multiple lines, import `SparkPartition` directly, and rename `partition`?
   What about this?
   
   ```
   List<SparkPartition> partitions = Spark3Util.getPartitions(spark(), location, format.name());
   List<SparkPartition> matchingPartitions = filterPartitions(partitions, partitionFilter);
   importPartitions(table, matchingPartitions);
   ```




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.spark.procedures;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+public class AddFilesProcedure extends BaseProcedure {
+
+  enum Format {
+    catalog,
+    orc,
+    parquet
+  }
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    String sourceTable = args.getString(1);
+    Format format = parseFormat(sourceTable);
+
+    Map<String, String> partition = Maps.newHashMap();
+    if (!args.isNullAt(2)) {
+      args.getMap(2).foreach(DataTypes.StringType, DataTypes.StringType,
+          (k, v) -> {
+            partition.put(k.toString(), v.toString());
+            return BoxedUnit.UNIT;
+          });
+    }
+
+    Long filesAdded = importToIceberg(tableIdent, sourceTable, format, partition);
+    return new InternalRow[]{newInternalRow(filesAdded)};
+  }
+
+  private Format parseFormat(String source) {
+    String[] parts = source.split("\\.", 2);
+    if (parts.length == 2) {
+      if (parts[0].toLowerCase(Locale.ROOT).equals("orc")) {
+        return Format.orc;
+      }
+      if (parts[0].toLowerCase(Locale.ROOT).equals("parquet")) {
+        return Format.parquet;
+      }
+      return Format.catalog;
+    }
+    return Format.catalog;
+  }
+
+  private long importToIceberg(Identifier destIdent, String sourceIdent, Format format, Map<String, String> partition) {
+    return modifyIcebergTable(destIdent, table -> {
+
+      validatePartitionSpec(table, partition);
+      applyNameMappingIfMissing(table);
+
+      if (format != Format.catalog) {
+        String[] parts = sourceIdent.split("\\.", 2);
+        Path sourcePath = new Path(parts[1]);
+
+        Configuration conf = spark().sessionState().newHadoopConf();
+        FileSystem fs;
+        Boolean isFile;
+        try {
+          fs = sourcePath.getFileSystem(conf);
+          isFile = fs.getFileStatus(sourcePath).isFile();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to access add_file path", e);
+        }
+
+        if (isFile) {
+          importFile(table, sourcePath, format, partition);
+        } else {
+          importFileTable(table, sourcePath, format, partition);
+        }
+      } else {
+        importCatalogTable(table, sourceIdent, partition);
+      }
+
+      Snapshot snapshot = table.currentSnapshot();
+      long numAddedFiles = Long.parseLong(snapshot.summary().get(SnapshotSummary.ADDED_FILES_PROP));
+      return numAddedFiles;
+    });
+  }
+
+  private static void applyNameMappingIfMissing(Table table) {
+    if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) {
+      // Forces Name based resolution instead of position based resolution
+      NameMapping mapping = MappingUtil.create(table.schema());
+      String mappingJson = NameMappingParser.toJson(mapping);
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson)
+          .commit();
+    }
+  }
+
+  private int importFile(Table table, Path file, Format format,  Map<String, String> partition) {
+    if (partition.isEmpty() && !table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to a partitioned table without specifying the partition " +
+          "which it should be placed in");
+    }
+    if (!partition.isEmpty() && table.spec().isUnpartitioned()) {
+      throw new IllegalArgumentException("Cannot add a file to an unpartitioned table while specifying the partition " +
+          "which it should be placed in");
+    }
+
+    PartitionSpec spec = table.spec();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(table.properties());
+    String partitionURI = file.toString();
+    Configuration conf = spark().sessionState().newHadoopConf();
+
+    List<DataFile> files =
+        SparkTableUtil.listPartition(partition, partitionURI, format.name(), spec, conf, metricsConfig);
+
+    if (files.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("No file found for add_file command. Looking for a file at URI %s", partitionURI));
+    }
+
+    // Add Snapshot Summary Info?
+    AppendFiles append = table.newAppend();
+    files.forEach(append::appendFile);
+    append.commit();
+    return 1;
+  }
+
+  private void importFileTable(Table table, Path tableLocation, Format format, Map<String, String> partition) {
+    // List Partitions via Spark InMemory file search interface
+    List<SparkTableUtil.SparkPartition> partitions =
+        filterPartitions(Spark3Util.getPartitions(spark(), tableLocation, format.name()), partition);
+
+    importPartitions(table, partitions);
+  }
+
+  private void importCatalogTable(Table table, String sourceTable, Map<String, String> partition) {
+    CatalogTable catalogTable = SparkTableUtil.getCatalogTable(spark(), sourceTable);
+    List<SparkTableUtil.SparkPartition> partitions;
+
+    if (catalogTable.partitionColumnNames().isEmpty()) {
+      // Unpartitioned table
+      if (!partition.isEmpty()) {
+        throw new IllegalArgumentException("Cannot import files from a unpartitioned Hive table when a partition " +
+            "is specified");
+      }
+      Option<String> format =
+          catalogTable.storage().serde().nonEmpty() ? catalogTable.storage().serde() : catalogTable.provider();
+      partitions = ImmutableList.of(new SparkTableUtil.SparkPartition(
+          Collections.emptyMap(),
+          catalogTable.location().toString(),
+          catalogTable.storage().serde().get()));
+
+    } else {
+      // Read partitions Spark via Catalog Interface
+      partitions = filterPartitions(SparkTableUtil.getPartitions(spark(), sourceTable), partition);
+    }
+
+    importPartitions(table, partitions);
+  }
+
+  private void importPartitions(Table table, List<SparkTableUtil.SparkPartition> partitions) {
+    String stagingLocation = table.properties()
+        .getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/metadata");
+    SparkTableUtil.importSparkPartitions(spark(), partitions, table, table.spec(), stagingLocation);
+  }
+
+  private List<SparkTableUtil.SparkPartition> filterPartitions(List<SparkTableUtil.SparkPartition> partitions,
+                                                               Map<String, String> partition) {
+    if (partition.isEmpty()) {
+      // No partition filter arg
+
+      if (partitions.isEmpty()) {
+        throw new IllegalArgumentException("Cannot add files, no files found in the table.");
+      }
+      return partitions;
+    } else {
+      // Partition filter arg passed
+
+      List<SparkTableUtil.SparkPartition> filteredPartitions = partitions
+          .stream().filter(p -> p.getValues().equals(partition)) // Todo Support Wildcards
+          .collect(Collectors.toList());
+
+      if (filteredPartitions.isEmpty()) {
+        throw new IllegalArgumentException(

Review comment:
       nit: Preconditions?




----------------------------------------------------------------
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 a change in pull request #2210: Adds AddFiles Procedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.connector.catalog.CatalogPlugin;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.runtime.BoxedUnit;
+
+class AddFilesProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[]{
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.required("source_table", DataTypes.StringType),
+      ProcedureParameter.optional("partition_filter", STRING_MAP)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty())
+  });
+
+  private AddFilesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<AddFilesProcedure>() {
+      @Override
+      protected AddFilesProcedure doBuild() {
+        return new AddFilesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
+
+    CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog();

Review comment:
       Then let's keep it as is.




----------------------------------------------------------------
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 #2210: Adds AddFiles Procedure

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


   I'm not sure what you are asking, the syntax for this command is 
   ```
   CALL spark_catalog.system.add_files(
     table => 'db.tbl',
     source_table => '`parquet`.`path/to/table`'
   )
   ```
   I'll be adding docs shortly but you should probably ask your question with more detail about what you are trying to accomplish on the mailing list or the Slack since this may not do what you expect it to do.


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