|
32 | 32 | import org.apache.iceberg.Table; |
33 | 33 | import org.apache.iceberg.TableProperties; |
34 | 34 | import org.apache.iceberg.hadoop.HadoopTables; |
| 35 | +import org.apache.iceberg.spark.SparkSchemaUtil; |
35 | 36 | import org.apache.iceberg.types.Types; |
36 | 37 | import org.apache.spark.sql.Dataset; |
37 | 38 | import org.apache.spark.sql.Encoders; |
@@ -338,4 +339,65 @@ public void testPartitionedCreateWithTargetFileSizeViaOption() throws IOExceptio |
338 | 339 | Assert.assertEquals("Should have 8 DataFiles", 8, files.size()); |
339 | 340 | Assert.assertTrue("All DataFiles contain 1000 rows", files.stream().allMatch(d -> d.recordCount() == 1000)); |
340 | 341 | } |
| 342 | + |
| 343 | + @Test |
| 344 | + public void testNestedPartitioning() throws IOException { |
| 345 | + Schema nestedSchema = new Schema( |
| 346 | + optional(1, "id", Types.IntegerType.get()), |
| 347 | + optional(2, "data", Types.StringType.get()), |
| 348 | + optional(3, "nestedData", Types.StructType.of( |
| 349 | + optional(4, "id", Types.IntegerType.get()), |
| 350 | + optional(5, "moreData", Types.StringType.get()))) |
| 351 | + ); |
| 352 | + |
| 353 | + File parent = temp.newFolder("parquet"); |
| 354 | + File location = new File(parent, "test"); |
| 355 | + |
| 356 | + HadoopTables tables = new HadoopTables(new Configuration()); |
| 357 | + PartitionSpec spec = PartitionSpec.builderFor(nestedSchema) |
| 358 | + .identity("id") |
| 359 | + .identity("nestedData.moreData") |
| 360 | + .build(); |
| 361 | + Table table = tables.create(nestedSchema, spec, location.toString()); |
| 362 | + |
| 363 | + List<String> jsons = Lists.newArrayList( |
| 364 | + "{ \"id\": 1, \"data\": \"a\", \"nestedData\": { \"id\": 100, \"moreData\": \"p1\"} }", |
| 365 | + "{ \"id\": 2, \"data\": \"b\", \"nestedData\": { \"id\": 200, \"moreData\": \"p1\"} }", |
| 366 | + "{ \"id\": 3, \"data\": \"c\", \"nestedData\": { \"id\": 300, \"moreData\": \"p2\"} }", |
| 367 | + "{ \"id\": 4, \"data\": \"d\", \"nestedData\": { \"id\": 400, \"moreData\": \"p2\"} }" |
| 368 | + ); |
| 369 | + Dataset<Row> df = spark.read().schema(SparkSchemaUtil.convert(nestedSchema)) |
| 370 | + .json(spark.createDataset(jsons, Encoders.STRING())); |
| 371 | + |
| 372 | + // TODO: incoming columns must be ordered according to the table's schema |
| 373 | + df.select("id", "data", "nestedData").write() |
| 374 | + .format("iceberg") |
| 375 | + .mode("append") |
| 376 | + .save(location.toString()); |
| 377 | + |
| 378 | + table.refresh(); |
| 379 | + |
| 380 | + Dataset<Row> result = spark.read() |
| 381 | + .format("iceberg") |
| 382 | + .load(location.toString()); |
| 383 | + |
| 384 | + List<Row> actual = result.orderBy("id").collectAsList(); |
| 385 | + Assert.assertEquals("Number of rows should match", jsons.size(), actual.size()); |
| 386 | + Assert.assertEquals("Row 1 col 1 is 1", 1, actual.get(0).getInt(0)); |
| 387 | + Assert.assertEquals("Row 1 col 2 is a", "a", actual.get(0).getString(1)); |
| 388 | + Assert.assertEquals("Row 1 col 3,1 is 100", 100, actual.get(0).getStruct(2).getInt(0)); |
| 389 | + Assert.assertEquals("Row 1 col 3,2 is p1", "p1", actual.get(0).getStruct(2).getString(1)); |
| 390 | + Assert.assertEquals("Row 2 col 1 is 1", 2, actual.get(1).getInt(0)); |
| 391 | + Assert.assertEquals("Row 2 col 2 is a", "b", actual.get(1).getString(1)); |
| 392 | + Assert.assertEquals("Row 2 col 3,1 is 100", 200, actual.get(1).getStruct(2).getInt(0)); |
| 393 | + Assert.assertEquals("Row 2 col 3,2 is p1", "p1", actual.get(1).getStruct(2).getString(1)); |
| 394 | + Assert.assertEquals("Row 3 col 1 is 1", 3, actual.get(2).getInt(0)); |
| 395 | + Assert.assertEquals("Row 3 col 2 is a", "c", actual.get(2).getString(1)); |
| 396 | + Assert.assertEquals("Row 3 col 3,1 is 100", 300, actual.get(2).getStruct(2).getInt(0)); |
| 397 | + Assert.assertEquals("Row 3 col 3,2 is p1", "p2", actual.get(2).getStruct(2).getString(1)); |
| 398 | + Assert.assertEquals("Row 4 col 1 is 1", 4, actual.get(3).getInt(0)); |
| 399 | + Assert.assertEquals("Row 4 col 2 is a", "d", actual.get(3).getString(1)); |
| 400 | + Assert.assertEquals("Row 4 col 3,1 is 100", 400, actual.get(3).getStruct(2).getInt(0)); |
| 401 | + Assert.assertEquals("Row 4 col 3,2 is p1", "p2", actual.get(3).getStruct(2).getString(1)); |
| 402 | + } |
341 | 403 | } |
0 commit comments