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/04/30 19:09:01 UTC

[GitHub] [beam] apilloud commented on a change in pull request #11272: [BEAM-9641] Support ZetaSQL DATE type as a Beam LogicalType

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



##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
##########
@@ -303,8 +303,8 @@ private static Object fieldToAvatica(Schema.FieldType type, Object beamValue) {
         String logicalId = type.getLogicalType().getIdentifier();
         if (logicalId.equals(TimeType.IDENTIFIER)) {
           return (int) ((ReadableInstant) beamValue).getMillis();
-        } else if (logicalId.equals(DateType.IDENTIFIER)) {
-          return (int) (((ReadableInstant) beamValue).getMillis() / MILLIS_PER_DAY);
+        } else if (logicalId.equals(SqlTypes.DATE.getIdentifier())) {

Review comment:
       nit: Call `equals` on constant to avoid null issues.

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/Date.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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 org.apache.beam.sdk.schemas.Schema;
+
+/**
+ * A date without a time-zone.
+ *
+ * <p>It cannot represent an instant on the time-line without additional information such as an
+ * offset or time-zone.
+ */
+public class Date implements Schema.LogicalType<LocalDate, Long> {

Review comment:
       If I'm reading the correctly, `LocalDate` is the in memory type (a struct) and `Long` is the wire format (an offset from epoch)? This conversion could be quite expensive. It appears the Calc nodes both take an offset in this case, when we start to think about performance we might need to change the in memory type to be offset based.

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
##########
@@ -427,17 +430,12 @@ private static Expression value(
 
     private static Expression value(Expression value, Schema.FieldType type) {
       if (type.getTypeName().isLogicalType()) {
-        Expression millisField = Expressions.call(value, "getMillis");
         String logicalId = type.getLogicalType().getIdentifier();
         if (logicalId.equals(TimeType.IDENTIFIER)) {
-          return nullOr(value, Expressions.convert_(millisField, int.class));
-        } else if (logicalId.equals(DateType.IDENTIFIER)) {
-          value =
-              nullOr(
-                  value,
-                  Expressions.convert_(
-                      Expressions.divide(millisField, Expressions.constant(MILLIS_PER_DAY)),
-                      int.class));
+          return nullOr(
+              value, Expressions.convert_(Expressions.call(value, "getMillis"), int.class));
+        } else if (logicalId.equals(SqlTypes.DATE.getIdentifier())) {

Review comment:
       nit: Can you clean up the style and call `equals` on the constant instead of `logicalId` (which could be null).

##########
File path: sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtils.java
##########
@@ -128,6 +165,17 @@ private static StructField beamFieldToZetaSqlStructField(Field field) {
     return new StructField(field.getName(), beamFieldTypeToZetaSqlType(field.getType()));
   }
 
+  private static Type beamLogicalTypeToZetaSqlType(String identifier) {
+    if (SqlTypes.DATE.getIdentifier().equals(identifier)) {

Review comment:
       nit: This order of `equals` is awesome!

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/Date.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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 org.apache.beam.sdk.schemas.Schema;
+
+/**
+ * A date without a time-zone.
+ *
+ * <p>It cannot represent an instant on the time-line without additional information such as an

Review comment:
       I think it is worth documenting that the `Long` is an offset from an epoch (and what that epoch is).




----------------------------------------------------------------
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