This is an automated email from the ASF dual-hosted git repository.

robinyqiu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new 1988cb0  Simple code cleanup for ZetaSqlUtils
     new 42e45e2  Merge pull request #11988 from robinyqiu/cleanup
1988cb0 is described below

commit 1988cb02759a65aaa76107dc072924228b043b45
Author: Yueyang Qiu <[email protected]>
AuthorDate: Wed Jun 10 23:41:25 2020 -0700

    Simple code cleanup for ZetaSqlUtils
---
 .../sdk/extensions/sql/zetasql/ZetaSqlUtils.java   | 66 +++++++++-------------
 .../zetasql/translation/ExpressionConverter.java   |  2 +-
 .../extensions/sql/zetasql/ZetaSqlUtilsTest.java   | 62 ++++++++++----------
 3 files changed, 60 insertions(+), 70 deletions(-)

diff --git 
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtils.java
 
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtils.java
index 63c89cb..b041b18 100644
--- 
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtils.java
+++ 
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtils.java
@@ -44,7 +44,7 @@ import org.joda.time.Instant;
  * Utility methods for ZetaSQL related operations.
  *
  * <p>Unsupported ZetaSQL types: INT32, UINT32, UINT64, FLOAT, ENUM, PROTO, 
GEOGRAPHY
- * TODO[BEAM-8630]: support ZetaSQL types: DATE, TIME, DATETIME
+ * TODO[BEAM-10238]: support ZetaSQL types: TIME, DATETIME, NUMERIC
  */
 @Internal
 public final class ZetaSqlUtils {
@@ -54,27 +54,25 @@ public final class ZetaSqlUtils {
   private ZetaSqlUtils() {}
 
   // Type conversion: ZetaSQL => Calcite
-  public static SqlTypeName zetaSqlTypeToCalciteTypeName(TypeKind zetaSqlType) 
{
-    switch (zetaSqlType) {
+  public static SqlTypeName zetaSqlTypeToCalciteTypeName(Type type) {
+    switch (type.getKind()) {
       case TYPE_INT64:
         return SqlTypeName.BIGINT;
-      case TYPE_NUMERIC:
-        return SqlTypeName.DECIMAL;
       case TYPE_DOUBLE:
         return SqlTypeName.DOUBLE;
+      case TYPE_BOOL:
+        return SqlTypeName.BOOLEAN;
       case TYPE_STRING:
         return SqlTypeName.VARCHAR;
+      case TYPE_BYTES:
+        return SqlTypeName.VARBINARY;
       case TYPE_DATE:
         return SqlTypeName.DATE;
       case TYPE_TIMESTAMP:
         return SqlTypeName.TIMESTAMP;
-      case TYPE_BOOL:
-        return SqlTypeName.BOOLEAN;
-      case TYPE_BYTES:
-        return SqlTypeName.VARBINARY;
         // TODO[BEAM-9179] Add conversion code for ARRAY and ROW types
       default:
-        throw new UnsupportedOperationException("Unknown ZetaSQL type: " + 
zetaSqlType.name());
+        throw new UnsupportedOperationException("Unknown ZetaSQL type: " + 
type.getKind().name());
     }
   }
 
@@ -83,20 +81,17 @@ public final class ZetaSqlUtils {
     switch (fieldType.getTypeName()) {
       case INT64:
         return TypeFactory.createSimpleType(TypeKind.TYPE_INT64);
-      case DECIMAL:
-        return TypeFactory.createSimpleType(TypeKind.TYPE_NUMERIC);
       case DOUBLE:
         return TypeFactory.createSimpleType(TypeKind.TYPE_DOUBLE);
-      case STRING:
-        return TypeFactory.createSimpleType(TypeKind.TYPE_STRING);
-      case DATETIME:
-        // TODO[BEAM-8630]: Mapping Timestamp to DATETIME results in some 
timezone/precision issues.
-        //  Can we convert Timestamp to a LogicalType? Will it solve the 
problem?
-        return TypeFactory.createSimpleType(TypeKind.TYPE_TIMESTAMP);
       case BOOLEAN:
         return TypeFactory.createSimpleType(TypeKind.TYPE_BOOL);
+      case STRING:
+        return TypeFactory.createSimpleType(TypeKind.TYPE_STRING);
       case BYTES:
         return TypeFactory.createSimpleType(TypeKind.TYPE_BYTES);
+      case DATETIME:
+        // TODO[BEAM-10238]: Mapping TIMESTAMP to a Beam LogicalType instead?
+        return TypeFactory.createSimpleType(TypeKind.TYPE_TIMESTAMP);
       case ARRAY:
         return 
beamElementFieldTypeToZetaSqlArrayType(fieldType.getCollectionElementType());
       case ROW:
@@ -142,26 +137,23 @@ public final class ZetaSqlUtils {
     switch (fieldType.getTypeName()) {
       case INT64:
         return Value.createInt64Value((Long) object);
-        // TODO[BEAM-8630]: Value.createNumericValue() is broken due to a 
dependency issue
-        // case DECIMAL:
-        //   return Value.createNumericValue((BigDecimal) object);
       case DOUBLE:
         return Value.createDoubleValue((Double) object);
-      case STRING:
-        return Value.createStringValue((String) object);
-      case DATETIME:
-        return jodaInstantToZetaSqlTimestampValue((Instant) object);
       case BOOLEAN:
         return Value.createBoolValue((Boolean) object);
+      case STRING:
+        return Value.createStringValue((String) object);
       case BYTES:
         return Value.createBytesValue(ByteString.copyFrom((byte[]) object));
+      case DATETIME:
+        return jodaInstantToZetaSqlTimestampValue((Instant) object);
       case ARRAY:
         return javaListToZetaSqlArrayValue(
             (List<Object>) object, fieldType.getCollectionElementType());
       case ROW:
         return beamRowToZetaSqlStructValue((Row) object, 
fieldType.getRowSchema());
       case LOGICAL_TYPE:
-        return 
beamLogicalObjectToZetaSqlValue(fieldType.getLogicalType().getIdentifier(), 
object);
+        return beamLogicalObjectToZetaSqlValue(object, 
fieldType.getLogicalType().getIdentifier());
       default:
         throw new UnsupportedOperationException(
             "Unknown Beam fieldType: " + fieldType.getTypeName());
@@ -169,12 +161,8 @@ public final class ZetaSqlUtils {
   }
 
   private static Value jodaInstantToZetaSqlTimestampValue(Instant instant) {
-    return javaLongToZetaSqlTimestampValue(instant.getMillis());
-  }
-
-  private static Value javaLongToZetaSqlTimestampValue(Long millis) {
     return Value.createTimestampValueFromUnixMicros(
-        LongMath.checkedMultiply(millis, MICROS_PER_MILLI));
+        LongMath.checkedMultiply(instant.getMillis(), MICROS_PER_MILLI));
   }
 
   private static Value javaListToZetaSqlArrayValue(List<Object> elements, 
FieldType elementType) {
@@ -196,7 +184,7 @@ public final class ZetaSqlUtils {
     return Value.createStructValue(beamSchemaToZetaSqlStructType(schema), 
values);
   }
 
-  private static Value beamLogicalObjectToZetaSqlValue(String identifier, 
Object object) {
+  private static Value beamLogicalObjectToZetaSqlValue(Object object, String 
identifier) {
     if (SqlTypes.DATE.getIdentifier().equals(identifier)) {
       if (object instanceof Long) { // base type
         return Value.createDateValue(((Long) object).intValue());
@@ -217,8 +205,6 @@ public final class ZetaSqlUtils {
     switch (fieldType.getTypeName()) {
       case INT64:
         return value.getInt64Value();
-      case DECIMAL:
-        return value.getNumericValue();
       case DOUBLE:
         // Floats with a floating part equal to zero are treated as whole 
(INT64).
         // Cast to double when that happens.
@@ -226,21 +212,21 @@ public final class ZetaSqlUtils {
           return (double) value.getInt64Value();
         }
         return value.getDoubleValue();
-      case STRING:
-        return value.getStringValue();
-      case DATETIME:
-        return zetaSqlTimestampValueToJodaInstant(value);
       case BOOLEAN:
         return value.getBoolValue();
+      case STRING:
+        return value.getStringValue();
       case BYTES:
         return value.getBytesValue().toByteArray();
+      case DATETIME:
+        return zetaSqlTimestampValueToJodaInstant(value);
       case ARRAY:
         return zetaSqlArrayValueToJavaList(
             value, fieldType.getCollectionElementType(), verifyValues);
       case ROW:
         return zetaSqlStructValueToBeamRow(value, fieldType.getRowSchema(), 
verifyValues);
       case LOGICAL_TYPE:
-        return 
zetaSqlValueToBeamLogicalObject(fieldType.getLogicalType().getIdentifier(), 
value);
+        return zetaSqlValueToBeamLogicalObject(value, 
fieldType.getLogicalType().getIdentifier());
       default:
         throw new UnsupportedOperationException(
             "Unknown Beam fieldType: " + fieldType.getTypeName());
@@ -274,7 +260,7 @@ public final class ZetaSqlUtils {
     return row;
   }
 
-  private static Object zetaSqlValueToBeamLogicalObject(String identifier, 
Value value) {
+  private static Object zetaSqlValueToBeamLogicalObject(Value value, String 
identifier) {
     if (SqlTypes.DATE.getIdentifier().equals(identifier)) {
       return LocalDate.ofEpochDay(value.getDateValue());
     } else {
diff --git 
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
 
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
index 87eb8ea7..ff49674 100644
--- 
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
+++ 
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
@@ -862,7 +862,7 @@ public class ExpressionConverter {
         if (returnType != null) {
           op =
               SqlOperators.createSimpleSqlFunction(
-                  funName, 
ZetaSqlUtils.zetaSqlTypeToCalciteTypeName(returnType.getKind()));
+                  funName, 
ZetaSqlUtils.zetaSqlTypeToCalciteTypeName(returnType));
         } else {
           throw new UnsupportedOperationException("Does not support ZetaSQL 
function: " + funName);
         }
diff --git 
a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtilsTest.java
 
b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtilsTest.java
index dbe7c18..f41d771 100644
--- 
a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtilsTest.java
+++ 
b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtilsTest.java
@@ -26,9 +26,11 @@ import com.google.zetasql.StructType.StructField;
 import com.google.zetasql.TypeFactory;
 import com.google.zetasql.Value;
 import com.google.zetasql.ZetaSQLType.TypeKind;
+import java.time.LocalDate;
 import java.util.Arrays;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes;
 import org.apache.beam.sdk.values.Row;
 import org.joda.time.Instant;
 import org.junit.Test;
@@ -44,16 +46,19 @@ public class ZetaSqlUtilsTest {
 
   private static final Schema TEST_SCHEMA =
       Schema.builder()
-          .addField("f1", FieldType.INT64)
-          // .addField("f2", FieldType.DECIMAL)
-          .addField("f3", FieldType.DOUBLE)
-          .addField("f4", FieldType.STRING)
-          .addField("f5", FieldType.DATETIME)
-          .addField("f6", FieldType.BOOLEAN)
-          .addField("f7", FieldType.BYTES)
-          .addArrayField("f8", FieldType.DOUBLE)
-          .addRowField("f9", TEST_INNER_SCHEMA)
-          .addNullableField("f10", FieldType.INT64)
+          .addField("f_int64", FieldType.INT64)
+          .addField("f_float64", FieldType.DOUBLE)
+          .addField("f_boolean", FieldType.BOOLEAN)
+          .addField("f_string", FieldType.STRING)
+          .addField("f_bytes", FieldType.BYTES)
+          .addLogicalTypeField("f_date", SqlTypes.DATE)
+          // .addLogicalTypeField("f_datetime", SqlTypes.DATETIME)
+          // .addLogicalTypeField("f_time", SqlTypes.TIME)
+          .addField("f_timestamp", FieldType.DATETIME)
+          .addArrayField("f_array", FieldType.DOUBLE)
+          .addRowField("f_struct", TEST_INNER_SCHEMA)
+          // .addLogicalTypeField("f_numeric", SqlTypes.NUMERIC)
+          .addNullableField("f_null", FieldType.INT64)
           .build();
 
   private static final FieldType TEST_FIELD_TYPE = FieldType.row(TEST_SCHEMA);
@@ -70,26 +75,26 @@ public class ZetaSqlUtilsTest {
   private static final StructType TEST_TYPE =
       TypeFactory.createStructType(
           Arrays.asList(
-              new StructField("f1", 
TypeFactory.createSimpleType(TypeKind.TYPE_INT64)),
-              // new StructField("f2", 
TypeFactory.createSimpleType(TypeKind.TYPE_NUMERIC)),
-              new StructField("f3", 
TypeFactory.createSimpleType(TypeKind.TYPE_DOUBLE)),
-              new StructField("f4", 
TypeFactory.createSimpleType(TypeKind.TYPE_STRING)),
-              new StructField("f5", 
TypeFactory.createSimpleType(TypeKind.TYPE_TIMESTAMP)),
-              new StructField("f6", 
TypeFactory.createSimpleType(TypeKind.TYPE_BOOL)),
-              new StructField("f7", 
TypeFactory.createSimpleType(TypeKind.TYPE_BYTES)),
-              new StructField("f8", TEST_INNER_ARRAY_TYPE),
-              new StructField("f9", TEST_INNER_STRUCT_TYPE),
-              new StructField("f10", 
TypeFactory.createSimpleType(TypeKind.TYPE_INT64))));
+              new StructField("f_int64", 
TypeFactory.createSimpleType(TypeKind.TYPE_INT64)),
+              new StructField("f_float64", 
TypeFactory.createSimpleType(TypeKind.TYPE_DOUBLE)),
+              new StructField("f_boolean", 
TypeFactory.createSimpleType(TypeKind.TYPE_BOOL)),
+              new StructField("f_string", 
TypeFactory.createSimpleType(TypeKind.TYPE_STRING)),
+              new StructField("f_bytes", 
TypeFactory.createSimpleType(TypeKind.TYPE_BYTES)),
+              new StructField("f_date", 
TypeFactory.createSimpleType(TypeKind.TYPE_DATE)),
+              new StructField("f_timestamp", 
TypeFactory.createSimpleType(TypeKind.TYPE_TIMESTAMP)),
+              new StructField("f_array", TEST_INNER_ARRAY_TYPE),
+              new StructField("f_struct", TEST_INNER_STRUCT_TYPE),
+              new StructField("f_null", 
TypeFactory.createSimpleType(TypeKind.TYPE_INT64))));
 
   private static final Row TEST_ROW =
       Row.withSchema(TEST_SCHEMA)
           .addValue(64L)
-          // .addValue(BigDecimal.valueOf(9999L))
-          .addValue(5.0)
-          .addValue("Hello")
-          .addValue(Instant.ofEpochMilli(12345678L))
+          .addValue(5.15)
           .addValue(false)
+          .addValue("Hello")
           .addValue(new byte[] {0x11, 0x22})
+          .addValue(LocalDate.of(2020, 6, 4))
+          .addValue(Instant.ofEpochMilli(12345678L))
           .addArray(3.0, 6.5)
           .addValue(Row.withSchema(TEST_INNER_SCHEMA).addValues(0L, 
"world").build())
           .addValue(null)
@@ -100,13 +105,12 @@ public class ZetaSqlUtilsTest {
           TEST_TYPE,
           Arrays.asList(
               Value.createInt64Value(64L),
-              // TODO[BEAM-8630]: Value.createNumericValue() is broken due to 
a dependency issue
-              // Value.createNumericValue(BigDecimal.valueOf(9999L)),
-              Value.createDoubleValue(5.0),
-              Value.createStringValue("Hello"),
-              Value.createTimestampValueFromUnixMicros(12345678000L),
+              Value.createDoubleValue(5.15),
               Value.createBoolValue(false),
+              Value.createStringValue("Hello"),
               Value.createBytesValue(ByteString.copyFrom(new byte[] {0x11, 
0x22})),
+              Value.createDateValue((int) LocalDate.of(2020, 6, 
4).toEpochDay()),
+              Value.createTimestampValueFromUnixMicros(12345678000L),
               Value.createArrayValue(
                   TEST_INNER_ARRAY_TYPE,
                   Arrays.asList(Value.createDoubleValue(3.0), 
Value.createDoubleValue(6.5))),

Reply via email to