[
https://issues.apache.org/jira/browse/BEAM-6772?focusedWorklogId=212583&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-212583
]
ASF GitHub Bot logged work on BEAM-6772:
----------------------------------------
Author: ASF GitHub Bot
Created on: 13/Mar/19 18:16
Start Date: 13/Mar/19 18:16
Worklog Time Spent: 10m
Work Description: reuvenlax commented on issue #8006: [BEAM-6772] Change
Select semantics to match what a user expects
URL: https://github.com/apache/beam/pull/8006#issuecomment-472544613
Leaving the nested structure in place seem like it would be surprising to
users. For example:
Event:
location: ROW(Location)
A user would expect to be able to write the following:
pc.apply(ParDo.of(new DoFn....
@ProcessElement
public void process(@FieldAccess("location") Location loc) {
}));
(where Location is a POJO that matches the Location schema). However this
conversion will fail if the select actually returns an extra level of
nesting. In addition they would expect to be able to write:
pc.apply(ParDo.of(new DoFn....
@ProcessElement
public void process(@FieldAccess("location.lat") double lat) {
}));
But again, that will fail if the select leaves extra layers of nesting.
On Wed, Mar 13, 2019 at 10:59 AM Reuven Lax <[email protected]> wrote:
> what happens without UNNEST. If you simply do SELECT a.location?
>
> On Wed, Mar 13, 2019 at 10:55 AM Gleb Kanterov <[email protected]>
> wrote:
>
>> I checked a similar query in BigQuery:
>>
>> SELECT location FROM UNNEST([
>> STRUCT(STRUCT(1.0 as latitude, 2.0 as longtitude) as location, "abc" as
userId)
>> ]);
>>
>> It returns:
>>
>> [
>> {
>> "location": {
>> "latitude": "1.0",
>> "longtitude": "2.0"
>> }
>> }
>> ]
>>
>> I understand this PR, it make Beam to return:
>>
>> [{
>> "latitude": "1.0",
>> "longtitude": "2.0"
>> }]
>>
>> —
>> You are receiving this because you authored the thread.
>> Reply to this email directly, view it on GitHub
>> <https://github.com/apache/beam/pull/8006#issuecomment-472536279>, or mute
>> the thread
>>
<https://github.com/notifications/unsubscribe-auth/AUGE1cX8xq_UbvjQFrt1pu75bQm353eBks5vWTuegaJpZM4bh_oa>
>> .
>>
>
----------------------------------------------------------------
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:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 212583)
Time Spent: 1h 40m (was: 1.5h)
> Select transform has non-intuitive semantics
> --------------------------------------------
>
> Key: BEAM-6772
> URL: https://issues.apache.org/jira/browse/BEAM-6772
> Project: Beam
> Issue Type: Sub-task
> Components: sdk-java-core
> Reporter: Reuven Lax
> Assignee: Reuven Lax
> Priority: Major
> Time Spent: 1h 40m
> Remaining Estimate: 0h
>
> Consider the following schema:
> User:
> name: STRING
> location: Location
>
> Location:
> latitude: DOUBLE
> longitude: DOUBLE
>
> If you apply Select.fieldNames("location"), most users expect to get back a
> row matching the Location schema. Instead you get back an outer schema with a
> single location field in it. Select should instead unnest the output up to
> the point where multiple fields are selected.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)