Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,7 @@
import org.codehaus.janino.ScriptEvaluator;
import org.joda.time.DateTime;
import org.joda.time.Instant;
import org.joda.time.base.AbstractInstant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -632,8 +633,10 @@ private static Expression toCalciteValue(Expression value, FieldType fieldType)
case BOOLEAN:
return Expressions.convert_(value, Boolean.class);
case DATETIME:
// AbstractInstant handles both joda Instant and DateTime
return nullOr(
value, Expressions.call(Expressions.convert_(value, DateTime.class), "getMillis"));
value,
Expressions.call(Expressions.convert_(value, AbstractInstant.class), "getMillis"));
case BYTES:
return nullOr(
value, Expressions.new_(ByteString.class, Expressions.convert_(value, byte[].class)));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -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());
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);
Expand Down
Loading