-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Fix BeamSQL CalcRel DATETIME expression #35743
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,6 +23,7 @@ | |
| import java.time.LocalTime; | ||
| import java.util.Arrays; | ||
| import java.util.HashMap; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; | ||
| import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; | ||
|
|
@@ -43,6 +44,7 @@ | |
| import org.apache.beam.sdk.values.Row; | ||
| import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; | ||
| import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; | ||
| import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; | ||
| import org.joda.time.Duration; | ||
| import org.joda.time.Instant; | ||
| import org.junit.Ignore; | ||
|
|
@@ -349,6 +351,44 @@ public void testNestedArrayOfBytes() { | |
| pipeline.run(); | ||
| } | ||
|
|
||
| @Test | ||
| public void testNestedDatetime() { | ||
| List<Instant> dateTimes = | ||
| ImmutableList.of(Instant.EPOCH, Instant.ofEpochSecond(10000), Instant.now()); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added nullable array field to exercise this case |
||
| List<Instant> nullDateTimes = Lists.newArrayList(Instant.EPOCH, null, Instant.now()); | ||
|
|
||
| Schema nestedInputSchema = | ||
| Schema.of( | ||
| Schema.Field.of("c_dts", Schema.FieldType.array(Schema.FieldType.DATETIME)), | ||
| Schema.Field.of( | ||
| "c_null_dts", | ||
| Schema.FieldType.array(Schema.FieldType.DATETIME.withNullable(true)))); | ||
| Schema inputSchema = | ||
| Schema.of(Schema.Field.of("nested", Schema.FieldType.row(nestedInputSchema))); | ||
|
|
||
| Schema outputSchema = | ||
| Schema.of( | ||
| Schema.Field.of("f0", Schema.FieldType.DATETIME), | ||
| Schema.Field.of("f1", Schema.FieldType.DATETIME.withNullable(true))); | ||
|
|
||
| Row nestedRow = | ||
| Row.withSchema(nestedInputSchema).addValue(dateTimes).addValue(nullDateTimes).build(); | ||
| Row row = Row.withSchema(inputSchema).addValue(nestedRow).build(); | ||
| Row expected = | ||
| Row.withSchema(outputSchema).addValues(dateTimes.get(1), nullDateTimes.get(1)).build(); | ||
|
|
||
| PCollection<Row> result = | ||
| pipeline | ||
| .apply(Create.of(row).withRowSchema(inputSchema)) | ||
| .apply( | ||
| SqlTransform.query( | ||
| "SELECT t.nested.c_dts[2], t.nested.c_null_dts[2] AS f0 FROM PCOLLECTION t")); | ||
|
|
||
| PAssert.that(result).containsInAnyOrder(expected); | ||
|
|
||
| pipeline.run(); | ||
| } | ||
|
|
||
| @Test | ||
| public void testRowConstructor() { | ||
| BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(readOnlyTableProvider); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Using
Instant.now()in test data can lead to non-deterministic tests if the test logic changes in the future. It's better to use a fixed instant for reproducibility, even if this value is not directly used in the assertion.