beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Work logged] (BEAM-4461) Create a library of useful transforms that use schemas
Date Wed, 05 Sep 2018 16:39:00 GMT

     [ https://issues.apache.org/jira/browse/BEAM-4461?focusedWorklogId=141407&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-141407
]

ASF GitHub Bot logged work on BEAM-4461:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 05/Sep/18 16:38
            Start Date: 05/Sep/18 16:38
    Worklog Time Spent: 10m 
      Work Description: akedin commented on a change in pull request #6317: [BEAM-4461]  Add
mapping between FieldType and Java types.
URL: https://github.com/apache/beam/pull/6317#discussion_r215340974
 
 

 ##########
 File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldTypeDescriptors.java
 ##########
 @@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.collect.BiMap;
+import com.google.common.collect.ImmutableBiMap;
+import java.lang.reflect.ParameterizedType;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.Schema.TypeName;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.joda.time.Instant;
+/**
+ * Utilities for converting between {@link Schema} field types and {@link TypeDescriptor}s
that
+ * define Java objects which can represent these field types.
+ */
+public class FieldTypeDescriptors {
+  private static final BiMap<TypeName, TypeDescriptor> PRIMITIVE_MAPPING =
+      ImmutableBiMap.<TypeName, TypeDescriptor>builder()
+          .put(TypeName.BYTE, TypeDescriptors.bytes())
+          .put(TypeName.INT16, TypeDescriptors.shorts())
+          .put(TypeName.INT32, TypeDescriptors.integers())
+          .put(TypeName.INT64, TypeDescriptors.longs())
+          .put(TypeName.DECIMAL, TypeDescriptors.bigdecimals())
+          .put(TypeName.FLOAT, TypeDescriptors.floats())
+          .put(TypeName.DOUBLE, TypeDescriptors.doubles())
+          .put(TypeName.STRING, TypeDescriptors.strings())
+          .put(TypeName.DATETIME, TypeDescriptor.of(Instant.class))
+          .put(TypeName.BOOLEAN, TypeDescriptors.booleans())
+          .put(TypeName.BYTES, TypeDescriptor.of(byte[].class))
+          .build();
+  /** Get a {@link TypeDescriptor} from a {@link FieldType}. */
+  public static TypeDescriptor javaTypeForFieldType(FieldType fieldType) {
+    switch (fieldType.getTypeName()) {
+      case ARRAY:
+        return TypeDescriptors.lists(javaTypeForFieldType(fieldType.getCollectionElementType()));
+      case MAP:
+        return TypeDescriptors.maps(
+            javaTypeForFieldType(fieldType.getMapKeyType()),
+            javaTypeForFieldType(fieldType.getMapValueType()));
+      case ROW:
+        return TypeDescriptors.rows();
+      default:
+        return PRIMITIVE_MAPPING.get(fieldType.getTypeName());
+    }
+  }
+  /** Get a {@link FieldType} from a {@link TypeDescriptor}. */
+  public static FieldType fieldTypeForJavaType(TypeDescriptor typeDescriptor) {
+    if (typeDescriptor.isArray()
+        || typeDescriptor.isSubtypeOf(TypeDescriptor.of(Collection.class))) {
+      return getArrayFieldType(typeDescriptor);
+    } else if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(Map.class))) {
+      return getMapFieldType(typeDescriptor);
+    } else if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(Row.class))) {
+      throw new IllegalArgumentException(
+          "Cannot automatically determine a field type from a Row class"
+              + " as we cannot determine the schema. You should set a field type explicitly.");
+    } else {
+      TypeName typeName = PRIMITIVE_MAPPING.inverse().get(typeDescriptor);
+      if (typeName == null) {
+        throw new RuntimeException("Couldn't find field type for " + typeDescriptor);
+      }
+      FieldType fieldType = FieldType.of(typeName);
+      return fieldType;
+    }
+  }
+
+  private static FieldType getArrayFieldType(TypeDescriptor typeDescriptor) {
+    if (typeDescriptor.isArray()) {
+      if (typeDescriptor.getComponentType().getType().equals(byte.class)) {
+        return FieldType.BYTES;
+      } else {
+        return FieldType.array(fieldTypeForJavaType(typeDescriptor.getComponentType()));
 
 Review comment:
   Just curious, should `BYTES` and `Array of bytes` be different incompatible types, or can
one be an alias of another?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 141407)
    Time Spent: 4.5h  (was: 4h 20m)

> Create a library of useful transforms that use schemas
> ------------------------------------------------------
>
>                 Key: BEAM-4461
>                 URL: https://issues.apache.org/jira/browse/BEAM-4461
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-java-core
>            Reporter: Reuven Lax
>            Assignee: Reuven Lax
>            Priority: Major
>          Time Spent: 4.5h
>  Remaining Estimate: 0h
>
> e.g. JoinBy(fields). Project, Filter, etc.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message