|
33 | 33 | import org.apache.iceberg.Table; |
34 | 34 | import org.apache.iceberg.TableProperties; |
35 | 35 | import org.apache.iceberg.hadoop.HadoopTables; |
| 36 | +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; |
36 | 37 | import org.apache.iceberg.relocated.com.google.common.collect.Lists; |
37 | 38 | import org.apache.iceberg.spark.SparkWriteOptions; |
38 | 39 | import org.apache.iceberg.types.Types; |
@@ -188,6 +189,50 @@ public void testAppend() throws IOException { |
188 | 189 | Assert.assertEquals("Result rows should match", expected, actual); |
189 | 190 | } |
190 | 191 |
|
| 192 | + @Test |
| 193 | + public void testEmptyOverwrite() throws IOException { |
| 194 | + File parent = temp.newFolder(format.toString()); |
| 195 | + File location = new File(parent, "test"); |
| 196 | + |
| 197 | + HadoopTables tables = new HadoopTables(CONF); |
| 198 | + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); |
| 199 | + Table table = tables.create(SCHEMA, spec, location.toString()); |
| 200 | + |
| 201 | + List<SimpleRecord> records = Lists.newArrayList( |
| 202 | + new SimpleRecord(1, "a"), |
| 203 | + new SimpleRecord(2, "b"), |
| 204 | + new SimpleRecord(3, "c") |
| 205 | + ); |
| 206 | + |
| 207 | + List<SimpleRecord> expected = records; |
| 208 | + Dataset<Row> df = spark.createDataFrame(records, SimpleRecord.class); |
| 209 | + |
| 210 | + df.select("id", "data").write() |
| 211 | + .format("iceberg") |
| 212 | + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) |
| 213 | + .mode(SaveMode.Append) |
| 214 | + .save(location.toString()); |
| 215 | + |
| 216 | + Dataset<Row> empty = spark.createDataFrame(ImmutableList.of(), SimpleRecord.class); |
| 217 | + // overwrite with 2*id to replace record 2, append 4 and 6 |
| 218 | + empty.select("id", "data").write() |
| 219 | + .format("iceberg") |
| 220 | + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) |
| 221 | + .mode(SaveMode.Overwrite) |
| 222 | + .option("overwrite-mode", "dynamic") |
| 223 | + .save(location.toString()); |
| 224 | + |
| 225 | + table.refresh(); |
| 226 | + |
| 227 | + Dataset<Row> result = spark.read() |
| 228 | + .format("iceberg") |
| 229 | + .load(location.toString()); |
| 230 | + |
| 231 | + List<SimpleRecord> actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); |
| 232 | + Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); |
| 233 | + Assert.assertEquals("Result rows should match", expected, actual); |
| 234 | + } |
| 235 | + |
191 | 236 | @Test |
192 | 237 | public void testOverwrite() throws IOException { |
193 | 238 | File parent = temp.newFolder(format.toString()); |
|
0 commit comments