|
21 | 21 | import static org.hamcrest.MatcherAssert.assertThat;
|
22 | 22 | import static org.hamcrest.Matchers.containsInAnyOrder;
|
23 | 23 |
|
| 24 | +import java.io.File; |
| 25 | +import java.io.IOException; |
24 | 26 | import java.util.Arrays;
|
25 | 27 | import java.util.Collection;
|
26 | 28 | import java.util.List;
|
27 | 29 | import java.util.Map;
|
28 | 30 | import java.util.UUID;
|
29 | 31 | import java.util.stream.Collectors;
|
30 | 32 | import java.util.stream.Stream;
|
| 33 | +import org.apache.avro.generic.GenericData; |
| 34 | +import org.apache.avro.generic.GenericRecord; |
31 | 35 | import org.apache.beam.sdk.coders.RowCoder;
|
32 | 36 | import org.apache.beam.sdk.schemas.Schema;
|
33 | 37 | import org.apache.beam.sdk.testing.PAssert;
|
|
36 | 40 | import org.apache.beam.sdk.transforms.ParDo;
|
37 | 41 | import org.apache.beam.sdk.values.PCollection;
|
38 | 42 | import org.apache.beam.sdk.values.Row;
|
| 43 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; |
39 | 44 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
|
| 45 | +import org.apache.hadoop.conf.Configuration; |
| 46 | +import org.apache.hadoop.fs.Path; |
40 | 47 | import org.apache.iceberg.CatalogUtil;
|
| 48 | +import org.apache.iceberg.DataFile; |
| 49 | +import org.apache.iceberg.DataFiles; |
| 50 | +import org.apache.iceberg.FileFormat; |
| 51 | +import org.apache.iceberg.Metrics; |
| 52 | +import org.apache.iceberg.MetricsConfig; |
41 | 53 | import org.apache.iceberg.PartitionKey;
|
42 | 54 | import org.apache.iceberg.PartitionSpec;
|
43 | 55 | import org.apache.iceberg.Table;
|
| 56 | +import org.apache.iceberg.TableProperties; |
44 | 57 | import org.apache.iceberg.catalog.TableIdentifier;
|
| 58 | +import org.apache.iceberg.data.Record; |
| 59 | +import org.apache.iceberg.hadoop.HadoopInputFile; |
| 60 | +import org.apache.iceberg.mapping.MappedField; |
| 61 | +import org.apache.iceberg.mapping.MappedFields; |
| 62 | +import org.apache.iceberg.mapping.NameMapping; |
| 63 | +import org.apache.iceberg.mapping.NameMappingParser; |
| 64 | +import org.apache.iceberg.parquet.ParquetUtil; |
45 | 65 | import org.apache.iceberg.types.Types;
|
| 66 | +import org.apache.parquet.avro.AvroParquetWriter; |
| 67 | +import org.apache.parquet.hadoop.ParquetWriter; |
46 | 68 | import org.junit.ClassRule;
|
47 | 69 | import org.junit.Rule;
|
48 | 70 | import org.junit.Test;
|
@@ -210,4 +232,184 @@ public void testIdentityColumnScan() throws Exception {
|
210 | 232 |
|
211 | 233 | testPipeline.run();
|
212 | 234 | }
|
| 235 | + |
| 236 | + @Test |
| 237 | + public void testNameMappingScan() throws Exception { |
| 238 | + org.apache.avro.Schema metadataSchema = |
| 239 | + org.apache.avro.Schema.createRecord( |
| 240 | + "metadata", |
| 241 | + null, |
| 242 | + null, |
| 243 | + false, |
| 244 | + ImmutableList.of( |
| 245 | + new org.apache.avro.Schema.Field( |
| 246 | + "source", |
| 247 | + org.apache.avro.Schema.create(org.apache.avro.Schema.Type.STRING), |
| 248 | + null, |
| 249 | + null))); |
| 250 | + |
| 251 | + org.apache.avro.Schema avroSchema = |
| 252 | + org.apache.avro.Schema.createRecord( |
| 253 | + "test", |
| 254 | + null, |
| 255 | + null, |
| 256 | + false, |
| 257 | + ImmutableList.of( |
| 258 | + new org.apache.avro.Schema.Field( |
| 259 | + "data", |
| 260 | + org.apache.avro.Schema.create(org.apache.avro.Schema.Type.STRING), |
| 261 | + null, |
| 262 | + null), |
| 263 | + new org.apache.avro.Schema.Field( |
| 264 | + "id", |
| 265 | + org.apache.avro.Schema.create(org.apache.avro.Schema.Type.LONG), |
| 266 | + null, |
| 267 | + null), |
| 268 | + new org.apache.avro.Schema.Field("metadata", metadataSchema, null, null))); |
| 269 | + |
| 270 | + List<Map<String, Object>> recordData = |
| 271 | + ImmutableList.<Map<String, Object>>builder() |
| 272 | + .add( |
| 273 | + ImmutableMap.of( |
| 274 | + "id", |
| 275 | + 0L, |
| 276 | + "data", |
| 277 | + "clarification", |
| 278 | + "metadata", |
| 279 | + ImmutableMap.of("source", "systemA"))) |
| 280 | + .add( |
| 281 | + ImmutableMap.of( |
| 282 | + "id", 1L, "data", "risky", "metadata", ImmutableMap.of("source", "systemB"))) |
| 283 | + .add( |
| 284 | + ImmutableMap.of( |
| 285 | + "id", 2L, "data", "falafel", "metadata", ImmutableMap.of("source", "systemC"))) |
| 286 | + .build(); |
| 287 | + |
| 288 | + List<GenericRecord> avroRecords = |
| 289 | + recordData.stream() |
| 290 | + .map(data -> avroGenericRecord(avroSchema, data)) |
| 291 | + .collect(Collectors.toList()); |
| 292 | + |
| 293 | + Configuration hadoopConf = new Configuration(); |
| 294 | + String path = createParquetFile(avroSchema, avroRecords); |
| 295 | + HadoopInputFile inputFile = HadoopInputFile.fromLocation(path, hadoopConf); |
| 296 | + |
| 297 | + NameMapping defaultMapping = |
| 298 | + NameMapping.of( |
| 299 | + MappedField.of(1, "id"), |
| 300 | + MappedField.of(2, "data"), |
| 301 | + MappedField.of(3, "metadata", MappedFields.of(MappedField.of(4, "source")))); |
| 302 | + ImmutableMap<String, String> tableProperties = |
| 303 | + ImmutableMap.<String, String>builder() |
| 304 | + .put(TableProperties.DEFAULT_NAME_MAPPING, NameMappingParser.toJson(defaultMapping)) |
| 305 | + .build(); |
| 306 | + |
| 307 | + TableIdentifier tableId = |
| 308 | + TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); |
| 309 | + Table simpleTable = |
| 310 | + warehouse |
| 311 | + .buildTable(tableId, TestFixtures.NESTED_SCHEMA) |
| 312 | + .withProperties(tableProperties) |
| 313 | + .withPartitionSpec(PartitionSpec.unpartitioned()) |
| 314 | + .create(); |
| 315 | + |
| 316 | + MetricsConfig metricsConfig = MetricsConfig.forTable(simpleTable); |
| 317 | + Metrics metrics = ParquetUtil.fileMetrics(inputFile, metricsConfig); |
| 318 | + DataFile dataFile = |
| 319 | + DataFiles.builder(PartitionSpec.unpartitioned()) |
| 320 | + .withFormat(FileFormat.PARQUET) |
| 321 | + .withInputFile(inputFile) |
| 322 | + .withMetrics(metrics) |
| 323 | + .build(); |
| 324 | + |
| 325 | + final Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.NESTED_SCHEMA); |
| 326 | + |
| 327 | + simpleTable.newFastAppend().appendFile(dataFile).commit(); |
| 328 | + |
| 329 | + Map<String, String> catalogProps = |
| 330 | + ImmutableMap.<String, String>builder() |
| 331 | + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) |
| 332 | + .put("warehouse", warehouse.location) |
| 333 | + .build(); |
| 334 | + |
| 335 | + IcebergCatalogConfig catalogConfig = |
| 336 | + IcebergCatalogConfig.builder() |
| 337 | + .setCatalogName("name") |
| 338 | + .setCatalogProperties(catalogProps) |
| 339 | + .build(); |
| 340 | + |
| 341 | + PCollection<Row> output = |
| 342 | + testPipeline |
| 343 | + .apply(IcebergIO.readRows(catalogConfig).from(tableId)) |
| 344 | + .apply(ParDo.of(new PrintRow())) |
| 345 | + .setCoder(RowCoder.of(beamSchema)); |
| 346 | + |
| 347 | + final Row[] expectedRows = |
| 348 | + recordData.stream() |
| 349 | + .map(data -> icebergGenericRecord(TestFixtures.NESTED_SCHEMA.asStruct(), data)) |
| 350 | + .map(record -> IcebergUtils.icebergRecordToBeamRow(beamSchema, record)) |
| 351 | + .toArray(Row[]::new); |
| 352 | + |
| 353 | + PAssert.that(output) |
| 354 | + .satisfies( |
| 355 | + (Iterable<Row> rows) -> { |
| 356 | + assertThat(rows, containsInAnyOrder(expectedRows)); |
| 357 | + return null; |
| 358 | + }); |
| 359 | + |
| 360 | + testPipeline.run(); |
| 361 | + } |
| 362 | + |
| 363 | + @SuppressWarnings("unchecked") |
| 364 | + public static GenericRecord avroGenericRecord( |
| 365 | + org.apache.avro.Schema schema, Map<String, Object> values) { |
| 366 | + GenericRecord record = new GenericData.Record(schema); |
| 367 | + for (org.apache.avro.Schema.Field field : schema.getFields()) { |
| 368 | + Object rawValue = values.get(field.name()); |
| 369 | + Object avroValue = |
| 370 | + rawValue instanceof Map |
| 371 | + ? avroGenericRecord(field.schema(), (Map<String, Object>) rawValue) |
| 372 | + : rawValue; |
| 373 | + record.put(field.name(), avroValue); |
| 374 | + } |
| 375 | + return record; |
| 376 | + } |
| 377 | + |
| 378 | + @SuppressWarnings("unchecked") |
| 379 | + public static Record icebergGenericRecord(Types.StructType type, Map<String, Object> values) { |
| 380 | + org.apache.iceberg.data.GenericRecord record = |
| 381 | + org.apache.iceberg.data.GenericRecord.create(type); |
| 382 | + for (Types.NestedField field : type.fields()) { |
| 383 | + Object rawValue = values.get(field.name()); |
| 384 | + Object value = |
| 385 | + rawValue instanceof Map |
| 386 | + ? icebergGenericRecord(field.type().asStructType(), (Map<String, Object>) rawValue) |
| 387 | + : rawValue; |
| 388 | + record.setField(field.name(), value); |
| 389 | + } |
| 390 | + return record; |
| 391 | + } |
| 392 | + |
| 393 | + public static String createParquetFile(org.apache.avro.Schema schema, List<GenericRecord> records) |
| 394 | + throws IOException { |
| 395 | + |
| 396 | + File tempFile = createTempFile(); |
| 397 | + Path file = new Path(tempFile.getPath()); |
| 398 | + |
| 399 | + AvroParquetWriter.Builder<GenericRecord> builder = AvroParquetWriter.builder(file); |
| 400 | + ParquetWriter<GenericRecord> parquetWriter = builder.withSchema(schema).build(); |
| 401 | + for (GenericRecord record : records) { |
| 402 | + parquetWriter.write(record); |
| 403 | + } |
| 404 | + parquetWriter.close(); |
| 405 | + |
| 406 | + return tempFile.getPath(); |
| 407 | + } |
| 408 | + |
| 409 | + private static File createTempFile() throws IOException { |
| 410 | + File tempFile = File.createTempFile(ScanSourceTest.class.getSimpleName(), ".tmp"); |
| 411 | + tempFile.deleteOnExit(); |
| 412 | + boolean unused = tempFile.delete(); |
| 413 | + return tempFile; |
| 414 | + } |
213 | 415 | }
|
0 commit comments