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
@@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.schemas.logicaltypes;

import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.values.Row;

/**
* A datetime without a time-zone.
*
* <p>It cannot represent an instant on the time-line without additional information such as an
* offset or time-zone.
*
* <p>Its input type is a {@link LocalDateTime}, and base type is a {@link Row} containing Date
* field and Time field. Date field is the same as the base type of {@link Date}, which is a Long
* that represents incrementing count of days where day 0 is 1970-01-01 (ISO). Time field is the
* same as the base type of {@link Time}, which is a Long that represents a count of time in
* nanoseconds.
*/
public class DateTime implements Schema.LogicalType<LocalDateTime, Row> {
public static final String DATE_FIELD_NAME = "Date";
public static final String TIME_FIELD_NAME = "Time";
public static final Schema DATETIME_SCHEMA =
Schema.builder().addInt64Field(DATE_FIELD_NAME).addInt64Field(TIME_FIELD_NAME).build();

@Override
public String getIdentifier() {
return "beam:logical_type:datetime:v1";
}

// unused
@Override
public Schema.FieldType getArgumentType() {
return Schema.FieldType.STRING;
}

// unused
@Override
public String getArgument() {
return "";
}

@Override
public Schema.FieldType getBaseType() {
return Schema.FieldType.row(DATETIME_SCHEMA);
}

@Override
public Row toBaseType(LocalDateTime input) {
return input == null
? null
: Row.withSchema(DATETIME_SCHEMA)
.addValues(input.toLocalDate().toEpochDay(), input.toLocalTime().toNanoOfDay())
.build();
}

@Override
public LocalDateTime toInputType(Row base) {
return base == null
? null
: LocalDateTime.of(
LocalDate.ofEpochDay(base.getInt64(DATE_FIELD_NAME)),
LocalTime.ofNanoOfDay(base.getInt64(TIME_FIELD_NAME)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,10 @@
package org.apache.beam.sdk.schemas.logicaltypes;

import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import org.apache.beam.sdk.schemas.Schema.LogicalType;
import org.apache.beam.sdk.values.Row;

/** Beam {@link org.apache.beam.sdk.schemas.Schema.LogicalType}s corresponding to SQL data types. */
public class SqlTypes {
Expand All @@ -31,4 +33,7 @@ private SqlTypes() {}

/** Beam LogicalType corresponding to ZetaSQL/CalciteSQL TIME type. */
public static final LogicalType<LocalTime, Long> TIME = new Time();

/** Beam LogicalType corresponding to ZetaSQL DATETIME type. */
public static final LogicalType<LocalDateTime, Row> DATETIME = new DateTime();
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ class SchemaUtils {
.put("BOOL", FieldType.BOOLEAN)
.put("BYTES", FieldType.BYTES)
.put("DATE", FieldType.logicalType(SqlTypes.DATE))
.put("DATETIME", FieldType.DATETIME)
.put("DATETIME", FieldType.logicalType(SqlTypes.DATETIME))
.put("DOUBLE", FieldType.DOUBLE)
.put("FLOAT", FieldType.DOUBLE)
.put("FLOAT64", FieldType.DOUBLE)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,8 @@ public int getMaxPrecision(SqlTypeName typeName) {
switch (typeName) {
case TIME:
return 6; // support microsecond time precision
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
return 6; // support microsecond datetime precision
default:
return super.getMaxPrecision(typeName);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import java.lang.reflect.Type;
import java.math.BigDecimal;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.util.AbstractList;
import java.util.AbstractMap;
Expand All @@ -35,13 +36,14 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TimeZone;
import org.apache.beam.sdk.extensions.sql.impl.BeamSqlPipelineOptions;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamJavaTypeFactory;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils.CharType;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils.TimeWithLocalTzType;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils.TimestampWithLocalTzType;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.logicaltypes.DateTime;
import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
Expand Down Expand Up @@ -92,6 +94,7 @@
public class BeamCalcRel extends AbstractBeamCalcRel {

private static final long NANOS_PER_MILLISECOND = 1000000L;
private static final long MILLIS_PER_DAY = 86400000L;

private static final ParameterExpression outputSchemaParam =
Expressions.parameter(Schema.class, "outputSchema");
Expand Down Expand Up @@ -344,6 +347,18 @@ private static Expression castOutputTime(Expression value, FieldType toType) {
valueDateTime = Expressions.unbox(valueDateTime);
}
valueDateTime = Expressions.call(LocalDate.class, "ofEpochDay", valueDateTime);
} else if (CalciteUtils.TIMESTAMP_WITH_LOCAL_TZ.typesEqual(toType)
|| CalciteUtils.NULLABLE_TIMESTAMP_WITH_LOCAL_TZ.typesEqual(toType)) {
// Convert TimeStamp_With_Local_TimeZone to LocalDateTime
Expression dateValue =
Expressions.divide(valueDateTime, Expressions.constant(MILLIS_PER_DAY));
Expression date = Expressions.call(LocalDate.class, "ofEpochDay", dateValue);
Expression timeValue =
Expressions.multiply(
Expressions.modulo(valueDateTime, Expressions.constant(MILLIS_PER_DAY)),
Expressions.constant(NANOS_PER_MILLISECOND));
Expression time = Expressions.call(LocalTime.class, "ofNanoOfDay", timeValue);
valueDateTime = Expressions.call(LocalDateTime.class, "of", date, time);
} else {
throw new UnsupportedOperationException("Unknown DateTime type " + toType);
}
Expand Down Expand Up @@ -385,7 +400,7 @@ private static class InputGetterImpl implements RexToLixTranslator.InputGetter {
.put(SqlTypes.DATE.getIdentifier(), Long.class)
.put(SqlTypes.TIME.getIdentifier(), Long.class)
.put(TimeWithLocalTzType.IDENTIFIER, ReadableInstant.class)
.put(TimestampWithLocalTzType.IDENTIFIER, ReadableInstant.class)
.put(SqlTypes.DATETIME.getIdentifier(), Row.class)
.put(CharType.IDENTIFIER, String.class)
.build();

Expand Down Expand Up @@ -442,6 +457,16 @@ private static Expression value(Expression value, Schema.FieldType type) {
value, Expressions.divide(value, Expressions.constant(NANOS_PER_MILLISECOND)));
} else if (SqlTypes.DATE.getIdentifier().equals(logicalId)) {
return value;
} else if (SqlTypes.DATETIME.getIdentifier().equals(logicalId)) {
Expression dateValue =
Expressions.call(value, "getInt64", Expressions.constant(DateTime.DATE_FIELD_NAME));
Expression timeValue =
Expressions.call(value, "getInt64", Expressions.constant(DateTime.TIME_FIELD_NAME));
Expression returnValue =
Expressions.add(
Expressions.multiply(dateValue, Expressions.constant(MILLIS_PER_DAY)),
Expressions.divide(timeValue, Expressions.constant(NANOS_PER_MILLISECOND)));
return nullOr(value, returnValue);
} else if (!CharType.IDENTIFIER.equals(logicalId)) {
throw new UnsupportedOperationException(
"Unknown LogicalType " + type.getLogicalType().getIdentifier());
Expand Down Expand Up @@ -563,6 +588,8 @@ public Object get(String name) {
|| name.equals(DataContext.Variable.CURRENT_TIMESTAMP.camelName)
|| name.equals(DataContext.Variable.LOCAL_TIMESTAMP.camelName)) {
return System.currentTimeMillis();
} else if (name.equals(Variable.TIME_ZONE.camelName)) {
return TimeZone.getDefault();
}
return null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,15 +53,6 @@ public TimeWithLocalTzType() {
}
}

/** A LogicalType corresponding to TIMESTAMP_WITH_LOCAL_TIME_ZONE. */
public static class TimestampWithLocalTzType extends PassThroughLogicalType<Instant> {
public static final String IDENTIFIER = "SqlTimestampWithLocalTzType";

public TimestampWithLocalTzType() {
super(IDENTIFIER, FieldType.STRING, "", FieldType.DATETIME);
}
}

/** A LogicalType corresponding to CHAR. */
public static class CharType extends PassThroughLogicalType<String> {
public static final String IDENTIFIER = "SqlCharType";
Expand All @@ -82,7 +73,7 @@ public static boolean isDateTimeType(FieldType fieldType) {
return logicalId.equals(SqlTypes.DATE.getIdentifier())
|| logicalId.equals(SqlTypes.TIME.getIdentifier())
|| logicalId.equals(TimeWithLocalTzType.IDENTIFIER)
|| logicalId.equals(TimestampWithLocalTzType.IDENTIFIER);
|| logicalId.equals(SqlTypes.DATETIME.getIdentifier());
}
return false;
}
Expand Down Expand Up @@ -121,8 +112,9 @@ public static boolean isStringType(FieldType fieldType) {
FieldType.logicalType(new TimeWithLocalTzType());
public static final FieldType TIMESTAMP = FieldType.DATETIME;
public static final FieldType NULLABLE_TIMESTAMP = FieldType.DATETIME.withNullable(true);
public static final FieldType TIMESTAMP_WITH_LOCAL_TZ =
FieldType.logicalType(new TimestampWithLocalTzType());
public static final FieldType TIMESTAMP_WITH_LOCAL_TZ = FieldType.logicalType(SqlTypes.DATETIME);
public static final FieldType NULLABLE_TIMESTAMP_WITH_LOCAL_TZ =
FieldType.logicalType(SqlTypes.DATETIME).withNullable(true);

private static final BiMap<FieldType, SqlTypeName> BEAM_TO_CALCITE_TYPE_MAPPING =
ImmutableBiMap.<FieldType, SqlTypeName>builder()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.BitString;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.TimestampString;
import org.checkerframework.checker.nullness.qual.Nullable;

public class BeamSqlUnparseContext extends SqlImplementor.SimpleContext {
Expand Down Expand Up @@ -69,8 +70,12 @@ public BeamSqlUnparseContext(IntFunction<SqlNode> field) {
public SqlNode toSql(RexProgram program, RexNode rex) {
if (rex.getKind().equals(SqlKind.LITERAL)) {
final RexLiteral literal = (RexLiteral) rex;
SqlTypeFamily family = literal.getTypeName().getFamily();
if (SqlTypeFamily.BINARY.equals(family)) {
SqlTypeName name = literal.getTypeName();
SqlTypeFamily family = name.getFamily();
if (SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE.equals(name)) {
TimestampString timestampString = literal.getValueAs(TimestampString.class);
return new SqlDateTimeLiteral(timestampString, POS);
} else if (SqlTypeFamily.BINARY.equals(family)) {
ByteString byteString = literal.getValueAs(ByteString.class);
BitString bitString = BitString.createFromHexString(byteString.toString(16));
return new SqlByteStringLiteral(bitString, POS);
Expand All @@ -92,6 +97,21 @@ public SqlNode toSql(RexProgram program, RexNode rex) {
return super.toSql(program, rex);
}

private static class SqlDateTimeLiteral extends SqlLiteral {

private final TimestampString timestampString;

SqlDateTimeLiteral(TimestampString timestampString, SqlParserPos pos) {
super(timestampString, SqlTypeName.TIMESTAMP, pos);
this.timestampString = timestampString;
}

@Override
public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
writer.literal("DATETIME '" + timestampString.toString() + "'");
}
}

private static class SqlByteStringLiteral extends SqlLiteral {

SqlByteStringLiteral(BitString bytes, SqlParserPos pos) {
Expand Down
Loading