You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/08/04 22:54:06 UTC

[GitHub] [beam] apilloud commented on a change in pull request #12348: [BEAM-10240] Support ZetaSQL DATETIME functions in BeamSQL

apilloud commented on a change in pull request #12348:
URL: https://github.com/apache/beam/pull/12348#discussion_r465355724



##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/DateTime.java
##########
@@ -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.getValue(DATE_FIELD_NAME)),

Review comment:
       1. Instead of `getValue()`, use `getInt64()` as we expect this to always be Int64.
   2. nit:`getInt64(DATE_FIELD_INDEX)` is probably a better choice than `getInt64(DATE_FIELD_NAME)` if there is a fixed schema.

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
##########
@@ -442,6 +455,20 @@ 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, "getValue", Expressions.constant(DateTime.DATE_FIELD_NAME));
+          Expression timeValue =
+              Expressions.call(value, "getValue", Expressions.constant(DateTime.TIME_FIELD_NAME));
+          Expression returnValue =
+              Expressions.add(
+                  Expressions.multiply(
+                      Types.castIfNecessary(long.class, dateValue),

Review comment:
       `getInt64` returns a `Long`, so I believe that will allow you to remove the `castIfNecessary`?

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
##########
@@ -850,6 +848,25 @@ private RexNode convertSimpleValueToRexNode(TypeKind kind, Value value) {
         // TODO: Doing micro to mills truncation, need to throw exception.
         ret = rexBuilder().makeLiteral(convertTimeValueToTimeString(value), timeType, false);
         break;
+      case TYPE_DATETIME:
+        // Cannot simply call makeTimestampWithLocalTimeZoneLiteral() for ZetaSQL DATETIME type
+        // because later it will be unparsed to the string representation of timestamp (e.g. "SELECT
+        // DATETIME '2008-12-25 15:30:00'" will be unparsed to "SELECT TIMESTAMP '2008-12-25
+        // 15:30:00:000000'"). So we create a wrapper function here such that we can later recognize
+        // it and customize its unparsing in BeamBigQuerySqlDialect.
+        ret =
+            rexBuilder()
+                .makeCall(
+                    SqlOperators.createZetaSqlFunction(
+                        BeamBigQuerySqlDialect.DATETIME_LITERAL_FUNCTION,

Review comment:
       Instead of wrapping with a function here, it looks like you can do the same thing by creating a class overloading `SqlTimestampLiteral.toString()` and using it in `BeamSqlUnparseContext`?
   https://github.com/apache/beam/blob/6fdde4f4eab72b49b10a8bb1cb3be263c5c416b5/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java#L70

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java
##########
@@ -253,6 +259,11 @@ private void unparseTrim(SqlWriter writer, SqlCall call, int leftPrec, int right
     writer.endFunCall(trimFrame);
   }
 
+  private void unparseDateTimeLiteralWrapperFunction(
+      SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
+    writer.literal(call.operand(0).toString().replace("TIMESTAMP", "DATETIME"));

Review comment:
       I'm not a fan of using `replace` here. I believe the `operand(0)` will be a `SqlTimestampLiteral`? If so, you can call `toFormattedString` instead: https://github.com/apache/calcite/blob/52a57078ba081b24b9d086ed363c715485d1a519/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java#L54




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org