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

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

                Author: ASF GitHub Bot
            Created on: 05/Mar/20 17:21
            Start Date: 05/Mar/20 17:21
    Worklog Time Spent: 10m 
      Work Description: reuvenlax commented on issue #11034: [BEAM-9424] Allow 
grouping by LogicalType
URL: https://github.com/apache/beam/pull/11034#issuecomment-595347570
 
 
   IT's not just performance critical code. We wanted these transforms to be
   usable by code that doesn't understand the logical type.
   
   For example: you should be able to write a SQL expression over a type that
   contains a OneOf. SQL doesn't understand the OneOf logical type of course,
   however the base type is a Row with nullable fields, Since today getValue
   returns the base type, you should be able to query it. Similar story with
   Enumerations - you should be able to treat them as integers in a SQL
   statement or in one of the schema transforms. (note: I"m not sure if this
   has been tested with SQL, so it may not work today).
   
   We need to figure out a good API that allows these transformation to use
   the base type by default but _also_ allows you to write a combiner that
   accepts the appropriate value type.
   
   Reuven
   
   On Thu, Mar 5, 2020 at 9:12 AM Gleb Kanterov <notificati...@github.com>
   wrote:
   
   > @reuvenlax <https://github.com/reuvenlax> is there an option to have
   > alternative to attachValues, but for getting values to be used in
   > performance-critical code?
   >
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub
   > 
<https://github.com/apache/beam/pull/11034?email_source=notifications&email_token=AFAYJVJTSD3CR6X326ZZTCLRF7MQPA5CNFSM4LAVOCP2YY3PNVWWK3TUL52HS4DFVREXG43VMVBW63LNMVXHJKTDN5WW2ZLOORPWSZGOEN6DQJI#issuecomment-595343397>,
   > or unsubscribe
   > 
<https://github.com/notifications/unsubscribe-auth/AFAYJVMJPB2WUZ2G6MLDOTLRF7MQPANCNFSM4LAVOCPQ>
   > .
   >
   
 
----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 398521)
    Time Spent: 4h 40m  (was: 4.5h)

> Grouping By LogicalTypes is not supported
> -----------------------------------------
>
>                 Key: BEAM-9424
>                 URL: https://issues.apache.org/jira/browse/BEAM-9424
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-core
>    Affects Versions: 2.19.0
>            Reporter: fdiazgon
>            Assignee: fdiazgon
>            Priority: Minor
>              Labels: sql
>          Time Spent: 4h 40m
>  Remaining Estimate: 0h
>
> Creating a schema from a BQ schema that has either TIME, DATE or DATETIME 
> columns, and grouping by one of these fields throws NullPointerException.
> {code:java}
> Pipeline pipeline = Pipeline.create();
> Schema beamSchemaWithLogicalTypes =
>     BigQueryUtils.fromTableSchema(
>         new TableSchema()
>             .setFields(
>                 Arrays.asList(
>                     new TableFieldSchema().setName("fTime").setType("TIME"),
>                     new TableFieldSchema().setName("fDate").setType("DATE"),
>                     new 
> TableFieldSchema().setName("fDatetime").setType("DATETIME"))));
> Row row =
>     Row.withSchema(beamSchemaWithLogicalTypes)
>         .addValues(
>             DateTime.parse("2020-02-02"),
>             DateTime.parse("2020-02-02"),
>             DateTime.parse("2020-02-02T00:00:00"))
>         .build();
> PCollection<Row> outputRow =
>     pipeline
>         .apply(Create.of(row))
>         .setRowSchema(beamSchemaWithLogicalTypes)
>         .apply(
>             SqlTransform.query(
>                 "SELECT fTime, fDate, fDatetime FROM PCOLLECTION GROUP BY 
> fTime, fDate, fDatetime"));
> pipeline.run();
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to