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

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

                Author: ASF GitHub Bot
            Created on: 19/Jun/18 03:29
            Start Date: 19/Jun/18 03:29
    Worklog Time Spent: 10m 
      Work Description: amaliujia opened a new pull request #5682: [BEAM-4194] 
support unbounded limit
URL: https://github.com/apache/beam/pull/5682
 
 
   ##### What?
   Support unbounded limit in Beam SQL shell. In the past, due to default 
global window and default trigger, queries like "SELECT col_name FROM 
unbounded_table LIMIT 1" will not return in SQL Shell.
   
   This PR tries to support unbounded limit by starting a daemon thread to 
monitoring return value collection in BeanEnumerableConverter, and stop 
pipeline when collected values reach limit count. More detailed description can 
be found here: 
https://docs.google.com/document/d/13zeTewHH9nfwhSlcE4x77WQwr1U2Z4sTiNRjOXUj2aw/edit?usp=sharing.
   
   ##### Testing
    - [x] Unit test.
    
   Adding two unit tests to mock bounded and unbounded input tables to test 
LIMIT functionality.
   
    - [x] Integration test.
   
   Adding one e2e integration test, which utilizes Google Cloud Pub/Sub to test 
the unbounded limit on auto-generated Pub/Sub messages.
   
    - [x] Other test.
   
   Manually tested unbounded limit in Beam SQL shell on a Google Cloud Pub/Sub 
table. See the screenshot:
   <img width="1041" alt="screen shot 2018-06-18 at 3 48 07 pm" 
src="https://user-images.githubusercontent.com/1938382/41574436-106ca752-7336-11e8-91a7-33200513d4b8.png";>
   
   
   ------------------------
   
   Follow this checklist to help us incorporate your contribution quickly and 
easily:
   
    - [x] Format the pull request title like `[BEAM-XXX] Fixes bug in 
ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA 
issue, if applicable. This will automatically link the pull request to the 
issue.
    - [ ] If this contribution is large, please file an Apache [Individual 
Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   It will help us expedite review of your Pull Request if you tag someone 
(e.g. `@username`) to look at it.
   

----------------------------------------------------------------
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: 112986)
            Time Spent: 10m
    Remaining Estimate: 0h

> [SQL] Support LIMIT on Unbounded Data
> -------------------------------------
>
>                 Key: BEAM-4194
>                 URL: https://issues.apache.org/jira/browse/BEAM-4194
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql
>            Reporter: Anton Kedin
>            Assignee: Rui Wang
>            Priority: Major
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> We need to support queries with "LIMIT xxx".
> Problem is that we don't know when aggregates will trigger, they can 
> potentially accumulate values in global window and never trigger.
> If we have some trigger syntax (BEAM-4193), then the use case becomes similar 
> to what we have at the moment, where the user defines the trigger upstream 
> for all inputs. In this case LIMIT probably can be implemented as 
> sample.any(5) with trigger at count.



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

Reply via email to