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: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
