|
23 | 23 | import java.time.LocalTime; |
24 | 24 | import java.util.Arrays; |
25 | 25 | import java.util.HashMap; |
| 26 | +import java.util.List; |
26 | 27 | import java.util.Map; |
27 | 28 | import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; |
28 | 29 | import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; |
|
43 | 44 | import org.apache.beam.sdk.values.Row; |
44 | 45 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; |
45 | 46 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; |
| 47 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; |
46 | 48 | import org.joda.time.Duration; |
47 | 49 | import org.joda.time.Instant; |
48 | 50 | import org.junit.Ignore; |
@@ -349,6 +351,44 @@ public void testNestedArrayOfBytes() { |
349 | 351 | pipeline.run(); |
350 | 352 | } |
351 | 353 |
|
| 354 | + @Test |
| 355 | + public void testNestedDatetime() { |
| 356 | + List<Instant> dateTimes = |
| 357 | + ImmutableList.of(Instant.EPOCH, Instant.ofEpochSecond(10000), Instant.now()); |
| 358 | + List<Instant> nullDateTimes = Lists.newArrayList(Instant.EPOCH, null, Instant.now()); |
| 359 | + |
| 360 | + Schema nestedInputSchema = |
| 361 | + Schema.of( |
| 362 | + Schema.Field.of("c_dts", Schema.FieldType.array(Schema.FieldType.DATETIME)), |
| 363 | + Schema.Field.of( |
| 364 | + "c_null_dts", |
| 365 | + Schema.FieldType.array(Schema.FieldType.DATETIME.withNullable(true)))); |
| 366 | + Schema inputSchema = |
| 367 | + Schema.of(Schema.Field.of("nested", Schema.FieldType.row(nestedInputSchema))); |
| 368 | + |
| 369 | + Schema outputSchema = |
| 370 | + Schema.of( |
| 371 | + Schema.Field.of("f0", Schema.FieldType.DATETIME), |
| 372 | + Schema.Field.of("f1", Schema.FieldType.DATETIME.withNullable(true))); |
| 373 | + |
| 374 | + Row nestedRow = |
| 375 | + Row.withSchema(nestedInputSchema).addValue(dateTimes).addValue(nullDateTimes).build(); |
| 376 | + Row row = Row.withSchema(inputSchema).addValue(nestedRow).build(); |
| 377 | + Row expected = |
| 378 | + Row.withSchema(outputSchema).addValues(dateTimes.get(1), nullDateTimes.get(1)).build(); |
| 379 | + |
| 380 | + PCollection<Row> result = |
| 381 | + pipeline |
| 382 | + .apply(Create.of(row).withRowSchema(inputSchema)) |
| 383 | + .apply( |
| 384 | + SqlTransform.query( |
| 385 | + "SELECT t.nested.c_dts[2], t.nested.c_null_dts[2] AS f0 FROM PCOLLECTION t")); |
| 386 | + |
| 387 | + PAssert.that(result).containsInAnyOrder(expected); |
| 388 | + |
| 389 | + pipeline.run(); |
| 390 | + } |
| 391 | + |
352 | 392 | @Test |
353 | 393 | public void testRowConstructor() { |
354 | 394 | BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(readOnlyTableProvider); |
|
0 commit comments