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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to