[ https://issues.apache.org/jira/browse/BEAM-3437?focusedWorklogId=86470&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-86470 ]
ASF GitHub Bot logged work on BEAM-3437: ---------------------------------------- Author: ASF GitHub Bot Created on: 02/Apr/18 05:08 Start Date: 02/Apr/18 05:08 Worklog Time Spent: 10m Work Description: reuvenlax commented on a change in pull request #4964: [BEAM-3437] Introduce Schema class, and use it in BeamSQL URL: https://github.com/apache/beam/pull/4964#discussion_r178487899 ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java ########## @@ -38,149 +37,152 @@ */ public class CalciteUtils { private static final long UNLIMITED_ARRAY_SIZE = -1L; - private static final BiMap<SqlTypeCoder, SqlTypeName> BEAM_TO_CALCITE_TYPE_MAPPING = - ImmutableBiMap.<SqlTypeCoder, SqlTypeName>builder() - .put(SqlTypeCoders.TINYINT, SqlTypeName.TINYINT) - .put(SqlTypeCoders.SMALLINT, SqlTypeName.SMALLINT) - .put(SqlTypeCoders.INTEGER, SqlTypeName.INTEGER) - .put(SqlTypeCoders.BIGINT, SqlTypeName.BIGINT) + private static final BiMap<Schema.FieldType, SqlTypeName> BEAM_TO_CALCITE_TYPE_MAPPING = + ImmutableBiMap.<Schema.FieldType, SqlTypeName>builder() + .put(FieldType.BYTE, SqlTypeName.TINYINT) + .put(FieldType.INT16, SqlTypeName.SMALLINT) + .put(FieldType.INT32, SqlTypeName.INTEGER) + .put(FieldType.INT64, SqlTypeName.BIGINT) - .put(SqlTypeCoders.FLOAT, SqlTypeName.FLOAT) - .put(SqlTypeCoders.DOUBLE, SqlTypeName.DOUBLE) + .put(FieldType.FLOAT, SqlTypeName.FLOAT) + .put(FieldType.DOUBLE, SqlTypeName.DOUBLE) - .put(SqlTypeCoders.DECIMAL, SqlTypeName.DECIMAL) + .put(FieldType.DECIMAL, SqlTypeName.DECIMAL) - .put(SqlTypeCoders.CHAR, SqlTypeName.CHAR) - .put(SqlTypeCoders.VARCHAR, SqlTypeName.VARCHAR) + .put(FieldType.STRING, SqlTypeName.VARCHAR) - .put(SqlTypeCoders.DATE, SqlTypeName.DATE) - .put(SqlTypeCoders.TIME, SqlTypeName.TIME) - .put(SqlTypeCoders.TIMESTAMP, SqlTypeName.TIMESTAMP) + .put(FieldType.DATETIME, SqlTypeName.TIMESTAMP) - .put(SqlTypeCoders.BOOLEAN, SqlTypeName.BOOLEAN) - .build(); + .put(FieldType.BOOLEAN, SqlTypeName.BOOLEAN) - private static final BiMap<SqlTypeName, SqlTypeCoder> CALCITE_TO_BEAM_TYPE_MAPPING = + .put(FieldType.ARRAY, SqlTypeName.ARRAY) + .put(FieldType.ROW, SqlTypeName.ROW) Review comment: <!--thread_id:cc_178369253_t; commit:79c95678e593da730ba0472b77304ec1f916245e; resolved:0--> <!--section:context-quote--> > **akedin** wrote: > Not sure if this will work, but I would try to change this map to `BiMap<FieldTypeDescriptor, SqlTypeName>`, so that it is possible to do it like this: > > ```java > BEAM_TO_CALCITE_MAPPING = > ... > .put(FieldType.DECIMAL.typeDescriptor(), SqlTypeName.DECIMAL) > .put(FieldType.STRING.typeDescriptor().withMetadata(CHAR_METADATA), SqlTypeName.CHAR) > .put(FieldType.STRING.typeDescriptor().withMetadata(VARCHAR_METADATA), SqlTypeName.VARCHAR) > .put(FieldType.DATETIME.typeDescriptor().withMetadata(TIME_METADATA), SqlTypeName.TIME) > .put(FieldType.DATETIME.typeDescriptor().withMetadata(TIMESTAMP_METADATA), SqlTypeName.TIMESTAMP) > ... > ``` > > Then you probably don't need to look up metadata separately each time and now have a 1-1 mapping <!--section:body--> Good idea: done. Required a bit more work since we need to support mapping beam type -> sql type when the metadata isn't set (in this case we have a default mapping - e.g. String -> VARCHAR). ---------------------------------------------------------------- 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 86470) > Support schema in PCollections > ------------------------------ > > Key: BEAM-3437 > URL: https://issues.apache.org/jira/browse/BEAM-3437 > Project: Beam > Issue Type: Wish > Components: beam-model > Reporter: Jean-Baptiste Onofré > Assignee: Jean-Baptiste Onofré > Priority: Major > Time Spent: 2.5h > Remaining Estimate: 0h > > As discussed with some people in the team, it would be great to add schema > support in {{PCollections}}. It will allow us: > 1. To expect some data type in {{PTransforms}} > 2. Improve some runners with additional features (I'm thinking about Spark > runner with data frames for instance). > A technical draft document has been created: > https://docs.google.com/document/d/1tnG2DPHZYbsomvihIpXruUmQ12pHGK0QIvXS1FOTgRc/edit?disco=AAAABhykQIs&ts=5a203b46&usp=comment_email_document > I also started a PoC on a branch, I will update this Jira with a "discussion" > PR. -- This message was sent by Atlassian JIRA (v7.6.3#76005)